You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@whirr.apache.org by to...@apache.org on 2010/08/21 00:22:47 UTC

svn commit: r987653 - in /incubator/whirr/trunk: ./ cli/src/main/java/org/apache/whirr/cli/ cli/src/main/java/org/apache/whirr/cli/command/ cli/src/test/java/org/apache/whirr/cli/ cli/src/test/java/org/apache/whirr/cli/command/ cli/src/test/resources/ ...

Author: tomwhite
Date: Fri Aug 20 22:22:46 2010
New Revision: 987653

URL: http://svn.apache.org/viewvc?rev=987653&view=rev
Log:
WHIRR-75. Use Commons Configuration to manage cluster specs.

Added:
    incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ClusterSpecCommandTest.java   (with props)
    incubator/whirr/trunk/cli/src/test/resources/
    incubator/whirr/trunk/cli/src/test/resources/whirr-override-test.properties   (with props)
    incubator/whirr/trunk/services/cassandra/src/test/resources/whirr-cassandra-test.properties   (with props)
    incubator/whirr/trunk/services/hadoop/src/test/resources/whirr-hadoop-test.properties   (with props)
    incubator/whirr/trunk/services/zookeeper/src/test/resources/whirr-zookeeper-test.properties   (with props)
Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/Main.java
    incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ClusterSpecCommand.java
    incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/DestroyClusterCommand.java
    incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/LaunchClusterCommand.java
    incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/MainTest.java
    incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/DestroyClusterCommandTest.java
    incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/LaunchClusterCommandTest.java
    incubator/whirr/trunk/core/pom.xml
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/jclouds/FirewallSettings.java
    incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/integration/CassandraServiceTest.java
    incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/integration/HadoopServiceTest.java
    incubator/whirr/trunk/services/zookeeper/src/test/java/org/apache/whirr/service/zookeeper/integration/ZooKeeperServiceTest.java

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Fri Aug 20 22:22:46 2010
@@ -46,6 +46,8 @@ Trunk (unreleased changes)
     WHIRR-58. Introduce naming consistency for cloud service providers.
     (Adrian Cole via tomwhite)
 
+    WHIRR-75. Use Commons Configuration to manage cluster specs. (tomwhite)
+
   BUG FIXES
 
     WHIRR-50. Cassandra POM should depend on top-level. (tomwhite)

Modified: incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/Main.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/Main.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/Main.java (original)
+++ incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/Main.java Fri Aug 20 22:22:46 2010
@@ -18,6 +18,8 @@
 
 package org.apache.whirr.cli;
 
+import com.google.common.collect.Maps;
+
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
@@ -28,8 +30,6 @@ import java.util.Map;
 import org.apache.whirr.cli.command.DestroyClusterCommand;
 import org.apache.whirr.cli.command.LaunchClusterCommand;
 
-import com.google.common.collect.Maps;
-
 /**
  * The entry point for the Whirr CLI.
  */

Modified: incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ClusterSpecCommand.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ClusterSpecCommand.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ClusterSpecCommand.java (original)
+++ incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/ClusterSpecCommand.java Fri Aug 20 22:22:46 2010
@@ -18,19 +18,28 @@
 
 package org.apache.whirr.cli.command;
 
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.apache.whirr.service.ClusterSpec.Property.CLUSTER_NAME;
+import static org.apache.whirr.service.ClusterSpec.Property.IDENTITY;
+import static org.apache.whirr.service.ClusterSpec.Property.SERVICE_NAME;
 
-import java.util.Arrays;
-import java.util.List;
+import com.google.common.collect.Maps;
 
+import java.util.EnumSet;
+import java.util.Map;
+
+import joptsimple.ArgumentAcceptingOptionSpec;
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
 import joptsimple.OptionSpec;
 
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.whirr.cli.Command;
 import org.apache.whirr.service.ClusterSpec;
 import org.apache.whirr.service.ServiceFactory;
