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/07/06 20:40:16 UTC

svn commit: r960977 - in /incubator/whirr/trunk: ./ core/src/main/java/org/apache/whirr/service/ core/src/test/java/org/apache/whirr/service/ core/src/test/resources/ core/src/test/resources/META-INF/ core/src/test/resources/META-INF/services/ services...

Author: tomwhite
Date: Tue Jul  6 18:40:15 2010
New Revision: 960977

URL: http://svn.apache.org/viewvc?rev=960977&view=rev
Log:
WHIRR-53. Adopt the standard Java SPI interface.

Added:
    incubator/whirr/trunk/core/src/test/resources/META-INF/
    incubator/whirr/trunk/core/src/test/resources/META-INF/services/
    incubator/whirr/trunk/core/src/test/resources/META-INF/services/org.apache.whirr.service.Service
    incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/
    incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/services/
    incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/services/org.apache.whirr.service.Service
    incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/
    incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/services/
    incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/services/org.apache.whirr.service.Service
    incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/
    incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/services/
    incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/services/org.apache.whirr.service.Service
Removed:
    incubator/whirr/trunk/core/src/test/resources/whirr.properties
    incubator/whirr/trunk/services/cassandra/src/main/resources/whirr.properties
    incubator/whirr/trunk/services/hadoop/src/main/resources/whirr.properties
    incubator/whirr/trunk/services/zookeeper/src/main/resources/whirr.properties
Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ServiceFactory.java
    incubator/whirr/trunk/core/src/test/java/org/apache/whirr/service/ServiceFactoryTest.java
    incubator/whirr/trunk/pom.xml
    incubator/whirr/trunk/services/cassandra/src/main/java/org/apache/whirr/service/cassandra/CassandraService.java
    incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/CassandraServiceTest.java
    incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/integration/CassandraServiceTest.java
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopService.java
    incubator/whirr/trunk/services/hadoop/src/test/java/org/apache/whirr/service/hadoop/integration/HadoopServiceTest.java
    incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperService.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=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Tue Jul  6 18:40:15 2010
@@ -35,6 +35,8 @@ Trunk (unreleased changes)
     WHIRR-51. Allow the Hadoop service to be run on Rackspace Cloud servers.
     (tomwhite)
 
