Issue 130 added rsa key options to compute api

git-svn-id: http://jclouds.googlecode.com/svn/trunk@2734 3d8758e0-26b5-11de-8745-db77d3ebf521
This commit is contained in:
adrian.f.cole 2010-01-29 07:01:20 +00:00
parent d3b4383852
commit 24f1836f7a
39 changed files with 707 additions and 160 deletions

View File

@ -62,6 +62,7 @@ import org.jclouds.compute.domain.TemplateBuilder;
import org.jclouds.compute.domain.internal.NodeSetImpl;
import org.jclouds.compute.options.TemplateOptions;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.compute.util.ComputeUtils;
import org.jclouds.concurrent.ConcurrentUtils;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
@ -94,7 +95,7 @@ public class EC2ComputeService implements ComputeService {
@Override
public boolean apply(NodeMetadata from) {
return from.getTag().equals(tag);
return from.getTag().equals(tag) && from.getState() != NodeState.TERMINATED;
}
public NodeMatchesTag(String tag) {
@ -119,6 +120,7 @@ public class EC2ComputeService implements ComputeService {
protected final Predicate<RunningInstance> instanceStateTerminated;
protected final RunningInstanceToNodeMetadata runningInstanceToNodeMetadata;
protected final ExecutorService executor;
protected final ComputeUtils utils;
@Inject
public EC2ComputeService(EC2Client client, Provider<TemplateBuilder> templateBuilderProvider,
@ -128,7 +130,7 @@ public class EC2ComputeService implements ComputeService {
Map<RegionTag, KeyPairCredentials> credentialsMap,
Map<PortsRegionTag, String> securityGroupMap,
CreateKeyPairIfNeeded createKeyPairIfNeeded,
CreateSecurityGroupIfNeeded createSecurityGroupIfNeeded,
CreateSecurityGroupIfNeeded createSecurityGroupIfNeeded, ComputeUtils utils,
@Named("RUNNING") Predicate<RunningInstance> instanceStateRunning,
@Named("TERMINATED") Predicate<RunningInstance> instanceStateTerminated,
RunningInstanceToNodeMetadata runningInstanceToNodeMetadata,
@ -136,6 +138,7 @@ public class EC2ComputeService implements ComputeService {
this.templateBuilderProvider = templateBuilderProvider;
this.ec2Client = client;
this.images = images;
this.utils = utils;
this.sizes = sizes;
this.locations = locations;
this.credentialsMap = credentialsMap;
@ -149,7 +152,7 @@ public class EC2ComputeService implements ComputeService {
}
@Override
public NodeSet runNodesWithTag(String tag, int count, Template template) {
public NodeSet 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());
@ -188,9 +191,6 @@ public class EC2ComputeService implements ComputeService {
.withSecurityGroup(tag)// group I created above
.withAdditionalInfo(tag);
if (options.getRunScript() != null)
instanceOptions.withUserData(options.getRunScript());
Reservation reservation = ec2Client.getInstanceServices().runInstancesInRegion(region, zone,
template.getImage().getId(), 1, count, instanceOptions);
Iterable<String> ids = Iterables.transform(reservation, instanceToId);
@ -199,11 +199,30 @@ public class EC2ComputeService implements ComputeService {
logger.debug("<< started instances(%s)", idsString);
Iterables.all(reservation, instanceStateRunning);
logger.debug("<< running instances(%s)", idsString);
final Set<NodeMetadata> nodes = Sets.newHashSet();
Set<ListenableFuture<Void>> responses = Sets.newHashSet();
for (final NodeMetadata node : Iterables.transform(Iterables.concat(ec2Client
.getInstanceServices().describeInstancesInRegion(region,
Iterables.toArray(ids, String.class))), runningInstanceToNodeMetadata)) {
responses.add(ConcurrentUtils.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;
}
// refresh to get IP address
return new NodeSetImpl(Iterables.transform(Iterables.concat(ec2Client.getInstanceServices()
.describeInstancesInRegion(region, Iterables.toArray(ids, String.class))),
runningInstanceToNodeMetadata));
}), executor));
}
ConcurrentUtils.awaitCompletion(responses, executor, null, logger, "nodes");
return new NodeSetImpl(nodes);
}
@Override
@ -307,13 +326,14 @@ public class EC2ComputeService implements ComputeService {
logger.debug(">> terminating servers by tag(%s)", tag);
Set<ListenableFuture<Void>> responses = Sets.newHashSet();
for (final NodeMetadata node : doGetNodes(tag)) {
responses.add(ConcurrentUtils.makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
destroyNode(node);
return null;
}
}), executor));
if (node.getState() != NodeState.TERMINATED)
responses.add(ConcurrentUtils.makeListenable(executor.submit(new Callable<Void>() {
@Override
public Void call() throws Exception {
destroyNode(node);
return null;
}
}), executor));
}
ConcurrentUtils.awaitCompletion(responses, executor, null, logger, "nodes");
logger.debug("<< destroyed");

View File

