mirror of https://github.com/apache/jclouds.git
Issue 565: run script on a distinct node
This commit is contained in:
parent
557c0d75fe
commit
c621d53799
|
@ -37,6 +37,7 @@ import org.jclouds.Constants;
|
|||
import org.jclouds.aws.util.AWSUtils;
|
||||
import org.jclouds.collect.Memoized;
|
||||
import org.jclouds.compute.ComputeServiceContext;
|
||||
import org.jclouds.compute.callables.RunScriptOnNode;
|
||||
import org.jclouds.compute.domain.Hardware;
|
||||
import org.jclouds.compute.domain.Image;
|
||||
import org.jclouds.compute.domain.NodeMetadata;
|
||||
|
@ -91,14 +92,16 @@ public class EC2ComputeService extends BaseComputeService {
|
|||
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
|
||||
@Named("NODE_TERMINATED") Predicate<NodeMetadata> nodeTerminated,
|
||||
@Named("NODE_SUSPENDED") Predicate<NodeMetadata> nodeSuspended,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory, InitAdminAccess initAdminAccess,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory,
|
||||
RunScriptOnNode.Factory runScriptOnNodeFactory, InitAdminAccess initAdminAccess,
|
||||
PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
|
||||
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor, EC2Client ec2Client,
|
||||
Map<RegionAndName, KeyPair> credentialsMap, @Named("SECURITY") Map<RegionAndName, String> securityGroupMap) {
|
||||
super(context, credentialStore, images, sizes, locations, listNodesStrategy, getNodeMetadataStrategy,
|
||||
runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy, startNodeStrategy, stopNodeStrategy,
|
||||
templateBuilderProvider, templateOptionsProvider, nodeRunning, nodeTerminated, nodeSuspended,
|
||||
initScriptRunnerFactory, initAdminAccess, persistNodeCredentials, timeouts, executor);
|
||||
initScriptRunnerFactory, initAdminAccess, runScriptOnNodeFactory, persistNodeCredentials, timeouts,
|
||||
executor);
|
||||
this.ec2Client = ec2Client;
|
||||
this.credentialsMap = credentialsMap;
|
||||
this.securityGroupMap = securityGroupMap;
|
||||
|
|
|
@ -31,6 +31,7 @@ import javax.inject.Singleton;
|
|||
import org.jclouds.Constants;
|
||||
import org.jclouds.collect.Memoized;
|
||||
import org.jclouds.compute.ComputeServiceContext;
|
||||
import org.jclouds.compute.callables.RunScriptOnNode;
|
||||
import org.jclouds.compute.domain.Hardware;
|
||||
import org.jclouds.compute.domain.Image;
|
||||
import org.jclouds.compute.domain.NodeMetadata;
|
||||
|
@ -77,14 +78,16 @@ public class TerremarkVCloudComputeService extends BaseComputeService {
|
|||
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
|
||||
@Named("NODE_TERMINATED") Predicate<NodeMetadata> nodeTerminated,
|
||||
@Named("NODE_SUSPENDED") Predicate<NodeMetadata> nodeSuspended,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory, InitAdminAccess initAdminAccess,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory,
|
||||
RunScriptOnNode.Factory runScriptOnNodeFactory, InitAdminAccess initAdminAccess,
|
||||
PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
|
||||
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor, CleanupOrphanKeys cleanupOrphanKeys,
|
||||
ConcurrentMap<OrgAndName, KeyPairCredentials> credentialsMap, NodeMetadataToOrgAndName nodeToOrgAndName) {
|
||||
super(context, credentialStore, images, sizes, locations, listNodesStrategy, getNodeMetadataStrategy,
|
||||
runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy, resumeNodeStrategy,
|
||||
suspendNodeStrategy, templateBuilderProvider, templateOptionsProvider, nodeRunning, nodeTerminated,
|
||||
nodeSuspended, initScriptRunnerFactory, initAdminAccess, persistNodeCredentials, timeouts, executor);
|
||||
nodeSuspended, initScriptRunnerFactory, initAdminAccess, runScriptOnNodeFactory, persistNodeCredentials,
|
||||
timeouts, executor);
|
||||
this.cleanupOrphanKeys = cleanupOrphanKeys;
|
||||
}
|
||||
|
||||
|
|
|
@ -69,7 +69,8 @@ Here's an example of creating and running a small linux node in the group webser
|
|||
[org.jclouds.compute.domain
|
||||
Template TemplateBuilder ComputeMetadata NodeMetadata Hardware
|
||||
OsFamily Image]
|
||||
[org.jclouds.compute.options TemplateOptions]
|
||||
[org.jclouds.compute.options TemplateOptions RunScriptOptions
|
||||
RunScriptOptions$Builder]
|
||||
[org.jclouds.compute.predicates
|
||||
NodePredicates]
|
||||
[com.google.common.collect ImmutableSet])
|
||||
|
@ -233,9 +234,15 @@ Here's an example of creating and running a small linux node in the group webser
|
|||
([#^ComputeService compute id]
|
||||
(.destroyNode compute id)))
|
||||
|
||||
(defn run-script-on-nodes-matching [#^ComputeService compute pred command template]
|
||||
(defn run-script-on-node
|
||||
"Run a script on a node"
|
||||
([#^ComputeService compute id command #^RunScriptOptions options]
|
||||
(.runScriptOnNode compute id command options)))
|
||||
|
||||
(defn run-script-on-nodes-matching
|
||||
"Run a script on the nodes matching the given predicate"
|
||||
(.runScriptOnNodesMatching compute (to-predicate pred) command template))
|
||||
([#^ComputeService compute pred command #^RunScriptOptions options]
|
||||
(.runScriptOnNodesMatching compute (to-predicate pred) command options)))
|
||||
|
||||
(defmacro state-predicate [node state]
|
||||
`(= (.getState ~node)
|
||||
|
|
|
@ -319,4 +319,40 @@ public interface ComputeService {
|
|||
Map<? extends NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter,
|
||||
Statement runScript, RunScriptOptions options) throws RunScriptOnNodesException;
|
||||
|
||||
/**
|
||||
* Run the script on a specific node
|
||||
*
|
||||
* @param id
|
||||
* node the script is to be executed on
|
||||
* @param runScript
|
||||
* statement containing the script to run
|
||||
* @param options
|
||||
* nullable options to how to run the script, whether to override credentials
|
||||
* @return map with node identifiers and corresponding responses
|
||||
* @throws NoSuchElementException
|
||||
* if the node is not found
|
||||
* @throws IllegalStateException
|
||||
* if the node is not in running state
|
||||
*
|
||||
* @see org.jclouds.compute.predicates.NodePredicates#runningWithTag(String)
|
||||
* @see org.jclouds.scriptbuilder.domain.Statements
|
||||
*/
|
||||
ExecResponse runScriptOnNode(String id, Statement runScript, RunScriptOptions options);
|
||||
|
||||
/**
|
||||
* @see #runScriptOnNode(String, Statement, RunScriptOptions)
|
||||
*/
|
||||
ExecResponse runScriptOnNode(String id, Statement runScript);
|
||||
|
||||
/**
|
||||
* @see #runScriptOnNode(String, Statement, RunScriptOptions)
|
||||
* @see org.jclouds.scriptbuilder.domain.Statements#exec
|
||||
*/
|
||||
ExecResponse runScriptOnNode(String id, String runScript, RunScriptOptions options);
|
||||
|
||||
/**
|
||||
* @see #runScriptOnNode(String, String, RunScriptOptions)
|
||||
*/
|
||||
ExecResponse runScriptOnNode(String id, String runScript);
|
||||
|
||||
}
|
||||
|
|
|
@ -62,6 +62,7 @@ import org.jclouds.compute.domain.Hardware;
|
|||
import org.jclouds.compute.domain.Image;
|
||||
import org.jclouds.compute.domain.NodeMetadata;
|
||||
import org.jclouds.compute.domain.NodeMetadataBuilder;
|
||||
import org.jclouds.compute.domain.NodeState;
|
||||
import org.jclouds.compute.domain.Template;
|
||||
import org.jclouds.compute.domain.TemplateBuilder;
|
||||
import org.jclouds.compute.options.RunScriptOptions;
|
||||
|
@ -78,8 +79,8 @@ import org.jclouds.compute.strategy.ResumeNodeStrategy;
|
|||
import org.jclouds.compute.strategy.RunScriptOnNodeAndAddToGoodMapOrPutExceptionIntoBadMap;
|
||||
import org.jclouds.compute.strategy.SuspendNodeStrategy;
|
||||
import org.jclouds.domain.Credentials;
|
||||
import org.jclouds.domain.Credentials.Builder;
|
||||
import org.jclouds.domain.Location;
|
||||
import org.jclouds.domain.Credentials.Builder;
|
||||
import org.jclouds.io.Payload;
|
||||
import org.jclouds.logging.Logger;
|
||||
import org.jclouds.predicates.RetryablePredicate;
|
||||
|
@ -132,22 +133,25 @@ public class BaseComputeService implements ComputeService {
|
|||
private final Timeouts timeouts;
|
||||
private final InitAdminAccess initAdminAccess;
|
||||
private final PersistNodeCredentials persistNodeCredentials;
|
||||
private final RunScriptOnNode.Factory runScriptOnNodeFactory;
|
||||
private final ExecutorService executor;
|
||||
|
||||
@Inject
|
||||
protected BaseComputeService(ComputeServiceContext context, Map<String, Credentials> credentialStore,
|
||||
@Memoized Supplier<Set<? extends Image>> images, @Memoized Supplier<Set<? extends Hardware>> hardwareProfiles,
|
||||
@Memoized Supplier<Set<? extends Location>> locations, ListNodesStrategy listNodesStrategy,
|
||||
GetNodeMetadataStrategy getNodeMetadataStrategy, CreateNodesInGroupThenAddToSet runNodesAndAddToSetStrategy,
|
||||
RebootNodeStrategy rebootNodeStrategy, DestroyNodeStrategy destroyNodeStrategy,
|
||||
ResumeNodeStrategy resumeNodeStrategy, SuspendNodeStrategy suspendNodeStrategy,
|
||||
Provider<TemplateBuilder> templateBuilderProvider, Provider<TemplateOptions> templateOptionsProvider,
|
||||
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
|
||||
@Named("NODE_TERMINATED") Predicate<NodeMetadata> nodeTerminated,
|
||||
@Named("NODE_SUSPENDED") Predicate<NodeMetadata> nodeSuspended,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory, InitAdminAccess initAdminAccess,
|
||||
PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
|
||||
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
|
||||
@Memoized Supplier<Set<? extends Image>> images,
|
||||
@Memoized Supplier<Set<? extends Hardware>> hardwareProfiles,
|
||||
@Memoized Supplier<Set<? extends Location>> locations, ListNodesStrategy listNodesStrategy,
|
||||
GetNodeMetadataStrategy getNodeMetadataStrategy,
|
||||
CreateNodesInGroupThenAddToSet runNodesAndAddToSetStrategy, RebootNodeStrategy rebootNodeStrategy,
|
||||
DestroyNodeStrategy destroyNodeStrategy, ResumeNodeStrategy resumeNodeStrategy,
|
||||
SuspendNodeStrategy suspendNodeStrategy, Provider<TemplateBuilder> templateBuilderProvider,
|
||||
Provider<TemplateOptions> templateOptionsProvider,
|
||||
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
|
||||
@Named("NODE_TERMINATED") Predicate<NodeMetadata> nodeTerminated,
|
||||
@Named("NODE_SUSPENDED") Predicate<NodeMetadata> nodeSuspended,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory, InitAdminAccess initAdminAccess,
|
||||
RunScriptOnNode.Factory runScriptOnNodeFactory, PersistNodeCredentials persistNodeCredentials,
|
||||
Timeouts timeouts, @Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
|
||||
this.context = checkNotNull(context, "context");
|
||||
this.credentialStore = checkNotNull(credentialStore, "credentialStore");
|
||||
this.images = checkNotNull(images, "images");
|
||||
|
@ -168,6 +172,7 @@ public class BaseComputeService implements ComputeService {
|
|||
this.initScriptRunnerFactory = checkNotNull(initScriptRunnerFactory, "initScriptRunnerFactory");
|
||||
this.timeouts = checkNotNull(timeouts, "timeouts");
|
||||
this.initAdminAccess = checkNotNull(initAdminAccess, "initAdminAccess");
|
||||
this.runScriptOnNodeFactory = checkNotNull(runScriptOnNodeFactory, "runScriptOnNodeFactory");
|
||||
this.persistNodeCredentials = checkNotNull(persistNodeCredentials, "persistNodeCredentials");
|
||||
this.executor = checkNotNull(executor, "executor");
|
||||
}
|
||||
|
@ -185,7 +190,7 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Set<? extends NodeMetadata> runNodesWithTag(String group, int count, Template template)
|
||||
throws RunNodesException {
|
||||
throws RunNodesException {
|
||||
return createNodesInGroup(group, count, template);
|
||||
}
|
||||
|
||||
|
@ -194,7 +199,7 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Set<? extends NodeMetadata> runNodesWithTag(String group, int count, TemplateOptions templateOptions)
|
||||
throws RunNodesException {
|
||||
throws RunNodesException {
|
||||
return createNodesInGroup(group, count, templateBuilder().any().options(templateOptions).build());
|
||||
}
|
||||
|
||||
|
@ -208,12 +213,12 @@ public class BaseComputeService implements ComputeService {
|
|||
|
||||
@Override
|
||||
public Set<? extends NodeMetadata> createNodesInGroup(String group, int count, Template template)
|
||||
throws RunNodesException {
|
||||
throws RunNodesException {
|
||||
checkNotNull(group, "group cannot be null");
|
||||
checkNotNull(template.getLocation(), "location");
|
||||
logger.debug(">> running %d node%s group(%s) location(%s) image(%s) hardwareProfile(%s) options(%s)", count,
|
||||
count > 1 ? "s" : "", group, template.getLocation().getId(), template.getImage().getId(), template
|
||||
.getHardware().getId(), template.getOptions());
|
||||
count > 1 ? "s" : "", group, template.getLocation().getId(), template.getImage().getId(), template
|
||||
.getHardware().getId(), template.getOptions());
|
||||
Set<NodeMetadata> goodNodes = newLinkedHashSet();
|
||||
Map<NodeMetadata, Exception> badNodes = newLinkedHashMap();
|
||||
Multimap<NodeMetadata, CustomizationResponse> customizationResponses = LinkedHashMultimap.create();
|
||||
|
@ -222,9 +227,9 @@ public class BaseComputeService implements ComputeService {
|
|||
template.getOptions().runScript(initAdminAccess.apply(template.getOptions().getRunScript()));
|
||||
|
||||
Map<?, Future<Void>> responses = runNodesAndAddToSetStrategy.execute(group, count, template, goodNodes, badNodes,
|
||||
customizationResponses);
|
||||
customizationResponses);
|
||||
Map<?, Exception> executionExceptions = awaitCompletion(responses, executor, null, logger, "runNodesWithTag("
|
||||
+ group + ")");
|
||||
+ group + ")");
|
||||
Function<NodeMetadata, NodeMetadata> fn = persistNodeCredentials.always(template.getOptions().getRunScript());
|
||||
badNodes = Maps2.transformKeys(badNodes, fn);
|
||||
goodNodes = ImmutableSet.copyOf(Iterables.transform(goodNodes, fn));
|
||||
|
@ -236,7 +241,7 @@ public class BaseComputeService implements ComputeService {
|
|||
|
||||
@Override
|
||||
public Set<? extends NodeMetadata> createNodesInGroup(String group, int count, TemplateOptions templateOptions)
|
||||
throws RunNodesException {
|
||||
throws RunNodesException {
|
||||
return createNodesInGroup(group, count, templateBuilder().any().options(templateOptions).build());
|
||||
}
|
||||
|
||||
|
@ -282,27 +287,27 @@ public class BaseComputeService implements ComputeService {
|
|||
public Set<? extends NodeMetadata> destroyNodesMatching(Predicate<NodeMetadata> filter) {
|
||||
logger.debug(">> destroying nodes matching(%s)", filter);
|
||||
Set<NodeMetadata> set = newLinkedHashSet(transformParallel(nodesMatchingFilterAndNotTerminated(filter),
|
||||
new Function<NodeMetadata, Future<NodeMetadata>>() {
|
||||
new Function<NodeMetadata, Future<NodeMetadata>>() {
|
||||
|
||||
// TODO make an async interface instead of re-wrapping
|
||||
@Override
|
||||
public Future<NodeMetadata> apply(final NodeMetadata from) {
|
||||
return executor.submit(new Callable<NodeMetadata>() {
|
||||
// TODO make an async interface instead of re-wrapping
|
||||
@Override
|
||||
public Future<NodeMetadata> apply(final NodeMetadata from) {
|
||||
return executor.submit(new Callable<NodeMetadata>() {
|
||||
|
||||
@Override
|
||||
public NodeMetadata call() throws Exception {
|
||||
destroyNode(from.getId());
|
||||
return from;
|
||||
}
|
||||
@Override
|
||||
public NodeMetadata call() throws Exception {
|
||||
destroyNode(from.getId());
|
||||
return from;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "destroyNode(" + from.getId() + ")";
|
||||
}
|
||||
});
|
||||
}
|
||||
@Override
|
||||
public String toString() {
|
||||
return "destroyNode(" + from.getId() + ")";
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
}, executor, null, logger, "destroyNodesMatching(" + filter + ")"));
|
||||
}, executor, null, logger, "destroyNodesMatching(" + filter + ")"));
|
||||
logger.debug("<< destroyed(%d)", set.size());
|
||||
return set;
|
||||
}
|
||||
|
@ -316,7 +321,7 @@ public class BaseComputeService implements ComputeService {
|
|||
* if none found
|
||||
*/
|
||||
Iterable<? extends NodeMetadata> nodesMatchingFilterAndNotTerminatedExceptionIfNotFound(
|
||||
Predicate<NodeMetadata> filter) {
|
||||
Predicate<NodeMetadata> filter) {
|
||||
Iterable<? extends NodeMetadata> nodes = nodesMatchingFilterAndNotTerminated(filter);
|
||||
if (Iterables.size(nodes) == 0)
|
||||
throw new NoSuchElementException("no nodes matched filter: " + filter);
|
||||
|
@ -406,15 +411,15 @@ public class BaseComputeService implements ComputeService {
|
|||
public void rebootNodesMatching(Predicate<NodeMetadata> filter) {
|
||||
logger.debug(">> rebooting nodes matching(%s)", filter);
|
||||
transformParallel(nodesMatchingFilterAndNotTerminatedExceptionIfNotFound(filter),
|
||||
new Function<NodeMetadata, Future<Void>>() {
|
||||
// TODO use native async
|
||||
@Override
|
||||
public Future<Void> apply(NodeMetadata from) {
|
||||
rebootNode(from.getId());
|
||||
return immediateFuture(null);
|
||||
}
|
||||
new Function<NodeMetadata, Future<Void>>() {
|
||||
// TODO use native async
|
||||
@Override
|
||||
public Future<Void> apply(NodeMetadata from) {
|
||||
rebootNode(from.getId());
|
||||
return immediateFuture(null);
|
||||
}
|
||||
|
||||
}, executor, null, logger, "rebootNodesMatching(" + filter + ")");
|
||||
}, executor, null, logger, "rebootNodesMatching(" + filter + ")");
|
||||
logger.debug("<< rebooted");
|
||||
}
|
||||
|
||||
|
@ -437,15 +442,15 @@ public class BaseComputeService implements ComputeService {
|
|||
public void resumeNodesMatching(Predicate<NodeMetadata> filter) {
|
||||
logger.debug(">> resuming nodes matching(%s)", filter);
|
||||
transformParallel(nodesMatchingFilterAndNotTerminatedExceptionIfNotFound(filter),
|
||||
new Function<NodeMetadata, Future<Void>>() {
|
||||
// TODO use native async
|
||||
@Override
|
||||
public Future<Void> apply(NodeMetadata from) {
|
||||
resumeNode(from.getId());
|
||||
return immediateFuture(null);
|
||||
}
|
||||
new Function<NodeMetadata, Future<Void>>() {
|
||||
// TODO use native async
|
||||
@Override
|
||||
public Future<Void> apply(NodeMetadata from) {
|
||||
resumeNode(from.getId());
|
||||
return immediateFuture(null);
|
||||
}
|
||||
|
||||
}, executor, null, logger, "resumeNodesMatching(" + filter + ")");
|
||||
}, executor, null, logger, "resumeNodesMatching(" + filter + ")");
|
||||
logger.debug("<< resumed");
|
||||
}
|
||||
|
||||
|
@ -468,15 +473,15 @@ public class BaseComputeService implements ComputeService {
|
|||
public void suspendNodesMatching(Predicate<NodeMetadata> filter) {
|
||||
logger.debug(">> suspending nodes matching(%s)", filter);
|
||||
transformParallel(nodesMatchingFilterAndNotTerminatedExceptionIfNotFound(filter),
|
||||
new Function<NodeMetadata, Future<Void>>() {
|
||||
// TODO use native async
|
||||
@Override
|
||||
public Future<Void> apply(NodeMetadata from) {
|
||||
suspendNode(from.getId());
|
||||
return immediateFuture(null);
|
||||
}
|
||||
new Function<NodeMetadata, Future<Void>>() {
|
||||
// TODO use native async
|
||||
@Override
|
||||
public Future<Void> apply(NodeMetadata from) {
|
||||
suspendNode(from.getId());
|
||||
return immediateFuture(null);
|
||||
}
|
||||
|
||||
}, executor, null, logger, "suspendNodesMatching(" + filter + ")");
|
||||
}, executor, null, logger, "suspendNodesMatching(" + filter + ")");
|
||||
logger.debug("<< suspended");
|
||||
}
|
||||
|
||||
|
@ -485,7 +490,7 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Map<NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter, Payload runScript)
|
||||
throws RunScriptOnNodesException {
|
||||
throws RunScriptOnNodesException {
|
||||
return runScriptOnNodesMatching(filter, runScript, RunScriptOptions.NONE);
|
||||
}
|
||||
|
||||
|
@ -494,10 +499,10 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Map<NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter, Payload runScript,
|
||||
RunScriptOptions options) throws RunScriptOnNodesException {
|
||||
RunScriptOptions options) throws RunScriptOnNodesException {
|
||||
try {
|
||||
return runScriptOnNodesMatching(filter,
|
||||
Statements.exec(Strings2.toStringAndClose(checkNotNull(runScript, "runScript").getInput())), options);
|
||||
return runScriptOnNodesMatching(filter, Statements.exec(Strings2.toStringAndClose(checkNotNull(runScript,
|
||||
"runScript").getInput())), options);
|
||||
} catch (IOException e) {
|
||||
Throwables.propagate(e);
|
||||
return null;
|
||||
|
@ -509,7 +514,7 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Map<NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter, String runScript)
|
||||
throws RunScriptOnNodesException {
|
||||
throws RunScriptOnNodesException {
|
||||
return runScriptOnNodesMatching(filter, Statements.exec(checkNotNull(runScript, "runScript")));
|
||||
}
|
||||
|
||||
|
@ -518,15 +523,15 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Map<NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter, Statement runScript)
|
||||
throws RunScriptOnNodesException {
|
||||
throws RunScriptOnNodesException {
|
||||
return runScriptOnNodesMatching(filter, runScript, RunScriptOptions.NONE);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<? extends NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter,
|
||||
String runScript, RunScriptOptions options) throws RunScriptOnNodesException {
|
||||
String runScript, RunScriptOptions options) throws RunScriptOnNodesException {
|
||||
return runScriptOnNodesMatching(filter, Statements.exec(checkNotNull(runScript, "runScript")),
|
||||
RunScriptOptions.NONE);
|
||||
RunScriptOptions.NONE);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -534,7 +539,7 @@ public class BaseComputeService implements ComputeService {
|
|||
*/
|
||||
@Override
|
||||
public Map<NodeMetadata, ExecResponse> runScriptOnNodesMatching(Predicate<NodeMetadata> filter, Statement runScript,
|
||||
RunScriptOptions options) throws RunScriptOnNodesException {
|
||||
RunScriptOptions options) throws RunScriptOnNodesException {
|
||||
|
||||
checkNotNull(filter, "filter");
|
||||
checkNotNull(runScript, "runScript");
|
||||
|
@ -548,11 +553,11 @@ public class BaseComputeService implements ComputeService {
|
|||
runScript = initAdminAccess.apply(runScript);
|
||||
|
||||
Iterable<? extends RunScriptOnNode> scriptRunners = transformNodesIntoInitializedScriptRunners(
|
||||
nodesMatchingFilterAndNotTerminatedExceptionIfNotFound(filter), runScript, options, badNodes);
|
||||
nodesMatchingFilterAndNotTerminatedExceptionIfNotFound(filter), runScript, options, badNodes);
|
||||
if (Iterables.size(scriptRunners) > 0) {
|
||||
for (RunScriptOnNode runner : scriptRunners) {
|
||||
responses.put(runner.getNode(), executor.submit(new RunScriptOnNodeAndAddToGoodMapOrPutExceptionIntoBadMap(
|
||||
runner, goodNodes, badNodes)));
|
||||
runner, goodNodes, badNodes)));
|
||||
}
|
||||
exceptions = awaitCompletion(responses, executor, null, logger, "runScriptOnNodesMatching(" + filter + ")");
|
||||
}
|
||||
|
@ -566,13 +571,50 @@ public class BaseComputeService implements ComputeService {
|
|||
}
|
||||
return goodNodes;
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ExecResponse runScriptOnNode(String id, String runScript) {
|
||||
return runScriptOnNode(id, runScript, RunScriptOptions.NONE);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ExecResponse runScriptOnNode(String id, String runScript, RunScriptOptions options) {
|
||||
return runScriptOnNode(id, Statements.exec(checkNotNull(runScript, "runScript")), options);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ExecResponse runScriptOnNode(String id, Statement runScript) {
|
||||
return runScriptOnNode(id, runScript, RunScriptOptions.NONE);
|
||||
}
|
||||
|
||||
/**
|
||||
* {@inheritDoc}
|
||||
*/
|
||||
@Override
|
||||
public ExecResponse runScriptOnNode(String id, Statement runScript, RunScriptOptions options) {
|
||||
NodeMetadata node = this.getNodeMetadata(id);
|
||||
if (node == null)
|
||||
throw new NoSuchElementException(id);
|
||||
if (node.getState() != NodeState.RUNNING)
|
||||
throw new IllegalStateException("node " + id
|
||||
+ " needs to be running before executing a script on it. current state: " + node.getState());
|
||||
return runScriptOnNodeFactory.create(node, runScript, options).init().call();
|
||||
}
|
||||
|
||||
private Iterable<? extends RunScriptOnNode> transformNodesIntoInitializedScriptRunners(
|
||||
Iterable<? extends NodeMetadata> nodes, Statement script, RunScriptOptions options,
|
||||
Map<NodeMetadata, Exception> badNodes) {
|
||||
return filter(
|
||||
transformParallel(nodes, new TransformNodesIntoInitializedScriptRunners(script, options, badNodes),
|
||||
executor, null, logger, "initialize script runners"), notNull());
|
||||
Iterable<? extends NodeMetadata> nodes, Statement script, RunScriptOptions options,
|
||||
Map<NodeMetadata, Exception> badNodes) {
|
||||
return filter(transformParallel(nodes, new TransformNodesIntoInitializedScriptRunners(script, options, badNodes),
|
||||
executor, null, logger, "initialize script runners"), notNull());
|
||||
}
|
||||
|
||||
private Set<? extends NodeMetadata> detailsOnAllNodes() {
|
||||
|
@ -585,13 +627,13 @@ public class BaseComputeService implements ComputeService {
|
|||
}
|
||||
|
||||
private final class TransformNodesIntoInitializedScriptRunners implements
|
||||
Function<NodeMetadata, Future<RunScriptOnNode>> {
|
||||
Function<NodeMetadata, Future<RunScriptOnNode>> {
|
||||
private final Map<NodeMetadata, Exception> badNodes;
|
||||
private final Statement script;
|
||||
private final RunScriptOptions options;
|
||||
|
||||
private TransformNodesIntoInitializedScriptRunners(Statement script, RunScriptOptions options,
|
||||
Map<NodeMetadata, Exception> badNodes) {
|
||||
Map<NodeMetadata, Exception> badNodes) {
|
||||
this.badNodes = checkNotNull(badNodes, "badNodes");
|
||||
this.script = checkNotNull(script, "script");
|
||||
this.options = checkNotNull(options, "options");
|
||||
|
@ -602,7 +644,7 @@ public class BaseComputeService implements ComputeService {
|
|||
checkNotNull(node, "node");
|
||||
if (options.getOverridingCredentials() != null) {
|
||||
Builder<? extends Credentials> builder = node.getCredentials() != null ? node.getCredentials().toBuilder()
|
||||
: new Credentials.Builder<Credentials>();
|
||||
: new Credentials.Builder<Credentials>();
|
||||
if (options.getOverridingCredentials().identity != null)
|
||||
builder.identity(options.getOverridingCredentials().identity);
|
||||
if (options.getOverridingCredentials().credential != null)
|
||||
|
|
|
@ -28,6 +28,8 @@
|
|||
org.jclouds.scriptbuilder.domain.Statements
|
||||
org.jclouds.compute.options.TemplateOptions
|
||||
org.jclouds.compute.options.TemplateOptions$Builder
|
||||
org.jclouds.compute.options.RunScriptOptions
|
||||
org.jclouds.compute.options.RunScriptOptions$Builder
|
||||
org.jclouds.domain.Credentials
|
||||
java.util.NoSuchElementException
|
||||
))
|
||||
|
@ -98,9 +100,9 @@ list, Alan Dipert and MeikelBrandmeyer."
|
|||
(is (= 0 (count (nodes-with-details-matching *compute* #(and (running? %) (localhost? %))))))
|
||||
(is (= 1 (count (nodes-with-details-matching *compute* #(and (running? %) (not (localhost? %))))))))
|
||||
|
||||
(deftest run-script-on-nodes-matching-test
|
||||
(deftest run-script-on-nodes-matching-with-options-test
|
||||
(let [echo (Statements/exec "echo hello")
|
||||
script-options (.. (TemplateOptions$Builder/overrideCredentialsWith (Credentials. "user" "password"))
|
||||
script-options (.. (RunScriptOptions$Builder/overrideCredentialsWith (Credentials. "user" "password"))
|
||||
(runAsRoot false)
|
||||
(wrapInInitScript false))
|
||||
pred #(= (.getGroup %) "scriptednode")]
|
||||
|
@ -109,6 +111,16 @@ list, Alan Dipert and MeikelBrandmeyer."
|
|||
(is (thrown? NoSuchElementException
|
||||
(run-script-on-nodes-matching *compute* #(= (.getGroup %) "nonexistingnode") echo script-options)))))
|
||||
|
||||
(deftest run-script-on-node-with-options-test
|
||||
(let [echo (Statements/exec "echo hello")
|
||||
script-options (.. (RunScriptOptions$Builder/overrideCredentialsWith (Credentials. "user" "password"))
|
||||
(runAsRoot false)
|
||||
(wrapInInitScript false))
|
||||
test_node (create-node *compute* "scriptednode" (build-template *compute* {}))]
|
||||
(is (run-script-on-node *compute* (id test_node) echo script-options))
|
||||
(is (thrown? NoSuchElementException
|
||||
(run-script-on-node *compute* "nonexistingnode" echo script-options)))))
|
||||
|
||||
(deftest build-template-test
|
||||
(let [service (compute-service "stub" "user" "password")]
|
||||
(testing "nullary"
|
||||
|
|
|
@ -30,6 +30,7 @@ import static com.google.common.collect.Maps.uniqueIndex;
|
|||
import static com.google.common.collect.Sets.filter;
|
||||
import static com.google.common.collect.Sets.newTreeSet;
|
||||
import static org.jclouds.compute.ComputeTestUtils.buildScript;
|
||||
import static org.jclouds.compute.options.RunScriptOptions.Builder.wrapInInitScript;
|
||||
import static org.jclouds.compute.options.TemplateOptions.Builder.blockOnComplete;
|
||||
import static org.jclouds.compute.options.TemplateOptions.Builder.overrideCredentialsWith;
|
||||
import static org.jclouds.compute.predicates.NodePredicates.TERMINATED;
|
||||
|
@ -44,12 +45,12 @@ import java.io.FileNotFoundException;
|
|||
import java.io.IOException;
|
||||
import java.util.Collection;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.SortedSet;
|
||||
import java.util.TreeSet;
|
||||
import java.util.Map.Entry;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
|
@ -242,6 +243,11 @@ public abstract class BaseComputeServiceLiveTest {
|
|||
overrideCredentialsWith(good).wrapInInitScript(false).runAsRoot(false)).entrySet())
|
||||
assert response.getValue().getOutput().trim().equals("hello") : response.getKey() + ": "
|
||||
+ response.getValue();
|
||||
|
||||
// test single-node execution
|
||||
ExecResponse response = client.runScriptOnNode(get(nodes, 0).getId(), "echo hello", wrapInInitScript(false)
|
||||
.runAsRoot(false));
|
||||
assert response.getOutput().trim().equals("hello") : get(nodes, 0).getId() + ": " + response;
|
||||
|
||||
runScriptWithCreds(group, os, good);
|
||||
|
||||
|
|
|
@ -144,15 +144,20 @@ public class StubComputeServiceIntegrationTest extends BaseComputeServiceLiveTes
|
|||
runScriptAndService(client1, 1);
|
||||
|
||||
expect(factory.create(new IPSocket("144.175.1.2", 22), new Credentials("root", "password2"))).andReturn(
|
||||
client2).times(3);
|
||||
client2).times(4);
|
||||
expect(factory.create(new IPSocket("144.175.1.2", 22), new Credentials("root", "romeo"))).andThrow(
|
||||
new SshException("Auth fail"));
|
||||
|
||||
// run script without backgrounding
|
||||
// run script without backgrounding (via predicate)
|
||||
client2.connect();
|
||||
expect(client2.exec("echo hello\n")).andReturn(new ExecResponse("hello\n", "", 0));
|
||||
client2.disconnect();
|
||||
|
||||
|
||||
// run script without backgrounding (via id)
|
||||
client2.connect();
|
||||
expect(client2.exec("echo hello\n")).andReturn(new ExecResponse("hello\n", "", 0));
|
||||
client2.disconnect();
|
||||
|
||||
client2.connect();
|
||||
try {
|
||||
runScript(client2, "runScriptWithCreds",
|
||||
|
|
|
@ -36,6 +36,7 @@ import org.jclouds.aws.ec2.domain.PlacementGroup;
|
|||
import org.jclouds.aws.ec2.domain.PlacementGroup.State;
|
||||
import org.jclouds.collect.Memoized;
|
||||
import org.jclouds.compute.ComputeServiceContext;
|
||||
import org.jclouds.compute.callables.RunScriptOnNode;
|
||||
import org.jclouds.compute.domain.Hardware;
|
||||
import org.jclouds.compute.domain.Image;
|
||||
import org.jclouds.compute.domain.NodeMetadata;
|
||||
|
@ -85,7 +86,8 @@ public class AWSEC2ComputeService extends EC2ComputeService {
|
|||
@Named("NODE_RUNNING") Predicate<NodeMetadata> nodeRunning,
|
||||
@Named("NODE_TERMINATED") Predicate<NodeMetadata> nodeTerminated,
|
||||
@Named("NODE_SUSPENDED") Predicate<NodeMetadata> nodeSuspended,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory, InitAdminAccess initAdminAccess,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory,
|
||||
RunScriptOnNode.Factory runScriptOnNodeFactory, InitAdminAccess initAdminAccess,
|
||||
PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
|
||||
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor, AWSEC2Client ec2Client,
|
||||
Map<RegionAndName, KeyPair> credentialsMap, @Named("SECURITY") Map<RegionAndName, String> securityGroupMap,
|
||||
|
@ -94,8 +96,8 @@ public class AWSEC2ComputeService extends EC2ComputeService {
|
|||
super(context, credentialStore, images, sizes, locations, listNodesStrategy, getNodeMetadataStrategy,
|
||||
runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy, startNodeStrategy, stopNodeStrategy,
|
||||
templateBuilderProvider, templateOptionsProvider, nodeRunning, nodeTerminated, nodeSuspended,
|
||||
initScriptRunnerFactory, initAdminAccess, persistNodeCredentials, timeouts, executor, ec2Client,
|
||||
credentialsMap, securityGroupMap);
|
||||
initScriptRunnerFactory, runScriptOnNodeFactory, initAdminAccess, persistNodeCredentials, timeouts,
|
||||
executor, ec2Client, credentialsMap, securityGroupMap);
|
||||
this.ec2Client = ec2Client;
|
||||
this.placementGroupMap = placementGroupMap;
|
||||
this.placementGroupDeleted = placementGroupDeleted;
|
||||
|
|
|
@ -30,6 +30,7 @@ import javax.inject.Singleton;
|
|||
import org.jclouds.Constants;
|
||||
import org.jclouds.collect.Memoized;
|
||||
import org.jclouds.compute.ComputeServiceContext;
|
||||
import org.jclouds.compute.callables.RunScriptOnNode;
|
||||
import org.jclouds.compute.domain.Hardware;
|
||||
import org.jclouds.compute.domain.Image;
|
||||
import org.jclouds.compute.domain.NodeMetadata;
|
||||
|
@ -71,12 +72,13 @@ public class GoGridComputeService extends BaseComputeService {
|
|||
@Named("NODE_TERMINATED") Predicate<NodeMetadata> nodeTerminated,
|
||||
@Named("NODE_SUSPENDED") Predicate<NodeMetadata> nodeSuspended,
|
||||
InitializeRunScriptOnNodeOrPlaceInBadMap.Factory initScriptRunnerFactory, InitAdminAccess initAdminAccess,
|
||||
PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
|
||||
RunScriptOnNode.Factory runScriptOnNodeFactory, PersistNodeCredentials persistNodeCredentials, Timeouts timeouts,
|
||||
@Named(Constants.PROPERTY_USER_THREADS) ExecutorService executor) {
|
||||
super(context, credentialStore, images, hardwareProfiles, locations, listNodesStrategy, getNodeMetadataStrategy,
|
||||
runNodesAndAddToSetStrategy, rebootNodeStrategy, destroyNodeStrategy, resumeNodeStrategy,
|
||||
suspendNodeStrategy, templateBuilderProvider, templateOptionsProvider, nodeRunning, nodeTerminated,
|
||||
nodeSuspended, initScriptRunnerFactory, initAdminAccess, persistNodeCredentials, timeouts, executor);
|
||||
nodeSuspended, initScriptRunnerFactory, initAdminAccess, runScriptOnNodeFactory, persistNodeCredentials, timeouts,
|
||||
executor);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
Loading…
Reference in New Issue