+    WHIRR-53. Adopt the standard Java SPI interface. (tomwhite)
+
   BUG FIXES
 
     WHIRR-50. Cassandra POM should depend on top-level. (tomwhite)

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/Service.java Tue Jul  6 18:40:15 2010
@@ -27,37 +27,32 @@ import org.jclouds.compute.ComputeServic
  * used to start and stop clusters that provide the service.
  */
 public abstract class Service {
-  protected ServiceSpec serviceSpec;
 
   /**
-   * Subclasses must provide a single argument constructor that takes a
-   * {@link ServiceSpec} so that they can be constructed by
-   * {@link ServiceFactory}.
-   * @param serviceSpec
+   * @return the unique name of the service.
    */
-  public Service(ServiceSpec serviceSpec) {
-    this.serviceSpec = serviceSpec;
-  }
-  
+  public abstract String getName();
+
   /**
    * Start the cluster described by <code>clusterSpec</code> and block until the
    * cluster is
    * available. It is not guaranteed that the service running on the cluster
    * has started when this method returns.
+   * @param serviceSpec
    * @param clusterSpec
    * @return an object representing the running cluster
    * @throws IOException if there is a problem while starting the cluster. The
    * cluster may or may not have started.
    */
-  public abstract Cluster launchCluster(ClusterSpec clusterSpec)
+  public abstract Cluster launchCluster(ServiceSpec serviceSpec, ClusterSpec clusterSpec)
     throws IOException;
   
   /**
-   * Stop the cluster and destroy all reseouces associated with it.
+   * Stop the cluster and destroy all resources associated with it.
    * @throws IOException if there is a problem while stopping the cluster. The
    * cluster may or may not have been stopped.
    */
-  public void destroyCluster() throws IOException {
+  public void destroyCluster(ServiceSpec serviceSpec) throws IOException {
     ComputeService computeService = ComputeServiceBuilder.build(serviceSpec);
     computeService.destroyNodesWithTag(serviceSpec.getClusterName());
   }

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ServiceFactory.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ServiceFactory.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ServiceFactory.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ServiceFactory.java Tue Jul  6 18:40:15 2010
@@ -18,100 +18,30 @@
 
 package org.apache.whirr.service;
 
-import java.io.IOException;
-import java.io.InputStream;
-import java.lang.reflect.Constructor;
-import java.net.URL;
-import java.util.Collections;
-import java.util.Enumeration;
-import java.util.Map;
-import java.util.Properties;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.inject.internal.Maps;
+import java.util.ServiceLoader;
 
 /**
  * This class is used to create {@link Service} instances.
  * <p>
  * <i>Implementation note.</i> {@link Service} implementations are discovered
  * using a Service Provider
- * Interface (SPI), where each provider JAR on the classpath contains a
- * <i>whirr.properties</i>
- * file containing the following two properties:
- * <ul>
- * <li><code>whirr.service.name</code> - the name of the service, for example,
- * <code>hadoop</code></li>
- * <li><code>whirr.service.class</code> - the fully-qualified classname of the
- * implementation of {@link Service}, for example,
- * <code>org.apache.whirr.service.hadoop.HadoopService</code></li>
- * </ul>  
+ * Interface (SPI), described in {@link ServiceLoader}.
  */
 public class ServiceFactory {
 
-  private static final Logger LOG =
-    LoggerFactory.getLogger(ServiceFactory.class);
-  private Map<String, Class<Service>> serviceMap = Maps.newHashMap();
-
-  @SuppressWarnings("unchecked")
-  public ServiceFactory() throws IOException {
-    ClassLoader classLoader = getClass().getClassLoader();
-    Enumeration<URL> resources = classLoader.getResources("whirr.properties");
-
-    for (URL url : Collections.list(resources)) {
-      InputStream in = null;
-      try {
-        in = url.openStream();
-        Properties properties = new Properties();
-        properties.load(in);
-        String serviceName = properties.getProperty("whirr.service.name");
-        String serviceClassName = properties.getProperty("whirr.service.class");
-        if (serviceName == null) {
-          LOG.warn("Property whirr.service.name not set.");
-          continue;
-        }
-        if (serviceClassName == null) {
-          LOG.warn("Property whirr.service.class not set.");
-          continue;
-        }
-        Class<Service> serviceClass = (Class<Service>) Class
-            .forName(serviceClassName);
-        serviceMap.put(serviceName, serviceClass);
-      } catch (IOException e) {
-        LOG.warn("Problem reading whirr.properties.", e);
-        continue;
-      } catch (ClassNotFoundException e) {
-        LOG.warn("ServiceFactory class not found.", e);
-        continue;
-      } finally {
-        if (in != null) {
-          try {
-            in.close();
-          } catch (IOException e) {
-            LOG.warn("Problem closing stream.", e);
-            continue;
-          }
-        }
-      }
-    }
-  }
+  private ServiceLoader<Service> serviceLoader =
+    ServiceLoader.load(Service.class);
 
   /**
    * Create an instance of a {@link Service} according to the given
    * {@link ServiceSpec}.
    */
   public Service create(ServiceSpec serviceSpec) {
-    Class<Service> serviceClass = serviceMap.get(serviceSpec.getName());
-    if (serviceClass == null) {
-      return null;
-    }
-    try {
-      Constructor<Service> constructor = serviceClass
-          .getConstructor(ServiceSpec.class);
-      return constructor.newInstance(serviceSpec);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
+    for (Service service : serviceLoader) {
+      if (service.getName().equals(serviceSpec.getName())) {
+        return service;
+      }
     }
+    return null;
   }
 }

Modified: incubator/whirr/trunk/core/src/test/java/org/apache/whirr/service/ServiceFactoryTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/test/java/org/apache/whirr/service/ServiceFactoryTest.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/test/java/org/apache/whirr/service/ServiceFactoryTest.java (original)
+++ incubator/whirr/trunk/core/src/test/java/org/apache/whirr/service/ServiceFactoryTest.java Tue Jul  6 18:40:15 2010
@@ -28,18 +28,15 @@ import org.junit.Test;
 public class ServiceFactoryTest {
   
   public static class TestService extends Service {
-
-    public TestService(ServiceSpec serviceSpec) {
-      super(serviceSpec);
-      // TODO Auto-generated constructor stub
+    @Override
+    public String getName() {
+      return "test-service";
     }
-
     @Override
-    public Cluster launchCluster(ClusterSpec clusterSpec) throws IOException {
-      // TODO Auto-generated method stub
+    public Cluster launchCluster(ServiceSpec serviceSpec,
+        ClusterSpec clusterSpec) throws IOException {
       return null;
     }
-    
   }
   
   @Test

Added: incubator/whirr/trunk/core/src/test/resources/META-INF/services/org.apache.whirr.service.Service
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/test/resources/META-INF/services/org.apache.whirr.service.Service?rev=960977&view=auto
==============================================================================
--- incubator/whirr/trunk/core/src/test/resources/META-INF/services/org.apache.whirr.service.Service (added)
+++ incubator/whirr/trunk/core/src/test/resources/META-INF/services/org.apache.whirr.service.Service Tue Jul  6 18:40:15 2010
@@ -0,0 +1 @@
+org.apache.whirr.service.ServiceFactoryTest$TestService
\ No newline at end of file

Modified: incubator/whirr/trunk/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/pom.xml?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/pom.xml (original)
+++ incubator/whirr/trunk/pom.xml Tue Jul  6 18:40:15 2010
@@ -57,7 +57,6 @@
             <exclude>**/*.confluence</exclude>
             <exclude>**/.project</exclude>
             <exclude>**/META-INF/services/*</exclude>
-            <exclude>**/whirr.properties</exclude>
             <exclude>**/target/**</exclude>
           </excludes>
         </configuration>

Modified: incubator/whirr/trunk/services/cassandra/src/main/java/org/apache/whirr/service/cassandra/CassandraService.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cassandra/src/main/java/org/apache/whirr/service/cassandra/CassandraService.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cassandra/src/main/java/org/apache/whirr/service/cassandra/CassandraService.java (original)
+++ incubator/whirr/trunk/services/cassandra/src/main/java/org/apache/whirr/service/cassandra/CassandraService.java Tue Jul  6 18:40:15 2010
@@ -53,13 +53,14 @@ public class CassandraService extends Se
 
   public static final String CASSANDRA_ROLE = "cassandra";
   public static final int CLIENT_PORT = 9160;
-
-  public CassandraService(ServiceSpec serviceSpec) {
-    super(serviceSpec);
+  
+  @Override
+  public String getName() {
+    return "cassandra";
   }
 
   @Override
-  public Cluster launchCluster(ClusterSpec clusterSpec)
+  public Cluster launchCluster(ServiceSpec serviceSpec, ClusterSpec clusterSpec)
       throws IOException {
 
     ComputeService computeService = ComputeServiceBuilder.build(serviceSpec);

Added: incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/services/org.apache.whirr.service.Service
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/services/org.apache.whirr.service.Service?rev=960977&view=auto
==============================================================================
--- incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/services/org.apache.whirr.service.Service (added)
+++ incubator/whirr/trunk/services/cassandra/src/main/resources/META-INF/services/org.apache.whirr.service.Service Tue Jul  6 18:40:15 2010
@@ -0,0 +1 @@
+org.apache.whirr.service.cassandra.CassandraService
\ No newline at end of file

Modified: incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/CassandraServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/CassandraServiceTest.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/CassandraServiceTest.java (original)
+++ incubator/whirr/trunk/services/cassandra/src/test/java/org/apache/whirr/service/cassandra/CassandraServiceTest.java Tue Jul  6 18:40:15 2010
@@ -44,7 +44,7 @@ public class CassandraServiceTest {
     List<NodeMetadata> nodes = new ArrayList<NodeMetadata>();
     nodes.add(getNodeMetadata("1"));
     
-    CassandraService s = new CassandraService(null);
+    CassandraService s = new CassandraService();
     // check that the one node is returned
     List<NodeMetadata> seeds1 = s.getSeeds(nodes);
     assertEquals(1, seeds1.size());

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=960977&r1=960976&r2=960977&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 Tue Jul  6 18:40:15 2010
@@ -22,9 +22,6 @@ import static com.google.common.base.Pre
 import static org.junit.Assert.assertTrue;
 
 import java.io.IOException;
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -40,9 +37,6 @@ import org.apache.whirr.service.ServiceS
 import org.apache.whirr.service.Cluster.Instance;
 import org.apache.whirr.service.ClusterSpec.InstanceTemplate;
 import org.apache.whirr.service.cassandra.CassandraService;
-import org.jclouds.compute.domain.NodeMetadata;
-import org.jclouds.compute.domain.NodeState;
-import org.jclouds.compute.domain.internal.NodeMetadataImpl;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -52,6 +46,7 @@ public class CassandraServiceTest {
   private String clusterName = "cassandraclustertest";
   private static final String KEYSPACE = "Keyspace1";
 
+  private ServiceSpec serviceSpec;
   private CassandraService service;
   private Cluster cluster;
 
@@ -63,18 +58,18 @@ public class CassandraServiceTest {
     } catch (NullPointerException e) {
       secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
     }
-    ServiceSpec serviceSpec = new ServiceSpec();
+    serviceSpec = new ServiceSpec();
     serviceSpec.setProvider(checkNotNull(System.getProperty(
         "whirr.test.provider", "ec2")));
     serviceSpec.setAccount(checkNotNull(System.getProperty("whirr.test.user")));
     serviceSpec.setKey(checkNotNull(System.getProperty("whirr.test.key")));
     serviceSpec.setSecretKeyFile(secretKeyFile);
     serviceSpec.setClusterName(clusterName);
-    service = new CassandraService(serviceSpec);
+    service = new CassandraService();
 
     ClusterSpec clusterSpec = new ClusterSpec(new InstanceTemplate(2,
         CassandraService.CASSANDRA_ROLE));
-    cluster = service.launchCluster(clusterSpec);
+    cluster = service.launchCluster(serviceSpec, clusterSpec);
 
     // give it a sec to boot up the cluster
     waitForCassandra();
@@ -130,7 +125,7 @@ public class CassandraServiceTest {
   @After
   public void tearDown() throws IOException {
     if (service != null) {
-      service.destroyCluster();      
+      service.destroyCluster(serviceSpec);      
     }
   }
 

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopService.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopService.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopService.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopService.java Tue Jul  6 18:40:15 2010
@@ -55,12 +55,13 @@ public class HadoopService extends Servi
   public static final Set<String> MASTER_ROLE = Sets.newHashSet("nn", "jt");
   public static final Set<String> WORKER_ROLE = Sets.newHashSet("dn", "tt");
 
-  public HadoopService(ServiceSpec serviceSpec) {
-    super(serviceSpec);
+  @Override
+  public String getName() {
+    return "hadoop";
   }
-
+  
   @Override
-  public HadoopCluster launchCluster(ClusterSpec clusterSpec) throws IOException {
+  public HadoopCluster launchCluster(ServiceSpec serviceSpec, ClusterSpec clusterSpec) throws IOException {
     ComputeService computeService = ComputeServiceBuilder.build(serviceSpec);
 
     String privateKey = serviceSpec.readPrivateKey();

Added: incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/services/org.apache.whirr.service.Service
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/services/org.apache.whirr.service.Service?rev=960977&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/services/org.apache.whirr.service.Service (added)
+++ incubator/whirr/trunk/services/hadoop/src/main/resources/META-INF/services/org.apache.whirr.service.Service Tue Jul  6 18:40:15 2010
@@ -0,0 +1 @@
+org.apache.whirr.service.hadoop.HadoopService
\ No newline at end of file

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=960977&r1=960976&r2=960977&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 Tue Jul  6 18:40:15 2010
@@ -57,6 +57,7 @@ public class HadoopServiceTest {
   
   private String clusterName = "hadoopclustertest";
   
+  private ServiceSpec serviceSpec;
   private HadoopService service;
   private HadoopProxy proxy;
   private HadoopCluster cluster;
@@ -69,19 +70,19 @@ public class HadoopServiceTest {
     } catch (NullPointerException e) {
        secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
     }
-    ServiceSpec serviceSpec = new ServiceSpec();
+    serviceSpec = new ServiceSpec();
     serviceSpec.setProvider(checkNotNull(System.getProperty("whirr.test.provider", "ec2")));
     serviceSpec.setAccount(checkNotNull(System.getProperty("whirr.test.user")));
     serviceSpec.setKey(checkNotNull(System.getProperty("whirr.test.key")));
     serviceSpec.setSecretKeyFile(secretKeyFile);
     serviceSpec.setClusterName(clusterName);
-    service = new HadoopService(serviceSpec);
+    service = new HadoopService();
     
     ClusterSpec clusterSpec = new ClusterSpec(
 	new InstanceTemplate(1, HadoopService.MASTER_ROLE),
 	new InstanceTemplate(1, HadoopService.WORKER_ROLE));
 
-    cluster = service.launchCluster(clusterSpec);
+    cluster = service.launchCluster(serviceSpec, clusterSpec);
     proxy = new HadoopProxy(serviceSpec, cluster);
     proxy.start();
   }
@@ -146,7 +147,7 @@ public class HadoopServiceTest {
   @After
   public void tearDown() throws IOException {
     proxy.stop();
-    service.destroyCluster();
+    service.destroyCluster(serviceSpec);
   }
 
 }

Modified: incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperService.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperService.java?rev=960977&r1=960976&r2=960977&view=diff
==============================================================================
--- incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperService.java (original)
+++ incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperService.java Tue Jul  6 18:40:15 2010
@@ -52,12 +52,13 @@ public class ZooKeeperService extends Se
   public static final String ZOOKEEPER_ROLE = "zk";
   private static final int CLIENT_PORT = 2181;
   
-  public ZooKeeperService(ServiceSpec serviceSpec) {
-    super(serviceSpec);
+  @Override
+  public String getName() {
+    return "zookeeper";
   }
-
+  
   @Override
-  public ZooKeeperCluster launchCluster(ClusterSpec clusterSpec) throws IOException {
+  public ZooKeeperCluster launchCluster(ServiceSpec serviceSpec, ClusterSpec clusterSpec) throws IOException {
       
     ComputeService computeService = ComputeServiceBuilder.build(serviceSpec);
 

Added: incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/services/org.apache.whirr.service.Service
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/services/org.apache.whirr.service.Service?rev=960977&view=auto
==============================================================================
--- incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/services/org.apache.whirr.service.Service (added)
+++ incubator/whirr/trunk/services/zookeeper/src/main/resources/META-INF/services/org.apache.whirr.service.Service Tue Jul  6 18:40:15 2010
@@ -0,0 +1 @@
+org.apache.whirr.service.zookeeper.ZooKeeperService
\ No newline at end of file

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=960977&r1=960976&r2=960977&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 Tue Jul  6 18:40:15 2010
@@ -43,6 +43,7 @@ public class ZooKeeperServiceTest {
   
   private String clusterName = "zkclustertest";
   
+  private ServiceSpec serviceSpec;
   private ZooKeeperService service;
   private ZooKeeperCluster cluster;
   
@@ -54,17 +55,17 @@ public class ZooKeeperServiceTest {
     } catch (NullPointerException e) {
        secretKeyFile = System.getProperty("user.home") + "/.ssh/id_rsa";
     }
-    ServiceSpec serviceSpec = new ServiceSpec();
+    serviceSpec = new ServiceSpec();
     serviceSpec.setProvider(checkNotNull(System.getProperty("whirr.test.provider", "ec2")));
     serviceSpec.setAccount(checkNotNull(System.getProperty("whirr.test.user")));
     serviceSpec.setKey(checkNotNull(System.getProperty("whirr.test.key")));
     serviceSpec.setSecretKeyFile(secretKeyFile);
     serviceSpec.setClusterName(clusterName);
-    service = new ZooKeeperService(serviceSpec);
+    service = new ZooKeeperService();
     
     ClusterSpec clusterSpec = new ClusterSpec(
 	new InstanceTemplate(2, ZooKeeperService.ZOOKEEPER_ROLE));
-    cluster = service.launchCluster(clusterSpec);
+    cluster = service.launchCluster(serviceSpec, clusterSpec);
     System.out.println(cluster.getHosts());
   }
   
@@ -116,7 +117,7 @@ public class ZooKeeperServiceTest {
   
   @After
   public void tearDown() throws IOException {
-    service.destroyCluster();
+    service.destroyCluster(serviceSpec);
   }
   
 }