@ -25,6 +25,7 @@ import java.net.URI;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -55,14 +56,19 @@ 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.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
import org.jclouds.logging.Logger;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.rest.RestContext;
import org.jclouds.ssh.SshClient;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.base.Splitter;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
@ -77,6 +83,13 @@ import com.google.inject.Provides;
* @author Adrian Cole
*/
public class EC2ComputeServiceContextModule extends EC2ContextModule {
@Provides
@Singleton
@Named("NOT_RUNNING")
protected Predicate<SshClient> runScriptRunning(RunScriptRunning stateRunning) {
return new RetryablePredicate<SshClient>(Predicates.not(stateRunning), 600, 3,
TimeUnit.SECONDS);
}
@Override
protected void configure() {
@ -183,6 +196,7 @@ public class EC2ComputeServiceContextModule extends EC2ContextModule {
for (final org.jclouds.aws.ec2.domain.Image from : sync.getAMIServices()
.describeImagesInRegion(region, ownedBy(amiOwners))) {
OsFamily os = null;
String name = null;
String osDescription = from.getImageLocation();
String version = "";
@ -190,7 +204,7 @@ public class EC2ComputeServiceContextModule extends EC2ContextModule {
if (matcher.find()) {
try {
os = OsFamily.fromValue(matcher.group(1));
matcher.group(2);// TODO no field for os version
name = matcher.group(2);// TODO no field for os version
version = matcher.group(3);
} catch (IllegalArgumentException e) {
holder.logger.debug("<< didn't match os(%s)", matcher.group(1));
@ -199,10 +213,10 @@ public class EC2ComputeServiceContextModule extends EC2ContextModule {
images
.add(new ImageImpl(
from.getId(),
from.getName(),
name,
region.toString(),
null,
ImmutableMap.<String, String> of(),
ImmutableMap.<String, String> of("owner", from.getImageOwnerId()),
from.getDescription(),
version,
os,

View File

@ -18,6 +18,7 @@
*/
package org.jclouds.aws.ec2.config;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.Date;
import java.util.Map;
@ -65,6 +66,7 @@ import org.jclouds.http.annotation.ClientError;
import org.jclouds.http.annotation.Redirection;
import org.jclouds.http.annotation.ServerError;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.predicates.SocketOpen;
import org.jclouds.rest.ConfiguresRestClient;
import org.jclouds.rest.RestClientFactory;
@ -98,6 +100,12 @@ public class EC2RestClientModule extends AbstractModule {
TimeUnit.MILLISECONDS);
}
@Provides
@Singleton
protected Predicate<InetSocketAddress> socketTester(SocketOpen open) {
return new RetryablePredicate<InetSocketAddress>(open, 130, 1, TimeUnit.SECONDS);
}
@Override
protected void configure() {
bindErrorHandlers();

View File

@ -34,6 +34,8 @@ public interface NodeMetadata extends ComputeMetadata {
* Tag used for all resources that belong to the same logical group. run, destroy commands are
* scoped to tag.
*
* @return tag for this node, or null, if not a part of a group
*
*/
String getTag();

View File

@ -90,4 +90,54 @@ public class ImageImpl extends ComputeMetadataImpl implements Image {
return architecture;
}
@Override
public String toString() {
return "[id=" + getId() + ", name=" + getName() + ", locationId=" + getLocationId()
+ ", architecture=" + architecture + ", osDescription=" + osDescription
+ ", osFamily=" + osFamily + ", version=" + version + "]";
}
@Override
public int hashCode() {
final int prime = 31;
int result = super.hashCode();
result = prime * result + ((architecture == null) ? 0 : architecture.hashCode());
result = prime * result + ((osDescription == null) ? 0 : osDescription.hashCode());
result = prime * result + ((osFamily == null) ? 0 : osFamily.hashCode());
result = prime * result + ((version == null) ? 0 : version.hashCode());
return result;
}
@Override
public boolean equals(Object obj) {
if (this == obj)
return true;
if (!super.equals(obj))
return false;
if (getClass() != obj.getClass())
return false;
ImageImpl other = (ImageImpl) obj;
if (architecture == null) {
if (other.architecture != null)
return false;
} else if (!architecture.equals(other.architecture))
return false;
if (osDescription == null) {
if (other.osDescription != null)
return false;
} else if (!osDescription.equals(other.osDescription))
return false;
if (osFamily == null) {
if (other.osFamily != null)
return false;
} else if (!osFamily.equals(other.osFamily))
return false;
if (version == null) {
if (other.version != null)
return false;
} else if (!version.equals(other.version))
return false;
return true;
}
}

View File

@ -52,11 +52,11 @@ public class NodeMetadataImpl extends ComputeMetadataImpl implements NodeMetadat
private final String tag;
public NodeMetadataImpl(String id, String name, String locationId, URI uri,
Map<String, String> userMetadata, String tag, NodeState state,
Map<String, String> userMetadata, @Nullable String tag, NodeState state,
Iterable<InetAddress> publicAddresses, Iterable<InetAddress> privateAddresses,
Map<String, String> extra, @Nullable Credentials credentials) {
super(ComputeType.NODE, id, name, locationId, uri, userMetadata);
this.tag = checkNotNull(tag, "tag");
this.tag = tag;
this.state = checkNotNull(state, "state");
Iterables.addAll(this.publicAddresses, checkNotNull(publicAddresses, "publicAddresses"));
Iterables.addAll(this.privateAddresses, checkNotNull(privateAddresses, "privateAddresses"));

View File

@ -217,8 +217,10 @@ 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.getOsDescription(), right.getOsDescription())
.compare(left.getVersion(), right.getVersion()).result();
return ComparisonChain.start().compare(left.getArchitecture(), right.getArchitecture()).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();
}
};

View File

@ -6,7 +6,8 @@ import static com.google.common.base.Preconditions.checkNotNull;
/**
* Contains options supported in the {@code ComputeService#runNode} operation. <h2>
* Usage</h2> The recommended way to instantiate a TemplateOptions object is to statically import
* TemplateOptions.* and invoke a static creation method followed by an instance mutator (if needed):
* TemplateOptions.* and invoke a static creation method followed by an instance mutator (if
* needed):
* <p/>
* <code>
* import static org.jclouds.compute.options.TemplateOptions.Builder.*;
@ -25,6 +26,10 @@ public class TemplateOptions {
private byte[] script;
private String privateKey;
private String publicKey;
public int[] getInboundPorts() {
return inboundPorts;
}
@ -33,6 +38,14 @@ public class TemplateOptions {
return script;
}
public String getPrivateKey() {
return privateKey;
}
public String getPublicKey() {
return publicKey;
}
/**
* This script will be executed as the root user upon system startup.
*/
@ -43,6 +56,27 @@ public class TemplateOptions {
return this;
}
/**
* replaces the rsa ssh key used at login.
*/
public TemplateOptions installPrivateKey(String privateKey) {
checkArgument(checkNotNull(privateKey, "privateKey").startsWith(
"-----BEGIN RSA PRIVATE KEY-----"),
"key should start with -----BEGIN RSA PRIVATE KEY-----");
this.privateKey = privateKey;
return this;
}
/**
* authorized an rsa ssh key.
*/
public TemplateOptions authorizePublicKey(String publicKey) {
checkArgument(checkNotNull(publicKey, "publicKey").startsWith("ssh-rsa"),
"key should start with ssh-rsa");
this.publicKey = publicKey;
return this;
}
/**
* Opens the set of ports to public access.
*/
@ -69,5 +103,21 @@ public class TemplateOptions {
return options.runScript(script);
}
/**
* @see TemplateOptions#installPrivateKey
*/
public static TemplateOptions installPrivateKey(String rsaKey) {
TemplateOptions options = new TemplateOptions();
return options.installPrivateKey(rsaKey);
}
/**
* @see TemplateOptions#authorizePublicKey
*/
public static TemplateOptions authorizePublicKey(String rsaKey) {
TemplateOptions options = new TemplateOptions();
return options.authorizePublicKey(rsaKey);
}
}
}

View File

@ -0,0 +1,56 @@
/**
*
* 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.predicates;
import javax.annotation.Resource;
import javax.inject.Singleton;
import org.jclouds.logging.Logger;
import org.jclouds.ssh.ExecResponse;
import org.jclouds.ssh.SshClient;
import com.google.common.base.Predicate;
/**
*
* Tests to if the runscript is still running
*
* @author Adrian Cole
*/
@Singleton
public class RunScriptRunning implements Predicate<SshClient> {
@Resource
protected Logger logger = Logger.NULL;
public boolean apply(SshClient ssh) {
logger.trace("looking for runscript state on %s@%s", ssh.getUsername(), ssh.getHostAddress());
ExecResponse response = refresh(ssh);
while (response.getExitCode() == -1)
response = refresh(ssh);
logger.trace("%s@%s: looking for exit code 0: currently: %s", ssh.getUsername(), ssh
.getHostAddress(), response.getExitCode());
return 0 == response.getExitCode();
}
private ExecResponse refresh(SshClient ssh) {
return ssh.exec("./runscript.sh status");
}
}

View File

@ -18,6 +18,7 @@
*/
package org.jclouds.compute.util;
import static com.google.common.base.Preconditions.checkNotNull;
import static com.google.common.base.Preconditions.checkState;
import java.io.ByteArrayInputStream;
@ -25,20 +26,36 @@ import java.net.ConnectException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.Comparator;
import java.util.List;
import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import javax.annotation.Nullable;
import javax.annotation.Resource;
import javax.inject.Named;
import org.jclouds.Constants;
import org.jclouds.compute.domain.ComputeMetadata;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.options.TemplateOptions;
import org.jclouds.compute.reference.ComputeServiceConstants;
import org.jclouds.concurrent.ConcurrentUtils;
import org.jclouds.logging.Logger;
import org.jclouds.scriptbuilder.InitBuilder;
import org.jclouds.scriptbuilder.domain.OsFamily;
import org.jclouds.ssh.ExecResponse;
import org.jclouds.ssh.SshClient;
import com.google.common.base.Function;
import com.google.common.base.Predicate;
import com.google.common.base.Splitter;
import com.google.common.base.Throwables;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Sets;
import com.google.common.util.concurrent.ListenableFuture;
import com.google.inject.Inject;
/**
@ -51,7 +68,9 @@ public class ComputeUtils {
protected Logger logger = Logger.NULL;
@Inject(optional = true)
private SshClient.Factory sshFactory;
protected final Predicate<SshClient> runScriptNotRunning;
private final Predicate<InetSocketAddress> socketTester;
private final ExecutorService executor;
public static Function<ComputeMetadata, String> METADATA_TO_ID = new Function<ComputeMetadata, String>() {
@Override
@ -61,8 +80,12 @@ public class ComputeUtils {
};
@Inject
public ComputeUtils(Predicate<InetSocketAddress> socketTester) {
public ComputeUtils(Predicate<InetSocketAddress> socketTester,
@Named("NOT_RUNNING") Predicate<SshClient> runScriptNotRunning,
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
this.socketTester = socketTester;
this.runScriptNotRunning = runScriptNotRunning;
this.executor = executor;
}
public static Iterable<? extends ComputeMetadata> filterByName(
@ -84,7 +107,36 @@ public class ComputeUtils {
};
public void runScriptOnNode(NodeMetadata node, byte[] script) {
public void runOptionsOnNode(NodeMetadata node, TemplateOptions options) {
List<SshCallable<?>> callables = Lists.newArrayList();
if (options.getRunScript() != null) {
callables.add(runScriptOnNode(node, "runscript.sh", options.getRunScript()));
}
if (options.getPublicKey() != null) {
callables.add(authorizeKeyOnNode(node, options.getPublicKey()));
}
// changing the key "MUST" come last or else the other commands may fail.
if (callables.size() > 0 || options.getPrivateKey() != null) {
runCallablesOnNode(node, callables, options.getPrivateKey() != null ? installKeyOnNode(
node, options.getPrivateKey()) : null);
}
}
public InstallRSAPrivateKey installKeyOnNode(NodeMetadata node, String privateKey) {
return new InstallRSAPrivateKey(node, privateKey);
}
public AuthorizeRSAPublicKey authorizeKeyOnNode(NodeMetadata node, String publicKey) {
return new AuthorizeRSAPublicKey(node, publicKey);
}
public RunScriptOnNode runScriptOnNode(NodeMetadata node, String scriptName, byte[] script) {
return new RunScriptOnNode(runScriptNotRunning, node, scriptName, script);
}
public void runCallablesOnNode(NodeMetadata node, Iterable<? extends SshCallable<?>> parallel,
@Nullable SshCallable<?> last) {
checkState(this.sshFactory != null, "runScript requested, but no SshModule configured");
InetSocketAddress socket = new InetSocketAddress(Iterables.get(node.getPublicAddresses(), 0),
@ -96,11 +148,26 @@ public class ComputeUtils {
for (int i = 0; i < 3; i++) {
try {
ssh.connect();
runScriptOnNodeWithClient(ssh, node, script);
Set<ListenableFuture<?>> responses = Sets.newHashSet();
for (SshCallable<?> callable : parallel) {
callable.setConnection(ssh, logger);
responses.add(ConcurrentUtils.makeListenable(executor.submit(callable), executor));
}
ConcurrentUtils.awaitCompletion(responses, executor, null, logger, "ssh");
if (last != null) {
last.setConnection(ssh, logger);
try {
last.call();
} catch (Exception e) {
Throwables.propagate(e);
}
}
break;
} catch (RuntimeException from) {
if (Iterables.size(Iterables.filter(Throwables.getCausalChain(from),
ConnectException.class)) >= 1) {
ConnectException.class)) >= 1// auth fail sometimes happens in EC2
|| Throwables.getRootCause(from).getMessage().indexOf("Auth fail") != -1
|| Throwables.getRootCause(from).getMessage().indexOf("invalid privatekey") != -1) {
try {
Thread.sleep(100);
} catch (InterruptedException e) {
@ -116,22 +183,120 @@ public class ComputeUtils {
}
}
private void runScriptOnNodeWithClient(SshClient ssh, NodeMetadata node, byte[] script) {
String scriptName = node.getId() + ".sh";
ssh.put(scriptName, new ByteArrayInputStream(script));
ssh.exec("chmod 755 " + scriptName);
if (node.getCredentials().account.equals("root")) {
logger.debug(">> running %s as %s", scriptName, node.getCredentials().account);
logger.debug("<< complete(%d)", ssh.exec("./" + scriptName).getExitCode());
} else if (isKeyAuth(node)) {
logger.debug(">> running sudo %s as %s", scriptName, node.getCredentials().account);
logger.debug("<< complete(%d)", ssh.exec("sudo ./" + scriptName).getExitCode());
} else {
logger.debug(">> running sudo -S %s as %s", scriptName, node.getCredentials().account);
logger.debug("<< complete(%d)", ssh.exec(
String.format("echo %s|sudo -S ./%s", node.getCredentials().key, scriptName))
.getExitCode());
public static interface SshCallable<T> extends Callable<T> {
void setConnection(SshClient ssh, Logger logger);
}
public static class RunScriptOnNode implements SshCallable<ExecResponse> {
private SshClient ssh;
protected final Predicate<SshClient> runScriptNotRunning;
private final NodeMetadata node;
private final String scriptName;
private final byte[] script;
private Logger logger = Logger.NULL;
RunScriptOnNode(@Named("NOT_RUNNING") Predicate<SshClient> runScriptNotRunning,
NodeMetadata node, String scriptName, byte[] script) {
this.runScriptNotRunning = runScriptNotRunning;
this.node = checkNotNull(node, "node");
this.scriptName = checkNotNull(scriptName, "scriptName");
this.script = new InitBuilder("runscript", "/tmp", "/tmp", ImmutableMap
.<String, String> of(), Iterables.toArray(Splitter.on("\n").split(
new String(checkNotNull(script, "script"))), String.class)).build(OsFamily.UNIX)
.getBytes();
}
@Override
public ExecResponse call() throws Exception {
ssh.put(scriptName, new ByteArrayInputStream(script));
ExecResponse returnVal = ssh.exec("chmod 755 " + scriptName);
returnVal = ssh.exec("./" + scriptName + " init");
if (node.getCredentials().account.equals("root")) {
logger.debug(">> running %s as %s@%s", scriptName, node.getCredentials().account,
Iterables.get(node.getPublicAddresses(), 0).getHostAddress());
returnVal = ssh.exec("./" + scriptName + " start");
} else if (isKeyAuth(node)) {
logger.debug(">> running sudo %s as %s@%s", scriptName, node.getCredentials().account,
Iterables.get(node.getPublicAddresses(), 0).getHostAddress());
returnVal = ssh.exec("sudo ./" + scriptName + " start");
} else {
logger.debug(">> running sudo -S %s as %s@%s", scriptName,
node.getCredentials().account, Iterables.get(node.getPublicAddresses(), 0)
.getHostAddress());
returnVal = ssh.exec(String.format("echo %s|sudo -S ./%s", node.getCredentials().key,
scriptName + " start"));
}
runScriptNotRunning.apply(ssh);
logger.debug("<< complete(%d)", returnVal.getExitCode());
return returnVal;
}
@Override
public void setConnection(SshClient ssh, Logger logger) {
this.logger = checkNotNull(logger, "logger");
this.ssh = checkNotNull(ssh, "ssh");
}
}
public static class InstallRSAPrivateKey implements SshCallable<ExecResponse> {
private SshClient ssh;
private final NodeMetadata node;
private final String privateKey;
private Logger logger = Logger.NULL;
InstallRSAPrivateKey(NodeMetadata node, String privateKey) {
this.node = checkNotNull(node, "node");
this.privateKey = checkNotNull(privateKey, "privateKey");
}
@Override
public ExecResponse call() throws Exception {
ssh.exec("mkdir .ssh");
ssh.put(".ssh/id_rsa", new ByteArrayInputStream(privateKey.getBytes()));
logger.debug(">> installing rsa key for %s@%s", node.getCredentials().account, Iterables
.get(node.getPublicAddresses(), 0).getHostAddress());
return ssh.exec("chmod 600 .ssh/id_rsa");
}
@Override
public void setConnection(SshClient ssh, Logger logger) {
this.logger = checkNotNull(logger, "logger");
this.ssh = checkNotNull(ssh, "ssh");
}
}
public static class AuthorizeRSAPublicKey implements SshCallable<ExecResponse> {
private SshClient ssh;
private final NodeMetadata node;
private final String publicKey;
private Logger logger = Logger.NULL;
AuthorizeRSAPublicKey(NodeMetadata node, String publicKey) {
this.node = checkNotNull(node, "node");
this.publicKey = checkNotNull(publicKey, "publicKey");
}
@Override
public ExecResponse call() throws Exception {
ssh.exec("mkdir .ssh");
ssh.put(".ssh/id_rsa.pub", new ByteArrayInputStream(publicKey.getBytes()));
logger.debug(">> authorizing rsa public key for %s@%s", node.getCredentials().account,
Iterables.get(node.getPublicAddresses(), 0).getHostAddress());
ExecResponse returnVal = ssh.exec("cat .ssh/id_rsa.pub >> .ssh/authorized_keys");
returnVal = ssh.exec("chmod 600 .ssh/authorized_keys");
logger.debug("<< complete(%d)", returnVal.getExitCode());
return returnVal;
}
@Override
public void setConnection(SshClient ssh, Logger logger) {
this.logger = checkNotNull(logger, "logger");
this.ssh = checkNotNull(ssh, "ssh");
}
}
public static boolean isKeyAuth(NodeMetadata createdNode) {

View File

@ -20,6 +20,7 @@ package org.jclouds.ssh;
import java.io.InputStream;
import java.net.InetSocketAddress;
import java.util.Map;
import javax.annotation.PostConstruct;
import javax.annotation.PreDestroy;
@ -33,8 +34,14 @@ public interface SshClient {
SshClient create(InetSocketAddress socket, String username, String password);
SshClient create(InetSocketAddress socket, String username, byte[] privateKey);
Map<String, String> generateRSAKeyPair(String comment, String passphrase);
}
String getUsername();
String getHostAddress();
void put(String path, InputStream contents);
InputStream get(String path);

View File

@ -22,8 +22,10 @@ import static com.google.common.base.Preconditions.checkNotNull;
import static org.testng.Assert.assertEquals;
import static org.testng.Assert.assertNotNull;
import java.io.File;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.Map;
import java.util.NoSuchElementException;
import java.util.SortedSet;
import java.util.Map.Entry;
@ -37,7 +39,6 @@ import org.jclouds.compute.domain.Image;
import org.jclouds.compute.domain.NodeMetadata;
import org.jclouds.compute.domain.NodeSet;
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.TemplateBuilder;
@ -46,8 +47,6 @@ import org.jclouds.http.HttpResponseException;
import org.jclouds.logging.log4j.config.Log4JLoggingModule;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.predicates.SocketOpen;
import org.jclouds.scriptbuilder.ScriptBuilder;
import org.jclouds.scriptbuilder.domain.Statements;
import org.jclouds.ssh.ExecResponse;
import org.jclouds.ssh.SshClient;
import org.jclouds.ssh.SshException;
@ -56,10 +55,13 @@ import org.testng.annotations.BeforeClass;
import org.testng.annotations.BeforeGroups;
import org.testng.annotations.Test;
import com.google.common.base.Charsets;
import com.google.common.base.Predicate;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import com.google.common.collect.Sets;
import com.google.common.io.Files;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Module;
@ -83,7 +85,8 @@ public abstract class BaseComputeServiceLiveTest {
protected ComputeService client;
protected String user;
protected String password;
private Template template;
protected Template template;
protected Map<String, String> keyPair;
@BeforeGroups(groups = { "live" })
public void setupClient() throws InterruptedException, ExecutionException, TimeoutException,
@ -92,6 +95,15 @@ public abstract class BaseComputeServiceLiveTest {
tag = checkNotNull(service, "service");
user = checkNotNull(System.getProperty("jclouds.test.user"), "jclouds.test.user");
password = checkNotNull(System.getProperty("jclouds.test.key"), "jclouds.test.key");
String secretKeyFile;
try {
secretKeyFile = checkNotNull(System.getProperty("jclouds.test.ssh.keyfile"),
"jclouds.test.ssh.keyfile");
} catch (NullPointerException e) {
secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
}
String secret = Files.toString(new File(secretKeyFile), Charsets.UTF_8);
assert secret.startsWith("-----BEGIN RSA PRIVATE KEY-----") : "invalid key:\n" + secret;
context = new ComputeServiceContextFactory().createContext(service, user, password,
ImmutableSet.of(new Log4JLoggingModule(), getSshModule()));
Injector injector = Guice.createInjector(getSshModule());
@ -100,11 +112,9 @@ public abstract class BaseComputeServiceLiveTest {
socketTester = new RetryablePredicate<InetSocketAddress>(socketOpen, 60, 1, TimeUnit.SECONDS);
injector.injectMembers(socketOpen); // add logger
client = context.getComputeService();
}
protected boolean canRunScript(Template template) {
return template.getImage().getOsFamily() == OsFamily.UBUNTU
|| template.getImage().getOsFamily() == OsFamily.JEOS;
// keyPair = sshFactory.generateRSAKeyPair("", "");
keyPair = ImmutableMap.<String, String> of("private", secret, "public", Files.toString(
new File(secretKeyFile + ".pub"), Charsets.UTF_8));
}
abstract protected Module getSshModule();
@ -120,30 +130,21 @@ public abstract class BaseComputeServiceLiveTest {
}
template = buildTemplate(client.templateBuilder());
if (canRunScript(template))
template
.getOptions()
.runScript(
new ScriptBuilder()
// update add dns and install jdk
.addStatement(
Statements
.exec("echo nameserver 208.67.222.222 >> /etc/resolv.conf"))
.addStatement(Statements.exec("apt-get update"))
//
.addStatement(Statements.exec("apt-get upgrade -y"))
//
.addStatement(
Statements.exec("apt-get install -y openjdk-6-jdk"))
//
.addStatement(
Statements
.exec("wget -qO/usr/bin/runurl run.alestic.com/runurl"))
//
.addStatement(Statements.exec("chmod 755 /usr/bin/runurl"))
//
.build(org.jclouds.scriptbuilder.domain.OsFamily.UNIX)
.getBytes());
template
.getOptions()
.installPrivateKey(keyPair.get("private"))
.authorizePublicKey(keyPair.get("public"))
.runScript(
new StringBuilder()//
.append("echo nameserver 208.67.222.222 >> /etc/resolv.conf\n")//
.append("cp /etc/apt/sources.list /etc/apt/sources.list.old\n")//
.append(
"sed 's~us.archive.ubuntu.com~mirror.anl.gov/pub~g' /etc/apt/sources.list.old >/etc/apt/sources.list\n")//
.append("apt-get update\n")//
.append("apt-get install -f -y --force-yes openjdk-6-jdk\n")//
.append("wget -qO/usr/bin/runurl run.alestic.com/runurl\n")//
.append("chmod 755 /usr/bin/runurl\n")//
.toString().getBytes());
nodes = Sets.newTreeSet(client.runNodesWithTag(tag, 2, template));
assertEquals(nodes.size(), 2);
for (NodeMetadata node : nodes) {
@ -202,7 +203,7 @@ public abstract class BaseComputeServiceLiveTest {
for (Entry<String, ? extends Image> image : client.getImages().entrySet()) {
assertEquals(image.getKey(), image.getValue().getId());
assert image.getValue().getId() != null : image;
assert image.getValue().getLocationId() != null : image;
// image.getValue().getLocationId() can be null, if it is a location-free image
assertEquals(image.getValue().getType(), ComputeType.IMAGE);
}
}
@ -248,16 +249,14 @@ public abstract class BaseComputeServiceLiveTest {
22);
socketTester.apply(socket); // TODO add transitionTo option that accepts a socket conection
// state.
SshClient ssh = node.getCredentials().key.startsWith("-----BEGIN RSA PRIVATE KEY-----") ? sshFactory
.create(socket, node.getCredentials().account, node.getCredentials().key.getBytes())
: sshFactory
.create(socket, node.getCredentials().account, node.getCredentials().key);
SshClient ssh = sshFactory.create(socket, node.getCredentials().account, keyPair.get(
"private").getBytes());
try {
ssh.connect();
ExecResponse hello = ssh.exec("echo hello");
assertEquals(hello.getOutput().trim(), "hello");
if (canRunScript(template))
System.out.println(ssh.exec("java -version"));
ExecResponse exec = ssh.exec("java -version");
assert exec.getError().indexOf("OpenJDK") != -1 : exec;
} finally {
if (ssh != null)
ssh.disconnect();

View File

@ -101,9 +101,8 @@ public class ConcurrentUtils {
}
private static String message(String prefix, int size, int complete, int errors, long start) {
return String
.format("%s, completed: %d/%d, errors: %d, rate: %fms/op%n", prefix, complete,
size, errors, ((System.currentTimeMillis() - start) / ((double) size)));
return String.format("%s, completed: %d/%d, errors: %d, rate: %dms/op%n", prefix, complete,
size, errors, (long) ((System.currentTimeMillis() - start) / ((double) size)));
}
protected static boolean timeOut(long start, Long maxTime) {

View File

@ -215,4 +215,14 @@ public class JschSshClient implements SshClient {
}
}
@Override
public String getHostAddress() {
return this.host.getHostAddress();
}
@Override
public String getUsername() {
return this.username;
}
}

View File

@ -18,14 +18,21 @@
*/
package org.jclouds.ssh.jsch.config;
import java.io.ByteArrayOutputStream;
import java.net.InetSocketAddress;
import java.util.Map;
import org.jclouds.ssh.ConfiguresSshClient;
import org.jclouds.ssh.SshClient;
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.Scopes;
import com.jcraft.jsch.JSch;
import com.jcraft.jsch.JSchException;
import com.jcraft.jsch.KeyPair;
/**
*
@ -48,5 +55,23 @@ public class JschSshClientModule extends AbstractModule {
return new JschSshClient(socket, username, privateKey);
}
@Override
public Map<String, String> generateRSAKeyPair(String comment, String passphrase) {
KeyPair pair = null;
try {
pair = KeyPair.genKeyPair(new JSch(), KeyPair.RSA);
} catch (JSchException e) {
Throwables.propagate(e);
}
if (passphrase != null)
pair.setPassphrase(passphrase);
ByteArrayOutputStream privateKey = new ByteArrayOutputStream();
pair.writePrivateKey(privateKey);
ByteArrayOutputStream publicKey = new ByteArrayOutputStream();
pair.writePublicKey(publicKey, comment);
return ImmutableMap.of("comment", comment, "passphrase", passphrase, "private",
new String(privateKey.toByteArray()), "public", new String(publicKey
.toByteArray()));
}
}
}

View File

@ -95,6 +95,16 @@ public class JschSshClientLiveTest {
}
@Override
public String getHostAddress() {
return null;
}
@Override
public String getUsername() {
return null;
}
};
} else {
Injector i = Guice.createInjector(new JschSshClientModule());

View File

@ -21,6 +21,7 @@ package org.jclouds.ssh.jsch.config;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.net.UnknownHostException;
import java.util.Map;
import org.jclouds.ssh.SshClient;
import org.jclouds.ssh.jsch.JschSshClient;
@ -28,6 +29,7 @@ import org.testng.annotations.Test;
import com.google.inject.Guice;
import com.google.inject.Injector;
import static org.testng.Assert.assertEquals;
/**
* Tests the ability to configure a {@link JschSshClient}
@ -44,5 +46,10 @@ public class JschSshClientModuleTest {
SshClient connection = factory.create(new InetSocketAddress(InetAddress.getLocalHost(), 22),
"username", "password");
assert connection instanceof JschSshClient;
Map<String, String> keyPair = factory.generateRSAKeyPair("comment", "hola");
assertEquals(keyPair.get("comment"), "comment");
assertEquals(keyPair.get("passphrase"), "hola");
assert keyPair.get("private").indexOf("-----BEGIN RSA PRIVATE KEY-----") == 0 : keyPair;
assert keyPair.get("public").indexOf("ssh-rsa ") == 0 : keyPair;
}
}

View File

@ -170,6 +170,7 @@
<jclouds.wire.httpstream.url>http://apache.opensourceresources.org/commons/logging/binaries/commons-logging-1.1.1-bin.tar.gz</jclouds.wire.httpstream.url>
<jclouds.wire.httpstream.md5>e5de09672af9b386c30a311654d8541a</jclouds.wire.httpstream.md5>
<jclouds.test.listener>org.jclouds.test.testng.UnitTestStatusListener</jclouds.test.listener>
<jclouds.test.ssh.keyfile />
</properties>
<dependencyManagement>
@ -473,6 +474,10 @@ pageTracker._trackPageview();
<name>jclouds.test.appid</name>
<value>${jclouds.test.appid}</value>
</property>
<property>
<name>jclouds.test.ssh.keyfile</name>
<value>${jclouds.test.ssh.keyfile}</value>
</property>
<property>
<name>jclouds.blobstore.httpstream.url</name>
<value>${jclouds.blobstore.httpstream.url}</value>

View File

@ -61,7 +61,8 @@ public class ParseObjectFromHeadersAndHttpContent implements Function<HttpRespon
public CFObject apply(HttpResponse from) {
CFObject object = objectProvider.create(infoParser.apply(from));
addAllHeadersTo(from, object);
object.setPayload(from.getContent());
if (from.getContent() != null)
object.setPayload(from.getContent());
attemptToParseSizeAndRangeFromHeaders(from, object);
return object;
}

View File

@ -44,7 +44,7 @@ public class BindRebootTypeToJsonPayload extends BindToJsonPayload {
@Override
public void bindToRequest(HttpRequest request, Object toBind) {
checkArgument(toBind instanceof RebootType, "this binder is only valid for RebootTypes!");
super.bindToRequest(request, ImmutableMap.of("reboot", ImmutableMap.of("flavor",
checkNotNull(toBind, "flavor"))));
super.bindToRequest(request, ImmutableMap.of("reboot", ImmutableMap.of("type",
checkNotNull(toBind, "type"))));
}
}

View File

@ -177,14 +177,14 @@ public class CloudServersComputeService implements ComputeService {
.<String, String> of(), new Credentials("root", server
.getAdminPass()));
nodes.add(node);
logger.debug("<< started server(%s)", server.getId());
logger.debug("<< started server(%s)", node.getId());
serverActive.apply(server);
logger.debug("<< running server(%s)", server.getId());
if (template.getOptions().getRunScript() != null) {
utils.runScriptOnNode(node, template.getOptions().getRunScript());
}
logger.debug("<< running server(%s)", node.getId());
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied server(%s)", node.getId());
return null;
}
}), executor));
}
ConcurrentUtils.awaitCompletion(responses, executor, null, logger, "nodes");

View File

@ -22,6 +22,7 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -41,11 +42,13 @@ import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.internal.ImageImpl;
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.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
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;
@ -53,8 +56,11 @@ import org.jclouds.rackspace.cloudservers.config.CloudServersContextModule;
import org.jclouds.rackspace.cloudservers.domain.Flavor;
import org.jclouds.rackspace.cloudservers.options.ListOptions;
import org.jclouds.rest.RestContext;
import org.jclouds.ssh.SshClient;
import com.google.common.base.Function;
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.Maps;
@ -83,6 +89,14 @@ public class CloudServersComputeServiceContextModule extends CloudServersContext
computeService, context);
}
@Provides
@Singleton
@Named("NOT_RUNNING")
protected Predicate<SshClient> runScriptRunning(RunScriptRunning stateRunning) {
return new RetryablePredicate<SshClient>(Predicates.not(stateRunning), 600, 3,
TimeUnit.SECONDS);
}
@Provides
@Singleton
Location getRegion() {

View File

@ -18,28 +18,20 @@
*/
package org.jclouds.rackspace.cloudservers.config;
import java.net.InetSocketAddress;
import java.net.URI;
import java.util.concurrent.TimeUnit;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.http.RequiresHttp;
import org.jclouds.lifecycle.Closer;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.predicates.SocketOpen;
import org.jclouds.rackspace.CloudServers;
import org.jclouds.rackspace.cloudservers.CloudServersAsyncClient;
import org.jclouds.rackspace.cloudservers.CloudServersClient;
import org.jclouds.rackspace.cloudservers.domain.Server;
import org.jclouds.rackspace.cloudservers.predicates.ServerActive;
import org.jclouds.rackspace.cloudservers.predicates.ServerDeleted;
import org.jclouds.rackspace.reference.RackspaceConstants;
import org.jclouds.rest.RestContext;
import org.jclouds.rest.internal.RestContextImpl;
import com.google.common.base.Predicate;
import com.google.inject.AbstractModule;
import com.google.inject.Provides;
@ -50,26 +42,6 @@ public class CloudServersContextModule extends AbstractModule {
protected void configure() {
}
@Provides
@Singleton
@Named("ACTIVE")
protected Predicate<Server> serverRunning(ServerActive stateRunning) {
return new RetryablePredicate<Server>(stateRunning, 600, 1, TimeUnit.SECONDS);
}
@Provides
@Singleton
@Named("DELETED")
protected Predicate<Server> serverDeleted(ServerDeleted stateDeleted) {
return new RetryablePredicate<Server>(stateDeleted, 600, 50, TimeUnit.MILLISECONDS);
}
@Provides
@Singleton
protected Predicate<InetSocketAddress> socketTester(SocketOpen open) {
return new RetryablePredicate<InetSocketAddress>(open, 130, 1, TimeUnit.SECONDS);
}
@Provides
@Singleton
RestContext<CloudServersAsyncClient, CloudServersClient> provideContext(Closer closer,

View File

@ -18,15 +18,30 @@
*/
package org.jclouds.rackspace.cloudservers.config;
import java.net.InetSocketAddress;
import java.util.concurrent.TimeUnit;
import javax.inject.Named;
import javax.inject.Singleton;
import org.jclouds.concurrent.internal.SyncProxy;
import org.jclouds.http.HttpErrorHandler;
import org.jclouds.http.RequiresHttp;
import org.jclouds.http.annotation.ClientError;
import org.jclouds.http.annotation.Redirection;
import org.jclouds.http.annotation.ServerError;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.predicates.SocketOpen;
import org.jclouds.rackspace.cloudservers.CloudServersAsyncClient;
import org.jclouds.rackspace.cloudservers.CloudServersClient;
import org.jclouds.rackspace.cloudservers.domain.Server;
import org.jclouds.rackspace.cloudservers.handlers.ParseCloudServersErrorFromHttpResponse;
import org.jclouds.rackspace.cloudservers.predicates.ServerActive;
import org.jclouds.rackspace.cloudservers.predicates.ServerDeleted;
import org.jclouds.rest.ConfiguresRestClient;
import org.jclouds.rest.RestClientFactory;
import com.google.common.base.Predicate;
import com.google.inject.AbstractModule;
import com.google.inject.Provides;
@ -39,7 +54,27 @@ import com.google.inject.Provides;
public class CloudServersRestClientModule extends AbstractModule {
@Override
protected void configure() {
bindErrorHandlers();
}
@Provides
@Singleton
@Named("ACTIVE")
protected Predicate<Server> serverRunning(ServerActive stateRunning) {
return new RetryablePredicate<Server>(stateRunning, 600, 1, TimeUnit.SECONDS);
}
@Provides
@Singleton
@Named("DELETED")
protected Predicate<Server> serverDeleted(ServerDeleted stateDeleted) {
return new RetryablePredicate<Server>(stateDeleted, 600, 50, TimeUnit.MILLISECONDS);
}
@Provides
@Singleton
protected Predicate<InetSocketAddress> socketTester(SocketOpen open) {
return new RetryablePredicate<InetSocketAddress>(open, 130, 1, TimeUnit.SECONDS);
}
@Provides
@ -54,4 +89,13 @@ public class CloudServersRestClientModule extends AbstractModule {
throws IllegalArgumentException, SecurityException, NoSuchMethodException {
return SyncProxy.create(CloudServersClient.class, client);
}
protected void bindErrorHandlers() {
bind(HttpErrorHandler.class).annotatedWith(Redirection.class).to(
ParseCloudServersErrorFromHttpResponse.class);
bind(HttpErrorHandler.class).annotatedWith(ClientError.class).to(
ParseCloudServersErrorFromHttpResponse.class);
bind(HttpErrorHandler.class).annotatedWith(ServerError.class).to(
ParseCloudServersErrorFromHttpResponse.class);
}
}

View File

@ -0,0 +1,68 @@
package org.jclouds.rackspace.cloudservers.handlers;
import java.io.IOException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import javax.annotation.Resource;
import org.jclouds.http.HttpCommand;
import org.jclouds.http.HttpErrorHandler;
import org.jclouds.http.HttpResponse;
import org.jclouds.http.HttpResponseException;
import org.jclouds.logging.Logger;
import org.jclouds.rest.AuthorizationException;
import org.jclouds.rest.ResourceNotFoundException;
import org.jclouds.util.Utils;
import com.google.common.io.Closeables;
/**
* This will parse and set an appropriate exception on the command object.
*
* @author Adrian Cole
*
*/
public class ParseCloudServersErrorFromHttpResponse implements HttpErrorHandler {
@Resource
protected Logger logger = Logger.NULL;
public static final Pattern RESOURCE_PATTERN = Pattern
.compile("^/v1[^/]*/[0-9]+/([^/]+)/([0-9]+)");
public void handleError(HttpCommand command, HttpResponse response) {
Exception exception = new HttpResponseException(command, response);
try {
switch (response.getStatusCode()) {
case 401:
exception = new AuthorizationException(command.getRequest().getRequestLine());
break;
case 404:
if (!command.getRequest().getMethod().equals("DELETE")) {
String path = command.getRequest().getEndpoint().getPath();
Matcher matcher = RESOURCE_PATTERN.matcher(path);
String message;
if (matcher.find()) {
message = String.format("%s %s not found", matcher.group(1), matcher.group(2));
} else {
message = path;
}
exception = new ResourceNotFoundException(message);
}
break;
default:
if (response.getContent() != null) {
try {
String content = Utils.toStringAndClose(response.getContent());
exception = new HttpResponseException(command, response, content);
} catch (IOException e) {
logger.warn(e, "exception reading error from response", response);
exception = new HttpResponseException(command, response);
}
}
}
} finally {
Closeables.closeQuietly(response.getContent());
command.setException(exception);
}
}
}

View File

@ -921,7 +921,7 @@ public class CloudServersClientTest {
+ ""));
assertEquals(httpMethod.getHeaders().get(HttpHeaders.CONTENT_TYPE), Collections
.singletonList(MediaType.APPLICATION_JSON));
assertEquals("{\"reboot\":{\"flavor\":\"HARD\"}}", httpMethod.getPayload().getRawContent());
assertEquals("{\"reboot\":{\"type\":\"HARD\"}}", httpMethod.getPayload().getRawContent());
assertEquals(processor
.createExceptionParserOrThrowResourceNotFoundOn404IfNoAnnotation(method).getClass(),
ReturnFalseOn404.class);

View File

@ -66,7 +66,7 @@ public class BindRebootTypeToJsonPayloadTest {
injector.injectMembers(binder);
HttpRequest request = new HttpRequest(HttpMethod.POST, URI.create("http://localhost"));
binder.bindToRequest(request, RebootType.HARD);
assertEquals("{\"reboot\":{\"flavor\":\"HARD\"}}", request.getPayload().getRawContent());
assertEquals("{\"reboot\":{\"type\":\"HARD\"}}", request.getPayload().getRawContent());
}
@Test
@ -75,7 +75,7 @@ public class BindRebootTypeToJsonPayloadTest {
injector.injectMembers(binder);
HttpRequest request = new HttpRequest(HttpMethod.POST, URI.create("http://localhost"));
binder.bindToRequest(request, RebootType.SOFT);
assertEquals("{\"reboot\":{\"flavor\":\"SOFT\"}}", request.getPayload().getRawContent());
assertEquals("{\"reboot\":{\"type\":\"SOFT\"}}", request.getPayload().getRawContent());
}
@Test(expectedExceptions = { NullPointerException.class, IllegalStateException.class })

View File

@ -141,8 +141,8 @@ public class RimuHostingComputeService implements ComputeService {
@Override
public NodeSet runNodesWithTag(final String tag, int max, final Template template) {
checkArgument(tag.indexOf('-') == -1, "tag cannot contain hyphens");
logger.debug(">> running server image(%s) flavor(%s)", template.getImage().getId(), template
.getSize().getId());
logger.debug(">> running %d servers image(%s) flavor(%s)", max, template.getImage().getId(),
template.getSize().getId());
final Set<NodeMetadata> nodes = Sets.newHashSet();
Set<ListenableFuture<Void>> responses = Sets.newHashSet();
@ -171,9 +171,8 @@ public class RimuHostingComputeService implements ComputeService {
logger.debug("<< started server(%s)", node.getId());
// TODO! serverActive.apply(server);
logger.debug("<< running server(%s)", node.getId());
if (template.getOptions().getRunScript() != null) {
utils.runScriptOnNode(node, template.getOptions().getRunScript());
}
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied server(%s)", node.getId());
return null;
}
}), executor));

View File

@ -22,6 +22,7 @@ import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
@ -41,19 +42,24 @@ import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.internal.ImageImpl;
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.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
import org.jclouds.logging.Logger;
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.PricingPlan;
import org.jclouds.ssh.SshClient;
import com.google.common.base.Function;
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.Maps;
@ -82,6 +88,14 @@ public class RimuHostingComputeServiceContextModule extends RimuHostingContextMo
computeService, context);
}
@Provides
@Singleton
@Named("NOT_RUNNING")
protected Predicate<SshClient> runScriptRunning(RunScriptRunning stateRunning) {
return new RetryablePredicate<SshClient>(Predicates.not(stateRunning), 600, 3,
TimeUnit.SECONDS);
}
@Provides
@Singleton
Location getDefaultLocation(Map<String, ? extends Location> locations) {

View File

@ -19,25 +19,29 @@
package org.jclouds.rimuhosting.miro;
import static com.google.common.base.Preconditions.checkNotNull;
import com.google.common.base.Predicate;
import com.google.inject.*;
import static org.testng.Assert.assertEquals;
import java.io.IOException;
import java.net.InetAddress;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import org.jclouds.logging.log4j.config.Log4JLoggingModule;
import org.jclouds.predicates.AddressReachable;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.predicates.SocketOpen;
import org.jclouds.rimuhosting.miro.domain.Server;
import org.jclouds.ssh.jsch.config.JschSshClientModule;
import static org.testng.Assert.assertEquals;
import org.testng.annotations.AfterTest;
import org.testng.annotations.BeforeGroups;
import org.testng.annotations.Test;
import java.io.IOException;
import java.net.InetAddress;
import java.net.InetSocketAddress;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import com.google.common.base.Predicate;
import com.google.inject.AbstractModule;
import com.google.inject.Injector;
import com.google.inject.Key;
import com.google.inject.Provides;
import com.google.inject.TypeLiteral;
/**
* Tests behavior of {@code TerremarkVCloudClient}
@ -85,14 +89,6 @@ public class RimuHostingComputeClientLiveTest {
protected void configure() {
}
@SuppressWarnings("unused")
@Provides
private Predicate<InetSocketAddress> socketTester(SocketOpen open) {
return new RetryablePredicate<InetSocketAddress>(open, 130, 10,
TimeUnit.SECONDS);// make it longer then
// default internet
}
@SuppressWarnings("unused")
@Provides
private Predicate<InetAddress> addressTester(AddressReachable reachable) {

View File

@ -42,7 +42,6 @@ public class InitBuilder extends ScriptBuilder {
@SuppressWarnings("unchecked")
public InitBuilder(String instanceName, String instanceHome, String logDir,
Map<String, String> variables, String... execLines) {
super();
Map<String, String> defaultVariables = ImmutableMap.of("instanceName", instanceName,
"instanceHome", instanceHome, "logDir", logDir);
addEnvironmentVariableScope("default", defaultVariables)

View File

@ -13,6 +13,7 @@ function forget {
echo $INSTANCE_NAME already running pid [$FOUND_PID]
} || {
nohup $SCRIPT >$LOG_DIR/stdout.log 2>$LOG_DIR/stderr.log &
sleep 1
findPid $INSTANCE_NAME
[ -n "$FOUND_PID" ] || abort "$INSTANCE_NAME did not start"
}

View File

@ -49,6 +49,7 @@ function forget {
echo $INSTANCE_NAME already running pid [$FOUND_PID]
} || {
nohup $SCRIPT >$LOG_DIR/stdout.log 2>$LOG_DIR/stderr.log &
sleep 1
findPid $INSTANCE_NAME
[ -n "$FOUND_PID" ] || abort "$INSTANCE_NAME did not start"
}

View File

@ -47,6 +47,7 @@ function forget {
echo $INSTANCE_NAME already running pid [$FOUND_PID]
} || {
nohup $SCRIPT >$LOG_DIR/stdout.log 2>$LOG_DIR/stderr.log &
sleep 1
findPid $INSTANCE_NAME
[ -n "$FOUND_PID" ] || abort "$INSTANCE_NAME did not start"
}

View File

@ -42,7 +42,7 @@ public class VCloudPropertiesBuilder extends PropertiesBuilder {
protected Properties defaultProperties() {
Properties properties = super.defaultProperties();
properties.setProperty(PROPERTY_VCLOUD_VERSION, "0.8");
properties.setProperty(PROPERTY_VCLOUD_SESSIONINTERVAL, 9 * 60 + "");
properties.setProperty(PROPERTY_VCLOUD_SESSIONINTERVAL, 8 * 60 + "");
properties.setProperty(PROPERTY_VCLOUD_XML_NAMESPACE, "http://www.vmware.com/vcloud/v0.8");
properties.setProperty(PROPERTY_VCLOUD_XML_SCHEMA, "http://vcloud.safesecureweb.com/ns/vcloud.xsd");
return properties;

View File

@ -161,9 +161,8 @@ public class VCloudComputeService implements ComputeService, VCloudComputeClient
VApp vApp = client.getVApp(metaMap.get("id"));
NodeMetadata node = newCreateNodeResponse(tag, template, metaMap, vApp);
nodes.add(node);
if (template.getOptions().getRunScript() != null) {
utils.runScriptOnNode(node, template.getOptions().getRunScript());
}
utils.runOptionsOnNode(node, template.getOptions());
logger.debug("<< options applied vApp(%s)", node.getId());
}
protected NodeMetadata newCreateNodeResponse(String tag, Template template,

View File

@ -23,6 +23,7 @@ import java.util.Set;
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
import javax.annotation.Resource;
@ -40,13 +41,16 @@ import org.jclouds.compute.domain.Size;
import org.jclouds.compute.domain.internal.ImageImpl;
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.concurrent.ConcurrentUtils;
import org.jclouds.domain.Location;
import org.jclouds.domain.LocationScope;
import org.jclouds.domain.internal.LocationImpl;
import org.jclouds.logging.Logger;
import org.jclouds.predicates.RetryablePredicate;
import org.jclouds.rest.RestContext;
import org.jclouds.ssh.SshClient;
import org.jclouds.vcloud.VCloudAsyncClient;
import org.jclouds.vcloud.VCloudClient;
import org.jclouds.vcloud.VCloudMediaType;
@ -60,6 +64,8 @@ import org.jclouds.vcloud.domain.VAppTemplate;
import org.jclouds.vcloud.domain.VDC;
import com.google.common.base.Function;
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;
@ -93,6 +99,14 @@ public class VCloudComputeServiceContextModule extends VCloudContextModule {
return in;
}
@Provides
@Singleton
@Named("NOT_RUNNING")
protected Predicate<SshClient> runScriptRunning(RunScriptRunning stateRunning) {
return new RetryablePredicate<SshClient>(Predicates.not(stateRunning), 600, 3,
TimeUnit.SECONDS);
}
@Provides
@Singleton
protected ComputeServiceContext provideContext(ComputeService computeService,

View File

@ -119,7 +119,7 @@ public class VCloudContextBuilderTest {
"http://localhost");
assertEquals(builder.getProperties().getProperty(PROPERTY_VCLOUD_USER), "id");
assertEquals(builder.getProperties().getProperty(PROPERTY_VCLOUD_KEY), "secret");
assertEquals(builder.getProperties().getProperty(PROPERTY_VCLOUD_SESSIONINTERVAL), "540");
assertEquals(builder.getProperties().getProperty(PROPERTY_VCLOUD_SESSIONINTERVAL), "480");
}
public void testBuildContext() {

View File

@ -31,11 +31,6 @@ public class VCloudComputeServiceLiveTest extends BaseComputeServiceLiveTest {
service = "vcloud";
}
@Override
protected boolean canRunScript(Template template) {
return false;
}
@Override
protected Template buildTemplate(TemplateBuilder templateBuilder) {
return templateBuilder.osFamily(UBUNTU).smallest().build();