-import org.apache.whirr.service.ClusterSpec.InstanceTemplate;
+import org.apache.whirr.service.ClusterSpec.Property;
 
 /**
  * An abstract command for interacting with clusters.
@@ -40,36 +49,53 @@ public abstract class ClusterSpecCommand
   protected ServiceFactory factory;
   
   protected OptionParser parser = new OptionParser();
-  protected OptionSpec<String> cloudProvider = parser.accepts("cloud-provider")
-      .withRequiredArg().defaultsTo("ec2").ofType(String.class);
-  protected OptionSpec<String> cloudIdentity = parser.accepts("cloud-identity")
-      .withRequiredArg().ofType(String.class);
-  protected OptionSpec<String> cloudCredential = parser.accepts("cloud-credential")
-      .withRequiredArg().ofType(String.class);
-  protected OptionSpec<String> secretKeyFile = parser.accepts("secret-key-file")
-      .withRequiredArg()
-      .defaultsTo(System.getProperty("user.home") + "/.ssh/id_rsa")
-      .ofType(String.class);
-  
+  private Map<Property, OptionSpec> optionSpecs;
+  private OptionSpec<String> configOption = parser.accepts("config")
+    .withRequiredArg().ofType(String.class);
+
   public ClusterSpecCommand(String name, String description, ServiceFactory factory) {
     super(name, description);
     this.factory = factory;
+    
+    optionSpecs = Maps.newHashMap();
+    for (Property property : EnumSet.allOf(Property.class)) {
+      ArgumentAcceptingOptionSpec<?> spec = parser.accepts(property.getSimpleName())
+        .withRequiredArg()
+        .ofType(property.getType());
+      if (property.hasMultipleArguments()) {
+        spec.withValuesSeparatedBy(',');
+      }
+      optionSpecs.put(property, spec);
+    }
   }
   
-  protected ClusterSpec getClusterSpec(OptionSet optionSet,
-        InstanceTemplate... instanceTemplates) {
-    return getClusterSpec(optionSet, Arrays.asList(instanceTemplates));
-  }
-  
-  protected ClusterSpec getClusterSpec(OptionSet optionSet,
-        List<InstanceTemplate> instanceTemplates) {
-    ClusterSpec clusterSpec = new ClusterSpec(instanceTemplates);
-    clusterSpec.setProvider(optionSet.valueOf(cloudProvider));
-    clusterSpec.setIdentity(checkNotNull(optionSet.valueOf(cloudIdentity),
-        "cloud-identity"));
-    clusterSpec.setCredential(optionSet.valueOf(cloudCredential));
-    clusterSpec.setSecretKeyFile(optionSet.valueOf(secretKeyFile));
-    return clusterSpec;
+  protected ClusterSpec getClusterSpec(OptionSet optionSet) throws ConfigurationException {
+    Configuration optionsConfig = new PropertiesConfiguration();
+    for (Map.Entry<Property, OptionSpec> entry : optionSpecs.entrySet()) {
+      Property property = entry.getKey();
+      OptionSpec option = entry.getValue();
+      if (property.hasMultipleArguments()) {
+        optionsConfig.setProperty(property.getConfigName(),
+            optionSet.valuesOf(option));
+      } else {
+        optionsConfig.setProperty(property.getConfigName(),
+            optionSet.valueOf(option));
+      }
+    }
+    CompositeConfiguration config = new CompositeConfiguration();
+    config.addConfiguration(optionsConfig);
+    if (optionSet.has(configOption)) {
+      Configuration defaults = new PropertiesConfiguration(optionSet.valueOf(configOption));
+      config.addConfiguration(defaults);
+    }
+
+    for (Property required : EnumSet.of(SERVICE_NAME, CLUSTER_NAME, IDENTITY)) {
+      if (config.getString(required.getConfigName()) == null) {
+        throw new IllegalArgumentException(String.format("Option '%s' not set.",
+            required.getSimpleName()));
+      }
+    }
+    return ClusterSpec.fromConfiguration(config);
   }
 
 }

Modified: incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/DestroyClusterCommand.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/DestroyClusterCommand.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/DestroyClusterCommand.java (original)
+++ incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/DestroyClusterCommand.java Fri Aug 20 22:22:46 2010
@@ -49,22 +49,25 @@ public class DestroyClusterCommand exten
     
     OptionSet optionSet = parser.parse(args.toArray(new String[0]));
 
-    List<String> nonOptionArguments = optionSet.nonOptionArguments();
-    if (nonOptionArguments.size() < 2) {
+    if (!optionSet.nonOptionArguments().isEmpty()) {
       printUsage(parser, err);
       return -1;
     }
-    String serviceName = nonOptionArguments.get(0);
-    ClusterSpec clusterSpec = getClusterSpec(optionSet);
-    clusterSpec.setClusterName(nonOptionArguments.get(1));
+    try {
+      ClusterSpec clusterSpec = getClusterSpec(optionSet);
 
-    Service service = factory.create(serviceName);
-    service.destroyCluster(clusterSpec);
-    return 0;
+      Service service = factory.create(clusterSpec.getServiceName());
+      service.destroyCluster(clusterSpec);
+      return 0;
+    } catch (IllegalArgumentException e) {
+      err.println(e.getMessage());
+      printUsage(parser, err);
+      return -1;
+    }
   }
 
   private void printUsage(OptionParser parser, PrintStream stream) throws IOException {
-    stream.println("Usage: whirr destroy-cluster [OPTIONS] <service-name> <cluster-name>");
+    stream.println("Usage: whirr destroy-cluster [OPTIONS]");
     stream.println();
     parser.printHelpOn(stream);
   }

Modified: incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/LaunchClusterCommand.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/LaunchClusterCommand.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/LaunchClusterCommand.java (original)
+++ incubator/whirr/trunk/cli/src/main/java/org/apache/whirr/cli/command/LaunchClusterCommand.java Fri Aug 20 22:22:46 2010
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.PrintStream;
 import java.util.List;
-import java.util.Set;
 
 import joptsimple.OptionParser;
 import joptsimple.OptionSet;
@@ -31,11 +30,6 @@ import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.ClusterSpec;
 import org.apache.whirr.service.Service;
 import org.apache.whirr.service.ServiceFactory;
-import org.apache.whirr.service.ClusterSpec.InstanceTemplate;
-
-import com.google.common.base.Splitter;
-import com.google.common.collect.Sets;
-import com.google.inject.internal.Lists;
 
 /**
  * A command to launch a new cluster.
@@ -56,41 +50,28 @@ public class LaunchClusterCommand extend
     
     OptionSet optionSet = parser.parse(args.toArray(new String[0]));
 
-    List<String> nonOptionArguments = optionSet.nonOptionArguments();
-    if (nonOptionArguments.size() < 4) {
+    if (!optionSet.nonOptionArguments().isEmpty()) {
       printUsage(parser, err);
       return -1;
     }
     
-    final int nonTemplateArgumentsSize = 2;
-
-    if ((nonOptionArguments.size() - nonTemplateArgumentsSize) % 2 == 1) {
+    try {
+      ClusterSpec clusterSpec = getClusterSpec(optionSet);
+      Service service = factory.create(clusterSpec.getServiceName());
+      Cluster cluster = service.launchCluster(clusterSpec);
+      out.printf("Started cluster of %s instances\n",
+          cluster.getInstances().size());
+      out.println(cluster);
+      return 0;
+    } catch (IllegalArgumentException e) {
+      err.println(e.getMessage());
       printUsage(parser, err);
       return -1;
     }
-
-    List<InstanceTemplate> templates = Lists.newArrayList();
-    for (int i = 0; i < (nonOptionArguments.size() - 1) / 2; i++) {
-      int number = Integer.parseInt(nonOptionArguments.get(2 * i + nonTemplateArgumentsSize));
-      String rolesString = nonOptionArguments.get(2 * i + nonTemplateArgumentsSize + 1);
-      Set<String> roles = Sets.newHashSet(Splitter.on("+").split(rolesString));
-      templates.add(new InstanceTemplate(number, roles));
-    }
-    ClusterSpec clusterSpec = getClusterSpec(optionSet, templates);
-    String serviceName = nonOptionArguments.get(0);
-    clusterSpec.setClusterName(nonOptionArguments.get(1));
-    
-    Service service = factory.create(serviceName);
-    Cluster cluster = service.launchCluster(clusterSpec);
-    out.printf("Started cluster of %s instances\n",
-        cluster.getInstances().size());
-    out.println(cluster);
-    return 0;
   }
 
   private void printUsage(OptionParser parser, PrintStream stream) throws IOException {
-    stream.println("Usage: whirr launch-cluster [OPTIONS] <service-name> " +
-        "<cluster-name> <num> <roles> [<num> <roles>]*");
+    stream.println("Usage: whirr launch-cluster [OPTIONS]");
     stream.println();
     parser.printHelpOn(stream);
   }

Modified: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/MainTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/MainTest.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/MainTest.java (original)
+++ incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/MainTest.java Fri Aug 20 22:22:46 2010
@@ -25,6 +25,8 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import com.google.inject.internal.Lists;
+
 import java.io.ByteArrayOutputStream;
 import java.io.InputStream;
 import java.io.PrintStream;
@@ -33,8 +35,6 @@ import java.util.List;
 
 import org.junit.Test;
 
-import com.google.inject.internal.Lists;
-
 public class MainTest {
   
   static class TestCommand extends Command {

Added: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ClusterSpecCommandTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ClusterSpecCommandTest.java?rev=987653&view=auto
==============================================================================
--- incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ClusterSpecCommandTest.java (added)
+++ incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ClusterSpecCommandTest.java Fri Aug 20 22:22:46 2010
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.whirr.cli.command;
+
+import static org.hamcrest.Matchers.is;
+import static org.junit.Assert.assertThat;
+
+import java.io.InputStream;
+import java.io.PrintStream;
+import java.util.List;
+
+import joptsimple.OptionSet;
+
+import org.apache.whirr.service.ClusterSpec;
+import org.apache.whirr.service.ServiceFactory;
+import org.junit.Test;
+
+public class ClusterSpecCommandTest {
+
+  @Test
+  public void testOverrides() throws Exception {
+    ClusterSpecCommand clusterSpecCommand = new ClusterSpecCommand("name",
+        "description", new ServiceFactory()) {
+      @Override
+      public int run(InputStream in, PrintStream out, PrintStream err,
+          List<String> args) throws Exception {
+        return 0;
+      }
+    };
+    
+    OptionSet optionSet = clusterSpecCommand.parser.parse(
+        "--service-name", "overridden-test-service",
+        "--config", "whirr-override-test.properties");
+    ClusterSpec clusterSpec = clusterSpecCommand.getClusterSpec(optionSet);
+    assertThat(clusterSpec.getServiceName(), is("overridden-test-service"));
+    assertThat(clusterSpec.getClusterName(), is("test-cluster"));
+  }
+  
+}

Propchange: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/ClusterSpecCommandTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/DestroyClusterCommandTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/DestroyClusterCommandTest.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/DestroyClusterCommandTest.java (original)
+++ incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/DestroyClusterCommandTest.java Fri Aug 20 22:22:46 2010
@@ -25,6 +25,8 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import com.google.inject.internal.Lists;
+
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.util.Collections;
@@ -37,8 +39,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.internal.matchers.StringContains;
 
-import com.google.inject.internal.Lists;
-
 public class DestroyClusterCommandTest {
 
   private ByteArrayOutputStream outBytes;
@@ -55,7 +55,7 @@ public class DestroyClusterCommandTest {
   }
   
   @Test
-  public void testNoArgs() throws Exception {
+  public void testInsufficientOptions() throws Exception {
     DestroyClusterCommand command = new DestroyClusterCommand();
     int rc = command.run(null, null, err, Collections.<String>emptyList());
     assertThat(rc, is(-1));
@@ -63,8 +63,7 @@ public class DestroyClusterCommandTest {
   }
   
   private Matcher<String> containsUsageString() {
-    return StringContains.containsString("Usage: whirr destroy-cluster " +
-        "[OPTIONS] <service-name> <cluster-name>");
+    return StringContains.containsString("Usage: whirr destroy-cluster [OPTIONS]");
   }
   
   @Test
@@ -77,14 +76,16 @@ public class DestroyClusterCommandTest {
     DestroyClusterCommand command = new DestroyClusterCommand(factory);
     
     int rc = command.run(null, out, null, Lists.newArrayList(
-        "--cloud-provider", "rackspace",
-        "--cloud-identity", "myusername", "--cloud-credential", "mypassword",
-        "--secret-key-file", "secret-key",
-        "test-service", "test-cluster"));
+        "--service-name", "test-service",
+        "--cluster-name", "test-cluster",
+        "--provider", "rackspace",
+        "--identity", "myusername", "--credential", "mypassword",
+        "--secret-key-file", "secret-key"));
     
     assertThat(rc, is(0));
 
     ClusterSpec expectedClusterSpec = new ClusterSpec();
+    expectedClusterSpec.setServiceName("test-service");
     expectedClusterSpec.setProvider("rackspace");
     expectedClusterSpec.setIdentity("myusername");
     expectedClusterSpec.setCredential("mypassword");

Modified: incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/LaunchClusterCommandTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/LaunchClusterCommandTest.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/LaunchClusterCommandTest.java (original)
+++ incubator/whirr/trunk/cli/src/test/java/org/apache/whirr/cli/command/LaunchClusterCommandTest.java Fri Aug 20 22:22:46 2010
@@ -18,6 +18,7 @@
 
 package org.apache.whirr.cli.command;
 
+import static org.apache.whirr.service.ClusterSpec.Property.INSTANCE_TEMPLATES;
 import static org.hamcrest.Matchers.containsString;
 import static org.hamcrest.Matchers.is;
 import static org.junit.Assert.assertThat;
@@ -26,11 +27,15 @@ import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.verify;
 import static org.mockito.Mockito.when;
 
+import com.google.common.collect.Sets;
+import com.google.inject.internal.Lists;
+
 import java.io.ByteArrayOutputStream;
 import java.io.PrintStream;
 import java.util.Collections;
 
-import org.apache.whirr.cli.command.LaunchClusterCommand;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.ClusterSpec;
 import org.apache.whirr.service.Service;
@@ -40,9 +45,6 @@ import org.junit.Before;
 import org.junit.Test;
 import org.junit.internal.matchers.StringContains;
 
-import com.google.common.collect.Sets;
-import com.google.inject.internal.Lists;
-
 public class LaunchClusterCommandTest {
 
   private ByteArrayOutputStream outBytes;
@@ -58,34 +60,15 @@ public class LaunchClusterCommandTest {
     err = new PrintStream(errBytes);
   }
   @Test
-  public void testNoArgs() throws Exception {
+  public void testInsufficientArgs() throws Exception {
     LaunchClusterCommand command = new LaunchClusterCommand();
     int rc = command.run(null, null, err, Collections.<String>emptyList());
     assertThat(rc, is(-1));
     assertThat(errBytes.toString(), containsUsageString());
   }
   
-  @Test
-  public void testMissingNumberAndRoles() throws Exception {
-    LaunchClusterCommand command = new LaunchClusterCommand();
-    int rc = command.run(null, null, err, Lists.newArrayList("test-service",
-        "test-cluster"));
-    assertThat(rc, is(-1));
-    assertThat(errBytes.toString(), containsUsageString());
-  }
-  
-  @Test
-  public void testMissingRoles() throws Exception {
-    LaunchClusterCommand command = new LaunchClusterCommand();
-    int rc = command.run(null, null, err, Lists.newArrayList("test-service",
-        "test-cluster", "1", "role1+role2", "2"));
-    assertThat(rc, is(-1));
-    assertThat(errBytes.toString(), containsUsageString());
-  }
-  
   private Matcher<String> containsUsageString() {
-    return StringContains.containsString("Usage: whirr launch-cluster " +
-        "[OPTIONS] <service-name> <cluster-name> <num> <roles> [<num> <roles>]*");
+    return StringContains.containsString("Usage: whirr launch-cluster [OPTIONS]");
   }
   
   @Test
@@ -100,17 +83,21 @@ public class LaunchClusterCommandTest {
     LaunchClusterCommand command = new LaunchClusterCommand(factory);
     
     int rc = command.run(null, out, null, Lists.newArrayList(
-        "--cloud-provider", "rackspace",
-        "--cloud-identity", "myusername", "--cloud-credential", "mypassword",
-        "--secret-key-file", "secret-key",
-        "test-service", "test-cluster", "1", "role1+role2", "2", "role3"));
+        "--service-name", "test-service",
+        "--cluster-name", "test-cluster",
+        "--instance-templates", "1 role1+role2,2 role3",
+        "--provider", "rackspace",
+        "--identity", "myusername", "--credential", "mypassword",
+        "--secret-key-file", "secret-key"));
     
     assertThat(rc, is(0));
 
-    ClusterSpec expectedClusterSpec = new ClusterSpec(
+    ClusterSpec expectedClusterSpec = new ClusterSpec();
+    expectedClusterSpec.setInstanceTemplates(Lists.newArrayList(
         new ClusterSpec.InstanceTemplate(1, Sets.newHashSet("role1", "role2")),
         new ClusterSpec.InstanceTemplate(2, Sets.newHashSet("role3"))
-    );
+    ));
+    expectedClusterSpec.setServiceName("test-service");
     expectedClusterSpec.setProvider("rackspace");
     expectedClusterSpec.setIdentity("myusername");
     expectedClusterSpec.setCredential("mypassword");

Added: incubator/whirr/trunk/cli/src/test/resources/whirr-override-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/src/test/resources/whirr-override-test.properties?rev=987653&view=auto
==============================================================================
--- incubator/whirr/trunk/cli/src/test/resources/whirr-override-test.properties (added)
+++ incubator/whirr/trunk/cli/src/test/resources/whirr-override-test.properties Fri Aug 20 22:22:46 2010
@@ -0,0 +1,21 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+whirr.service-name=test-service
+whirr.cluster-name=test-cluster
+whirr.provider=test-provider
+whirr.identity=test-identity

Propchange: incubator/whirr/trunk/cli/src/test/resources/whirr-override-test.properties
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/whirr/trunk/core/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/pom.xml?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/core/pom.xml (original)
+++ incubator/whirr/trunk/core/pom.xml Fri Aug 20 22:22:46 2010
@@ -70,6 +70,11 @@
       <version>3.3.0</version>
     </dependency>
     <dependency>
+      <groupId>commons-configuration</groupId>
+      <artifactId>commons-configuration</artifactId>
+      <version>1.6</version>
+    </dependency>
+    <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <version>1.6.0</version>

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterSpec.java Fri Aug 20 22:22:46 2010
@@ -18,26 +18,61 @@
 
 package org.apache.whirr.service;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 import com.google.common.base.Charsets;
 import com.google.common.base.Objects;
-import com.google.common.collect.Maps;
+import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 import com.google.common.io.Files;
 
 import java.io.File;
 import java.io.IOException;
-import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 
+import org.apache.commons.configuration.Configuration;
+
 /**
  * This class represents the specification of a cluster. It is used to describe
  * the properties of a cluster before it is launched.
  */
 public class ClusterSpec {
   
+  public enum Property {
+    SERVICE_NAME(String.class, false),
+    INSTANCE_TEMPLATES(String.class, false),
+    PROVIDER(String.class, false),
+    CREDENTIAL(String.class, false),
+    IDENTITY(String.class, false),
+    CLUSTER_NAME(String.class, false),
+    SECRET_KEY_FILE(String.class, false),
+    CLIENT_CIDRS(String.class, true);
+    
+    private Class<?> type;
+    private boolean multipleArguments;
+    Property(Class<?> type, boolean multipleArguments) {
+      this.type = type;
+      this.multipleArguments = multipleArguments;
+    }
+    
+    public String getSimpleName() {
+      return name().toLowerCase().replace('_', '-');
+    }
+
+    public String getConfigName() {
+      return "whirr." + getSimpleName();
+    }
+    
+    public Class<?> getType() {
+      return type;
+    }
+    
+    public boolean hasMultipleArguments() {
+      return multipleArguments;
+    }
+  }
+  
   /**
    * This class describes the type of instances that should be in the cluster.
    * This is done by specifying the number of instances in each role.
@@ -83,50 +118,52 @@ public class ClusterSpec {
         .toString();
     }
     
+    public static List<InstanceTemplate> parse(String... strings) {
+      List<InstanceTemplate> templates = Lists.newArrayList();
+      for (String s : strings) {
+        String[] parts = s.split(" ");
+        int num = Integer.parseInt(parts[0]);
+        templates.add(new InstanceTemplate(num, parts[1].split("\\+")));
+      }
+      return templates;
+    }
   }
   
-  private Properties configuration;
-  private List<InstanceTemplate> instanceTemplates;
-  private Map<Set<String>, InstanceTemplate> instanceTemplatesMap = Maps.newHashMap();
-
+  private List<InstanceTemplate> instanceTemplates = Lists.newArrayList();
   private String serviceName;
   private String provider;
   private String identity;
   private String credential;
   private String clusterName;
   private String secretKeyFile;
-  private String[] clientCidrs;
+  private List<String> clientCidrs = Lists.newArrayList();
   
-  public ClusterSpec(InstanceTemplate... instanceTemplates) {
-    this(Arrays.asList(instanceTemplates));
-  }
-
-  public ClusterSpec(List<InstanceTemplate> instanceTemplates) {
-    this(new Properties(), instanceTemplates);
-  }
-
-  /**
-   * @param configuration The configuration properties for the service. These
-   * take precedence over service defaults.
-   */
-  public ClusterSpec(Properties configuration, List<InstanceTemplate> instanceTemplates) {
-    this.configuration = configuration;
-    this.instanceTemplates = instanceTemplates;
-    for (InstanceTemplate template : instanceTemplates) {
-      instanceTemplatesMap.put(template.roles, template);
-    }
-  }
-
-  public Properties getConfiguration() {
-    return configuration;
+  public static ClusterSpec fromConfiguration(Configuration config) {
+    ClusterSpec spec = new ClusterSpec();
+    spec.setServiceName(config.getString(Property.SERVICE_NAME.getConfigName()));
+    spec.setInstanceTemplates(InstanceTemplate.parse(
+        config.getStringArray(Property.INSTANCE_TEMPLATES.getConfigName())));
+    spec.setProvider(config.getString(Property.PROVIDER.getConfigName()));
+    spec.setIdentity(checkNotNull(
+        config.getString(Property.IDENTITY.getConfigName()), Property.IDENTITY));
+    spec.setCredential(config.getString(Property.CREDENTIAL.getConfigName()));
+    spec.setClusterName(config.getString(Property.CLUSTER_NAME.getConfigName()));
+    spec.setSecretKeyFile(config.getString(Property.SECRET_KEY_FILE.getConfigName()));
+    spec.setClientCidrs(config.getList(Property.CLIENT_CIDRS.getConfigName()));
+    return spec;
   }
 
   public List<InstanceTemplate> getInstanceTemplates() {
     return instanceTemplates;
   }
   
-  public InstanceTemplate getInstanceTemplate(Set<String> roles) {
-    return instanceTemplatesMap.get(roles);
+  public InstanceTemplate getInstanceTemplate(final Set<String> roles) {
+    for (InstanceTemplate template : instanceTemplates) {
+      if (roles.equals(template.roles)) {
+        return template;
+      }
+    }
+    return null;
   }
   
   public InstanceTemplate getInstanceTemplate(String... roles) {
@@ -151,32 +188,42 @@ public class ClusterSpec {
   public String getSecretKeyFile() {
     return secretKeyFile;
   }
-  public String[] getClientCidrs() {
+  public List<String> getClientCidrs() {
     return clientCidrs;
   }
   
+  public void setInstanceTemplates(List<InstanceTemplate> instanceTemplates) {
+    this.instanceTemplates = instanceTemplates;
+  }
+
   public void setServiceName(String serviceName) {
     this.serviceName = serviceName;
   }
+
   public void setProvider(String provider) {
     this.provider = provider;
   }
+
   public void setIdentity(String identity) {
     this.identity = identity;
   }
+
   public void setCredential(String credential) {
     this.credential = credential;
   }
+
   public void setClusterName(String clusterName) {
     this.clusterName = clusterName;
   }
+
   public void setSecretKeyFile(String secretKeyFile) {
     this.secretKeyFile = secretKeyFile;
   }
-  public void setClientCidrs(String[] clientCidrs) {
+
+  public void setClientCidrs(List<String> clientCidrs) {
     this.clientCidrs = clientCidrs;
   }
-  
+
   //
   public String readPrivateKey() throws IOException {
     return Files.toString(new File(getSecretKeyFile()), Charsets.UTF_8);
@@ -189,26 +236,26 @@ public class ClusterSpec {
   public boolean equals(Object o) {
     if (o instanceof ClusterSpec) {
       ClusterSpec that = (ClusterSpec) o;
-      return Objects.equal(configuration, that.configuration)
-        && Objects.equal(instanceTemplates, that.instanceTemplates)
+      return Objects.equal(instanceTemplates, that.instanceTemplates)
         && Objects.equal(serviceName, that.serviceName)
         && Objects.equal(provider, that.provider)
         && Objects.equal(identity, that.identity)
         && Objects.equal(credential, that.credential)
         && Objects.equal(clusterName, that.clusterName)
-        && Objects.equal(secretKeyFile, that.secretKeyFile);
+        && Objects.equal(secretKeyFile, that.secretKeyFile)
+        && Objects.equal(clientCidrs, that.clientCidrs);
     }
     return false;
   }
   
   public int hashCode() {
-    return Objects.hashCode(configuration, instanceTemplates, serviceName,
-        provider, identity, credential, clusterName, secretKeyFile);
+    return Objects.hashCode(instanceTemplates, serviceName,
+        provider, identity, credential, clusterName, secretKeyFile,
+        clientCidrs);
   }
   
   public String toString() {
     return Objects.toStringHelper(this)
-      .add("configuration", configuration)
       .add("instanceTemplates", instanceTemplates)
       .add("serviceName", serviceName)
       .add("provider", provider)
@@ -216,6 +263,7 @@ public class ClusterSpec {
       .add("credential", credential)
       .add("clusterName", clusterName)
       .add("secretKeyFile", secretKeyFile)
+      .add("clientCidrs", clientCidrs)
       .toString();
   }
   

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/jclouds/FirewallSettings.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/jclouds/FirewallSettings.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/jclouds/FirewallSettings.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/jclouds/FirewallSettings.java Fri Aug 20 22:22:46 2010
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.net.HttpURLConnection;
 import java.net.URL;
 import java.util.Collections;
+import java.util.List;
 import java.util.Set;
 
 import org.apache.commons.io.IOUtils;
@@ -33,6 +34,7 @@ import org.jclouds.compute.ComputeServic
 import org.jclouds.compute.domain.NodeMetadata;
 
 import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
 
 /**
  * Utility functions for controlling firewall settings for a cluster.
@@ -47,7 +49,7 @@ public class FirewallSettings {
     URL url = new URL("http://checkip.amazonaws.com/");
     HttpURLConnection connection = (HttpURLConnection) url.openConnection();
     connection.connect();
-    return IOUtils.toString(connection.getInputStream());
+    return IOUtils.toString(connection.getInputStream()).trim() + "/32";
   }
 
   public static void authorizeIngress(ComputeServiceContext computeServiceContext,
@@ -61,21 +63,20 @@ public class FirewallSettings {
       NodeMetadata node, ClusterSpec clusterSpec, String ip, int... ports) {
     
     authorizeIngress(computeServiceContext, Collections.singleton(node),
-        clusterSpec, new String[] { ip + "/32" }, ports);
+        clusterSpec, Lists.newArrayList(ip + "/32"), ports);
   }
 
   public static void authorizeIngress(ComputeServiceContext computeServiceContext,
       Set<? extends NodeMetadata> nodes, ClusterSpec clusterSpec, int... ports) throws IOException {
-    String[] cidrs = clusterSpec.getClientCidrs();
-    if (cidrs == null) {
-      cidrs = new String[] { getOriginatingIp() };
+    List<String> cidrs = clusterSpec.getClientCidrs();
+    if (cidrs == null || cidrs.isEmpty()) {
+      cidrs = Lists.newArrayList(getOriginatingIp());
     }
-    authorizeIngress(computeServiceContext, nodes,
-        clusterSpec, clusterSpec.getClientCidrs(), ports);
+    authorizeIngress(computeServiceContext, nodes, clusterSpec, cidrs, ports);
   }
 
   private static void authorizeIngress(ComputeServiceContext computeServiceContext,
-      Set<? extends NodeMetadata> nodes, ClusterSpec clusterSpec, String[] cidrs, int... ports) {
+      Set<? extends NodeMetadata> nodes, ClusterSpec clusterSpec, List<String> cidrs, int... ports) {
     
     if (clusterSpec.getProvider().equals("ec2")) {
       // This code (or something like it) may be added to jclouds (see

Modified: incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/integration/CassandraServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/integration/CassandraServiceTest.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/integration/CassandraServiceTest.java (original)
+++ incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/integration/CassandraServiceTest.java Fri Aug 20 22:22:46 2010
@@ -18,7 +18,8 @@
 
 package org.apache.whirr.service.cassandra.integration;
 
-import static com.google.common.base.Preconditions.checkNotNull;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
@@ -28,13 +29,18 @@ import java.util.Set;
 
 import org.apache.cassandra.thrift.Cassandra;
 import org.apache.cassandra.thrift.TokenRange;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.thrift.TException;
 import org.apache.thrift.protocol.TBinaryProtocol;
 import org.apache.thrift.transport.TSocket;
 import org.apache.whirr.service.Cluster;
 import org.apache.whirr.service.ClusterSpec;
+import org.apache.whirr.service.Service;
+import org.apache.whirr.service.ServiceFactory;
 import org.apache.whirr.service.Cluster.Instance;
-import org.apache.whirr.service.ClusterSpec.InstanceTemplate;
 import org.apache.whirr.service.cassandra.CassandraService;
 import org.junit.After;
 import org.junit.Before;
@@ -42,7 +48,6 @@ import org.junit.Test;
 
 public class CassandraServiceTest {
 
-  private String clusterName = "cassandraclustertest";
   private static final String KEYSPACE = "Keyspace1";
 
   private ClusterSpec clusterSpec;
@@ -50,27 +55,16 @@ public class CassandraServiceTest {
   private Cluster cluster;
 
   @Before
-  public void setUp() throws IOException, InterruptedException {
-    String secretKeyFile;
-    try {
-      secretKeyFile = checkNotNull(System.getProperty("whirr.test.ssh.keyfile"), "whirr.test.ssh.keyfile");
-    } catch (NullPointerException e) {
-      secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
-    }
-    clusterSpec = new ClusterSpec(new InstanceTemplate(2,
-        CassandraService.CASSANDRA_ROLE));
-    clusterSpec.setProvider(checkNotNull(System.getProperty(
-        "whirr.test.provider", "ec2")));
-    clusterSpec.setIdentity(checkNotNull(System.getProperty("whirr.test.identity"), "whirr.test.identity"));
-    clusterSpec.setCredential(checkNotNull(System.getProperty("whirr.test.credential"), "whirr.test.credential"));
-    String cidrs = System.getProperty("whirr.test.client-cidrs");
-    if (cidrs != null) {
-      clusterSpec.setClientCidrs(cidrs.split(","));
-    }
-    clusterSpec.setSecretKeyFile(secretKeyFile);
-    clusterSpec.setClusterName(clusterName);
-    service = new CassandraService();
-
+  public void setUp() throws ConfigurationException, IOException {
+    CompositeConfiguration config = new CompositeConfiguration();
+    if (System.getProperty("config") != null) {
+      config.addConfiguration(new PropertiesConfiguration(System.getProperty("config")));
+    }
+    config.addConfiguration(new PropertiesConfiguration("whirr-cassandra-test.properties"));
+    clusterSpec = ClusterSpec.fromConfiguration(config);
+    Service s = new ServiceFactory().create(clusterSpec.getServiceName());
+    assertThat(s, instanceOf(CassandraService.class));
+    service = (CassandraService) s;
     cluster = service.launchCluster(clusterSpec);
 
     // give it a sec to boot up the cluster

Added: incubator/whirr/trunk/services/cassandra/src/test/resources/whirr-cassandra-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cassandra/src/test/resources/whirr-cassandra-test.properties?rev=987653&view=auto
==============================================================================
--- incubator/whirr/trunk/services/cassandra/src/test/resources/whirr-cassandra-test.properties (added)
+++ incubator/whirr/trunk/services/cassandra/src/test/resources/whirr-cassandra-test.properties Fri Aug 20 22:22:46 2010
@@ -0,0 +1,24 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+whirr.service-name=cassandra
+whirr.cluster-name=cassandraclustertest
+whirr.instance-templates=2 cassandra
+whirr.provider=ec2
+whirr.identity=${sys:whirr.test.identity}
+whirr.credential=${sys:whirr.test.credential}
+whirr.secret-key-file=${sys:user.home}/.ssh/id_rsa

Propchange: incubator/whirr/trunk/services/cassandra/src/test/resources/whirr-cassandra-test.properties
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/integration/HadoopServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/integration/HadoopServiceTest.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/integration/HadoopServiceTest.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/integration/HadoopServiceTest.java Fri Aug 20 22:22:46 2010
@@ -18,9 +18,10 @@
 
 package org.apache.whirr.service.hadoop.integration;
 
-import static com.google.common.base.Preconditions.checkNotNull;
 import static junit.framework.Assert.assertEquals;
 import static junit.framework.Assert.assertNull;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
 
 import java.io.BufferedReader;
 import java.io.IOException;
@@ -30,6 +31,9 @@ import java.io.OutputStreamWriter;
 import java.io.Writer;
 import java.util.Map.Entry;
 
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -44,7 +48,8 @@ import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapred.lib.LongSumReducer;
 import org.apache.hadoop.mapred.lib.TokenCountMapper;
 import org.apache.whirr.service.ClusterSpec;
-import org.apache.whirr.service.ClusterSpec.InstanceTemplate;
+import org.apache.whirr.service.Service;
+import org.apache.whirr.service.ServiceFactory;
 import org.apache.whirr.service.hadoop.HadoopCluster;
 import org.apache.whirr.service.hadoop.HadoopProxy;
 import org.apache.whirr.service.hadoop.HadoopService;
@@ -54,34 +59,22 @@ import org.junit.Test;
 
 public class HadoopServiceTest {
   
-  private String clusterName = "hadoopclustertest";
-  
   private ClusterSpec clusterSpec;
   private HadoopService service;
   private HadoopProxy proxy;
   private HadoopCluster cluster;
   
   @Before
-  public void setUp() throws IOException {
-    String secretKeyFile;
-    try {
-       secretKeyFile = checkNotNull(System.getProperty("whirr.test.ssh.keyfile"));
-    } catch (NullPointerException e) {
-       secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
-    }
-    clusterSpec = new ClusterSpec(
-        new InstanceTemplate(1, HadoopService.MASTER_ROLE),
-        new InstanceTemplate(1, HadoopService.WORKER_ROLE));
-    clusterSpec.setProvider(checkNotNull(System.getProperty("whirr.test.provider", "ec2")));
-    clusterSpec.setIdentity(checkNotNull(System.getProperty("whirr.test.identity")));
-    clusterSpec.setCredential(checkNotNull(System.getProperty("whirr.test.credential")));
-    String cidrs = System.getProperty("whirr.test.client-cidrs");
-    if (cidrs != null) {
-      clusterSpec.setClientCidrs(cidrs.split(","));
+  public void setUp() throws ConfigurationException, IOException {
+    CompositeConfiguration config = new CompositeConfiguration();
+    if (System.getProperty("config") != null) {
+      config.addConfiguration(new PropertiesConfiguration(System.getProperty("config")));
     }
-    clusterSpec.setSecretKeyFile(secretKeyFile);
-    clusterSpec.setClusterName(clusterName);
-    service = new HadoopService();
+    config.addConfiguration(new PropertiesConfiguration("whirr-hadoop-test.properties"));
+    clusterSpec = ClusterSpec.fromConfiguration(config);
+    Service s = new ServiceFactory().create(clusterSpec.getServiceName());
+    assertThat(s, instanceOf(HadoopService.class));
+    service = (HadoopService) s;
     
     cluster = service.launchCluster(clusterSpec);
     proxy = new HadoopProxy(clusterSpec, cluster);

Added: incubator/whirr/trunk/services/hadoop/src/test/resources/whirr-hadoop-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/test/resources/whirr-hadoop-test.properties?rev=987653&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/test/resources/whirr-hadoop-test.properties (added)
+++ incubator/whirr/trunk/services/hadoop/src/test/resources/whirr-hadoop-test.properties Fri Aug 20 22:22:46 2010
@@ -0,0 +1,24 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+whirr.service-name=hadoop
+whirr.cluster-name=hadoopclustertest
+whirr.instance-templates=1 jt+nn,1 dn+tt
+whirr.provider=ec2
+whirr.identity=${sys:whirr.test.identity}
+whirr.credential=${sys:whirr.test.credential}
+whirr.secret-key-file=${sys:user.home}/.ssh/id_rsa

Propchange: incubator/whirr/trunk/services/hadoop/src/test/resources/whirr-hadoop-test.properties
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/whirr/trunk/services/zookeeper/src/test/java/org/apache/whirr/service/zookeeper/integration/ZooKeeperServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/zookeeper/src/test/java/org/apache/whirr/service/zookeeper/integration/ZooKeeperServiceTest.java?rev=987653&r1=987652&r2=987653&view=diff
==============================================================================
--- incubator/whirr/trunk/services/zookeeper/src/test/java/org/apache/whirr/service/zookeeper/integration/ZooKeeperServiceTest.java (original)
+++ incubator/whirr/trunk/services/zookeeper/src/test/java/org/apache/whirr/service/zookeeper/integration/ZooKeeperServiceTest.java Fri Aug 20 22:22:46 2010
@@ -18,14 +18,20 @@
 
 package org.apache.whirr.service.zookeeper.integration;
 
-import static com.google.common.base.Preconditions.checkNotNull;
 import static junit.framework.Assert.assertEquals;
+import static org.hamcrest.Matchers.instanceOf;
+import static org.junit.Assert.assertThat;
 
 import java.io.IOException;
 import java.util.concurrent.CountDownLatch;
 
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
 import org.apache.whirr.service.ClusterSpec;
-import org.apache.whirr.service.ClusterSpec.InstanceTemplate;
+import org.apache.whirr.service.Service;
+import org.apache.whirr.service.ServiceFactory;
 import org.apache.whirr.service.zookeeper.ZooKeeperCluster;
 import org.apache.whirr.service.zookeeper.ZooKeeperService;
 import org.apache.zookeeper.CreateMode;
@@ -40,33 +46,21 @@ import org.junit.Test;
 
 public class ZooKeeperServiceTest {
   
-  private String clusterName = "zkclustertest";
-  
   private ClusterSpec clusterSpec;
   private ZooKeeperService service;
   private ZooKeeperCluster cluster;
   
   @Before
-  public void setUp() throws IOException {
-    String secretKeyFile;
-    try {
-       secretKeyFile = checkNotNull(System.getProperty("whirr.test.ssh.keyfile"));
-    } catch (NullPointerException e) {
-       secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
-    }
-    clusterSpec = new ClusterSpec(
-        new InstanceTemplate(2, ZooKeeperService.ZOOKEEPER_ROLE));
-    clusterSpec.setProvider(checkNotNull(System.getProperty("whirr.test.provider", "ec2")));
-    clusterSpec.setIdentity(checkNotNull(System.getProperty("whirr.test.identity")));
-    clusterSpec.setCredential(checkNotNull(System.getProperty("whirr.test.credential")));
-    String cidrs = System.getProperty("whirr.test.client-cidrs");
-    if (cidrs != null) {
-      clusterSpec.setClientCidrs(cidrs.split(","));
+  public void setUp() throws ConfigurationException, IOException {
+    CompositeConfiguration config = new CompositeConfiguration();
+    if (System.getProperty("config") != null) {
+      config.addConfiguration(new PropertiesConfiguration(System.getProperty("config")));
     }
-    clusterSpec.setSecretKeyFile(secretKeyFile);
-    clusterSpec.setClusterName(clusterName);
-    service = new ZooKeeperService();
-    
+    config.addConfiguration(new PropertiesConfiguration("whirr-zookeeper-test.properties"));
+    clusterSpec = ClusterSpec.fromConfiguration(config);
+    Service s = new ServiceFactory().create(clusterSpec.getServiceName());
+    assertThat(s, instanceOf(ZooKeeperService.class));
+    service = (ZooKeeperService) s;
     cluster = service.launchCluster(clusterSpec);
     System.out.println(cluster.getHosts());
   }

Added: incubator/whirr/trunk/services/zookeeper/src/test/resources/whirr-zookeeper-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/zookeeper/src/test/resources/whirr-zookeeper-test.properties?rev=987653&view=auto
==============================================================================
--- incubator/whirr/trunk/services/zookeeper/src/test/resources/whirr-zookeeper-test.properties (added)
+++ incubator/whirr/trunk/services/zookeeper/src/test/resources/whirr-zookeeper-test.properties Fri Aug 20 22:22:46 2010
@@ -0,0 +1,24 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+whirr.service-name=zookeeper
+whirr.cluster-name=zkclustertest
+whirr.instance-templates=2 zk
+whirr.provider=ec2
+whirr.identity=${sys:whirr.test.identity}
+whirr.credential=${sys:whirr.test.credential}
+whirr.secret-key-file=${sys:user.home}/.ssh/id_rsa

Propchange: incubator/whirr/trunk/services/zookeeper/src/test/resources/whirr-zookeeper-test.properties
------------------------------------------------------------------------------
    svn:eol-style = native