You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@whirr.apache.org by as...@apache.org on 2011/07/25 22:26:54 UTC

svn commit: r1150885 - in /incubator/whirr/trunk: ./ core/src/main/java/org/apache/whirr/service/ recipes/ services/cdh/src/main/resources/functions/ services/cdh/src/test/java/org/apache/whirr/service/cdh/integration/ services/cdh/src/test/resources/ ...

Author: asavu
Date: Mon Jul 25 20:26:50 2011
New Revision: 1150885

URL: http://svn.apache.org/viewvc?rev=1150885&view=rev
Log:
WHIRR-334. Support for CDH3u0 HBase (Geoff Black, Andrei Savu, Bruno Dumon)

Added:
    incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties
Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterActionHandlerSupport.java
    incubator/whirr/trunk/recipes/hadoop-ec2.properties
    incubator/whirr/trunk/recipes/hadoop-rackspace.properties
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hadoop.sh
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hbase.sh
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_zookeeper.sh
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hadoop.sh
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hbase.sh
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_zookeeper.sh
    incubator/whirr/trunk/services/cdh/src/test/java/org/apache/whirr/service/cdh/integration/CdhZooKeeperServiceTest.java
    incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hadoop-test.properties
    incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hbase-test.properties
    incubator/whirr/trunk/services/cdh/src/test/resources/whirr-zookeeper-test.properties
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopClusterActionHandler.java
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java
    incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConstants.java
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java
    incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties
    incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperClusterActionHandler.java
    incubator/whirr/trunk/services/zookeeper/src/main/resources/functions/install_zookeeper.sh
    incubator/whirr/trunk/src/site/xdoc/faq.xml

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Mon Jul 25 20:26:50 2011
@@ -47,6 +47,8 @@ Trunk (unreleased changes)
     WHIRR-330. BYON doesn't work with HadoopConfigurationBuilder
     (John Amos and asavu)
 
+    WHIRR-334. Support for CDH3u0 HBase (Geoff Black, Andrei Savu, Bruno Dumon)
+
 Release 0.5.0 - 2011-05-16
 
   INCOMPATIBLE CHANGES

Modified: incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterActionHandlerSupport.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterActionHandlerSupport.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterActionHandlerSupport.java (original)
+++ incubator/whirr/trunk/core/src/main/java/org/apache/whirr/service/ClusterActionHandlerSupport.java Mon Jul 25 20:26:50 2011
@@ -31,6 +31,8 @@ import org.apache.whirr.ClusterSpec;
 import org.apache.whirr.service.jclouds.RunUrlStatement;
 import org.apache.whirr.util.BlobCache;
 import org.jclouds.scriptbuilder.domain.Statement;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * This is a utility class to make it easier to implement
@@ -39,6 +41,9 @@ import org.jclouds.scriptbuilder.domain.
  */
 public abstract class ClusterActionHandlerSupport extends ClusterActionHandler {
 
+  private static final Logger LOG =
+    LoggerFactory.getLogger(ClusterActionHandler.class);
+
   @Override
   public void beforeAction(ClusterActionEvent event)
       throws IOException, InterruptedException{
@@ -159,4 +164,52 @@ public abstract class ClusterActionHandl
     return rawUrl;
   }
 
+  /**
+   * Get service start function name from the configuration
+   */
+  public String getStartFunction(Configuration config, String service, String defaultFunction) {
+    return getFunctionName(config, service, "start", defaultFunction);
+  }
+
+  /**
+   * Get service start function name from the configuration
+   */
+  public String getStopFunction(Configuration config, String service, String defaultFunction) {
+    return getFunctionName(config, service, "stop", defaultFunction);
+  }
+
+  /**
+   * Get service install function name from the configuration
+   */
+  public String getInstallFunction(Configuration config, String service, String defaultFunction) {
+    return getFunctionName(config, service, "install", defaultFunction);
+  }
+
+  /**
+   * Get service configure function name from the configuration
+   */
+  public String getConfigureFunction(Configuration config, String service, String defaultFunction) {
+    return getFunctionName(config, service, "configure", defaultFunction);
+  }
+
+  /**
+   * Get service cleanup function name from the configuration
+   */
+  public String getCleanupFunction(Configuration config, String service, String defaultFunction) {
+    return getFunctionName(config, service, "cleanup", defaultFunction);
+  }
+
+  public String getFunctionName(Configuration config, String service, String functionName, String defaultFunction) {
+
+    String deprecatedKey = String.format("whirr.%s-%s-function", service, functionName);
+    String key = String.format("whirr.%s.%s-function", service, functionName);
+
+    if (config.containsKey(deprecatedKey)) {
+      LOG.warn("'{}' is deprecated. Replace with '{}'", deprecatedKey, key);
+      return config.getString(deprecatedKey);
+    }
+
+    return config.getString(key, defaultFunction);
+  }
+
 }

Modified: incubator/whirr/trunk/recipes/hadoop-ec2.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/hadoop-ec2.properties?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/recipes/hadoop-ec2.properties (original)
+++ incubator/whirr/trunk/recipes/hadoop-ec2.properties Mon Jul 25 20:26:50 2011
@@ -29,8 +29,8 @@ whirr.cluster-name=hadoop
 whirr.instance-templates=1 hadoop-namenode+hadoop-jobtracker,5 hadoop-datanode+hadoop-tasktracker
 
 # Uncomment out these lines to run CDH
-#whirr.hadoop-install-function=install_cdh_hadoop
-#whirr.hadoop-configure-function=configure_cdh_hadoop
+#whirr.hadoop.install-function=install_cdh_hadoop
+#whirr.hadoop.configure-function=configure_cdh_hadoop
 
 # For EC2 set AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY environment variables.
 whirr.provider=aws-ec2

Modified: incubator/whirr/trunk/recipes/hadoop-rackspace.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/hadoop-rackspace.properties?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/recipes/hadoop-rackspace.properties (original)
+++ incubator/whirr/trunk/recipes/hadoop-rackspace.properties Mon Jul 25 20:26:50 2011
@@ -29,8 +29,8 @@ whirr.cluster-name=hadoop
 whirr.instance-templates=1 hadoop-namenode+hadoop-jobtracker,5 hadoop-datanode+hadoop-tasktracker
 
 # Uncomment out these lines to run CDH
-#whirr.hadoop-install-function=install_cdh_hadoop
-#whirr.hadoop-configure-function=configure_cdh_hadoop
+#whirr.hadoop.install-function=install_cdh_hadoop
+#whirr.hadoop.configure-function=configure_cdh_hadoop
 
 # For Rackspace set RACKSPACE_USERNAME and RACKSPACE_API_KEY environment variables.
 whirr.provider=cloudservers-us

Added: incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties?rev=1150885&view=auto
==============================================================================
--- incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties (added)
+++ incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties Mon Jul 25 20:26:50 2011
@@ -0,0 +1,54 @@
+#
+# 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.
+#
+
+#
+# HBase CDH install
+#
+
+whirr.cluster-name=hbase-cdh
+
+whirr.instance-templates=1 zookeeper+hadoop-namenode+hadoop-jobtracker+hbase-master,1 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
+
+#custom CDH zookeeper install and configure functions
+whirr.zookeeper.install-function=install_cdh_zookeeper
+whirr.zookeeper.configure-function=configure_cdh_zookeeper
+
+#custom CDH hadoop install and configure functions
+whirr.hadoop.install-function=install_cdh_hadoop
+whirr.hadoop.configure-function=configure_cdh_hadoop
+
+#custom CDH hbase install and configure functions
+whirr.hbase.install-function=install_cdh_hbase
+whirr.hbase.configure-function=configure_cdh_hbase
+
+whirr.provider=aws-ec2
+
+whirr.identity=${env:AWS_ACCESS_KEY_ID}
+whirr.credential=${env:AWS_SECRET_ACCESS_KEY}
+
+# The size of the instance to use. See http://aws.amazon.com/ec2/instance-types/
+whirr.hardware-id=c1.xlarge
+
+# Ubuntu 10.04 LTS Lucid. See http://cloud.ubuntu.com/ami/
+whirr.image-id=us-east-1/ami-04c9306d
+
+# If you choose a different location, make sure whirr.image-id is updated too
+whirr.location-id=us-east-1
+
+# By default use the user system SSH keys. Override them here.
+# whirr.private-key-file=${sys:user.home}/.ssh/id_rsa
+# whirr.public-key-file=${whirr.private-key-file}.pub

Modified: incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hadoop.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hadoop.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hadoop.sh (original)
+++ incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hadoop.sh Mon Jul 25 20:26:50 2011
@@ -66,7 +66,7 @@ EOF
     $HADOOP_CONF_DIR/hadoop-env.sh
     
   # Disable IPv6
-  sed -i -e 's|# export HADOOP_OPTS=.*|export HADOOP_OPTS="-Djava.net.preferIPv4Stack=true"|' \
+  sed -i -e 's|# Extra Java runtime options.  Empty by default.|# Extra Java runtime options.  Empty by default.\nexport HADOOP_OPTS="$HADOOP_OPTS -Djava.net.preferIPv4Stack=true"|' \
     $HADOOP_CONF_DIR/hadoop-env.sh
 
   # Hadoop logs should be on the /data partition

Modified: incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hbase.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hbase.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hbase.sh (original)
+++ incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hbase.sh Mon Jul 25 20:26:50 2011
@@ -116,6 +116,10 @@ function configure_cdh_hbase() {
  <name>hbase.client.retries.number</name>
  <value>100</value>
 </property>
+<property>
+ <name>hbase.zookeeper.recoverable.waittime</name>
+ <value>600000</value>
+</property>
 </configuration>
 EOF
 
@@ -147,7 +151,7 @@ EOF
     $HBASE_CONF_DIR/hbase-env.sh
 
   # disable IPv6
-  sed -i -e 's|# export HBASE_OPTS=.*|export HBASE_OPTS="-Djava.net.preferIPv4Stack=true"|' \
+  sed -i -e 's|export HBASE_OPTS="$HBASE_OPTS -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode"|export HBASE_OPTS="$HBASE_OPTS -ea -XX:+UseConcMarkSweepGC -XX:+CMSIncrementalMode -Djava.net.preferIPv4Stack=true"|' \
     $HBASE_CONF_DIR/hbase-env.sh
 
   # hbase logs should be on the /data partition
@@ -158,14 +162,15 @@ EOF
   chown hbase:hbase /data/hbase/logs
   ln -s /data/hbase/logs /var/log/hbase
   chown -R hbase:hbase /var/log/hbase
-  
+
+  # Now that the configuration is done, install the daemon packages
   for role in $(echo "$ROLES" | tr "," "\n"); do
     case $role in
     hbase-master)
-      service hadoop-hbase-master start 
+      install_hbase_daemon hadoop-hbase-master
       ;;
     hbase-regionserver)
-      service hadoop-hbase-regionserver start
+      install_hbase_daemon hadoop-hbase-regionserver
       ;;
     hbase-restserver)
       # not supported
@@ -174,10 +179,41 @@ EOF
       # not supported
       ;;
     hbase-thriftserver)
-      service hadoop-hbase-thrift start
+      install_hbase_daemon hadoop-hbase-thrift
       ;;
     esac
   done
-}
 
+  # Start services
+  # For DEB, the services have already been started as part of the daemon package installation
+  if which rpm &> /dev/null; then
+    for role in $(echo "$ROLES" | tr "," "\n"); do
+      case $role in
+      hbase-master)
+        service hadoop-hbase-master restart
+        ;;
+      hbase-regionserver)
+        service hadoop-hbase-regionserver restart
+        ;;
+      hbase-restserver)
+        # not supported
+        ;;
+      hbase-avroserver)
+        # not supported
+        ;;
+      hbase-thriftserver)
+        service hadoop-hbase-thrift restart
+        ;;
+      esac
+    done
+  fi
+}
 
+function install_hbase_daemon() {
+  daemon=$1
+  if which dpkg &> /dev/null; then
+    apt-get -y install $daemon
+  elif which rpm &> /dev/null; then
+    yum install -y $daemon
+  fi
+}

Modified: incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_zookeeper.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_zookeeper.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_zookeeper.sh (original)
+++ incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_zookeeper.sh Mon Jul 25 20:26:50 2011
@@ -75,7 +75,16 @@ EOF
     echo $myid > $myid_file
   fi
   
-  # Now that it's configured, start ZooKeeper
-  service hadoop-zookeeper start
+  # Now that it's configured, install daemon package
+  if which dpkg &> /dev/null; then
+    apt-get -y install hadoop-zookeeper-server
+  elif which rpm &> /dev/null; then
+    yum install -y hadoop-zookeeper-server
+  fi
 
+  # Start ZooKeeper
+  # For DEB, the service is already started as part of the daemon package installation
+  if which rpm &> /dev/null; then
+    service hadoop-zookeeper-server start
+  fi
 }

Modified: incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hadoop.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hadoop.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hadoop.sh (original)
+++ incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hadoop.sh Mon Jul 25 20:26:50 2011
@@ -15,7 +15,7 @@
 # limitations under the License.
 #
 set -x
-function update_repo() {
+function register_cloudera_repo() {
   if which dpkg &> /dev/null; then
     cat > /etc/apt/sources.list.d/cloudera.list <<EOF
 deb http://archive.cloudera.com/debian lucid-$REPO contrib
@@ -54,7 +54,7 @@ function install_cdh_hadoop() {
   HADOOP=hadoop-${HADOOP_VERSION:-0.20}
   HADOOP_CONF_DIR=/etc/$HADOOP/conf.dist
 
-  update_repo
+  register_cloudera_repo
   
   if which dpkg &> /dev/null; then
     apt-get update

Modified: incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hbase.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hbase.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hbase.sh (original)
+++ incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_hbase.sh Mon Jul 25 20:26:50 2011
@@ -14,7 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-function update_repo() {
+function register_cloudera_repo() {
   if which dpkg &> /dev/null; then
     cat > /etc/apt/sources.list.d/cloudera.list <<EOF
 deb http://archive.cloudera.com/debian lucid-$REPO contrib
@@ -78,13 +78,13 @@ function install_cdh_hbase() {
   # if there is no hosts file then provide a minimal one
   [ ! -f /etc/hosts ] && echo "127.0.0.1 localhost" > /etc/hosts
 
-  update_repo
+  register_cloudera_repo
   
   if which dpkg &> /dev/null; then
     apt-get update
-    apt-get -y install hadoop-hbase hadoop-hbase-master hadoop-hbase-regionserver hadoop-hbase-thrift
+    apt-get -y install hadoop-hbase
   elif which rpm &> /dev/null; then
-    yum install -y hadoop-hbase hadoop-hbase-master hadoop-hbase-regionserver hadoop-hbase-thrift
+    yum install -y hadoop-hbase
   fi
   
   echo "export HBASE_HOME=$HBASE_HOME" >> ~root/.bashrc

Modified: incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_zookeeper.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_zookeeper.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_zookeeper.sh (original)
+++ incubator/whirr/trunk/services/cdh/src/main/resources/functions/install_cdh_zookeeper.sh Mon Jul 25 20:26:50 2011
@@ -14,7 +14,7 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-function update_repo() {
+function register_cloudera_repo() {
   if which dpkg &> /dev/null; then
     cat > /etc/apt/sources.list.d/cloudera.list <<EOF
 deb http://archive.cloudera.com/debian lucid-$REPO contrib
@@ -41,7 +41,7 @@ function install_cdh_zookeeper() {
   local OPTARG
   
   CLOUD_PROVIDER=
-  while getopts "c:" OPTION; do
+  while getopts "c:u:" OPTION; do
     case $OPTION in
     c)
       CLOUD_PROVIDER="$OPTARG"
@@ -64,13 +64,13 @@ function install_cdh_zookeeper() {
   ZK_LOG_DIR=/var/log/zookeeper
   ZK_DATA_DIR=$ZK_LOG_DIR/txlog
   
-  update_repo
+  register_cloudera_repo
   
   if which dpkg &> /dev/null; then
     apt-get update
-    apt-get -y install hadoop-zookeeper-server
+    apt-get -y install hadoop-zookeeper
   elif which rpm &> /dev/null; then
-    yum install -y hadoop-zookeeper-server
+    yum install -y hadoop-zookeeper
   fi
   
   echo "export ZOOKEEPER_HOME=$ZOOKEEPER_HOME" >> /etc/profile

Modified: incubator/whirr/trunk/services/cdh/src/test/java/org/apache/whirr/service/cdh/integration/CdhZooKeeperServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/test/java/org/apache/whirr/service/cdh/integration/CdhZooKeeperServiceTest.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/test/java/org/apache/whirr/service/cdh/integration/CdhZooKeeperServiceTest.java (original)
+++ incubator/whirr/trunk/services/cdh/src/test/java/org/apache/whirr/service/cdh/integration/CdhZooKeeperServiceTest.java Mon Jul 25 20:26:50 2011
@@ -59,7 +59,7 @@ public class CdhZooKeeperServiceTest {
     cluster = controller.launchCluster(clusterSpec);
     hosts = ZooKeeperCluster.getHosts(cluster);
   }
-  
+
   @Test
   public void test() throws Exception {
     class ConnectionWatcher implements Watcher {

Modified: incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hadoop-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hadoop-test.properties?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hadoop-test.properties (original)
+++ incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hadoop-test.properties Mon Jul 25 20:26:50 2011
@@ -15,10 +15,14 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-whirr.cluster-name=cdhclustertest
+whirr.cluster-name=hadoopcdhclustertest
+
 whirr.instance-templates=1 hadoop-jobtracker+hadoop-namenode,1 hadoop-datanode+hadoop-tasktracker
+
 whirr.provider=${sys:whirr.test.provider}
 whirr.identity=${sys:whirr.test.identity}
 whirr.credential=${sys:whirr.test.credential}
-whirr.hadoop-install-function=install_cdh_hadoop
-whirr.hadoop-configure-function=configure_cdh_hadoop
+
+whirr.hadoop.install-function=install_cdh_hadoop
+whirr.hadoop.configure-function=configure_cdh_hadoop
+

Modified: incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hbase-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hbase-test.properties?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hbase-test.properties (original)
+++ incubator/whirr/trunk/services/cdh/src/test/resources/whirr-hbase-test.properties Mon Jul 25 20:26:50 2011
@@ -15,10 +15,20 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-whirr.cluster-name=hbaseclustertest
-whirr.instance-templates=1 zookeeper,1 hadoop-namenode+hadoop-jobtracker+hbase-master,1 hbase-thriftserver,1 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
+whirr.cluster-name=hbasecdhclustertest
+
+whirr.instance-templates=1 hadoop-namenode+hadoop-jobtracker+zookeeper+hbase-master+hbase-thriftserver,1 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
+
 whirr.provider=${sys:whirr.test.provider}
 whirr.identity=${sys:whirr.test.identity}
 whirr.credential=${sys:whirr.test.credential}
-whirr.hbase-install-function=install_cdh_hbase
-whirr.hbase-configure-function=configure_cdh_hbase
+
+whirr.hadoop.install-function=install_cdh_hadoop
+whirr.hadoop.configure-function=configure_cdh_hadoop
+
+whirr.hbase.install-function=install_cdh_hbase
+whirr.hbase.configure-function=configure_cdh_hbase
+
+whirr.zookeeper.install-function=install_cdh_zookeeper
+whirr.zookeeper.configure-function=configure_cdh_zookeeper
+

Modified: incubator/whirr/trunk/services/cdh/src/test/resources/whirr-zookeeper-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/cdh/src/test/resources/whirr-zookeeper-test.properties?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/cdh/src/test/resources/whirr-zookeeper-test.properties (original)
+++ incubator/whirr/trunk/services/cdh/src/test/resources/whirr-zookeeper-test.properties Mon Jul 25 20:26:50 2011
@@ -15,10 +15,15 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 #
-whirr.cluster-name=zkclustertest
+whirr.cluster-name=zkcdhclustertest
 whirr.instance-templates=2 zookeeper
+
 whirr.provider=${sys:whirr.test.provider}
 whirr.identity=${sys:whirr.test.identity}
 whirr.credential=${sys:whirr.test.credential}
-whirr.zookeeper-install-function=install_cdh_zookeeper
-whirr.zookeeper-configure-function=configure_cdh_zookeeper
+
+whirr.zookeeper.install-function=install_cdh_zookeeper
+whirr.zookeeper.configure-function=configure_cdh_zookeeper
+
+whirr.hardware-min-ram=512
+

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -38,6 +38,14 @@ public abstract class HadoopClusterActio
       ClusterSpec clusterSpec) throws IOException {
     return getConfiguration(clusterSpec, "whirr-hadoop-default.properties");
   }
+
+  protected String getInstallFunction(Configuration config) {
+    return getInstallFunction(config, "hadoop", "install_hadoop");
+  }
+
+  protected String getConfigureFunction(Configuration config) {
+    return getConfigureFunction(config, "hadoop", "configure_hadoop");
+  }
   
   @Override
   protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
@@ -49,13 +57,10 @@ public abstract class HadoopClusterActio
     addStatement(event, call("install_java"));
     addStatement(event, call("install_tarball"));
 
-    String hadoopInstallFunction = conf.getString(
-        "whirr.hadoop-install-function", "install_hadoop");
-
     String tarball = prepareRemoteFileUrl(event,
         conf.getString("whirr.hadoop.tarball.url"));
 
-    addStatement(event, call(hadoopInstallFunction,
+    addStatement(event, call(getInstallFunction(conf),
         "-c", clusterSpec.getProvider(),
         "-u", tarball));
   }

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopDataNodeClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -55,12 +55,11 @@ public class HadoopDataNodeClusterAction
       throw new IOException(e);
     }
 
-    String hadoopConfigureFunction = clusterSpec.getConfiguration().getString(
-        "whirr.hadoop-configure-function", "configure_hadoop");
-    addStatement(event, call(hadoopConfigureFunction,
-        "hadoop-datanode,hadoop-tasktracker",
-        "-c", clusterSpec.getProvider()
-    ));
+    addStatement(event, call(
+      getConfigureFunction(getConfiguration(clusterSpec)),
+      "hadoop-datanode,hadoop-tasktracker",
+      "-c", clusterSpec.getProvider())
+    );
   }
   
 }

Modified: incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hadoop/src/main/java/org/apache/whirr/service/hadoop/HadoopNameNodeClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -86,12 +86,11 @@ public class HadoopNameNodeClusterAction
       throw new IOException(e);
     }
     
-    String hadoopConfigureFunction = clusterSpec.getConfiguration().getString(
-        "whirr.hadoop-configure-function", "configure_hadoop");
-    addStatement(event, call(hadoopConfigureFunction,
-        "hadoop-namenode,hadoop-jobtracker",
-        "-c", clusterSpec.getProvider()
-    ));
+    addStatement(event, call(
+      getConfigureFunction(getConfiguration(clusterSpec)),
+      "hadoop-namenode,hadoop-jobtracker",
+      "-c", clusterSpec.getProvider())
+    );
   }
   
   @Override

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -56,18 +56,18 @@ public class BasicServerClusterActionHan
 
     addStatement(event, call("configure_hostnames",
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider()));
+
     addStatement(event, call("install_java"));
     addStatement(event, call("install_tarball"));
 
-    String hbaseInstallFunction = getConfiguration(clusterSpec).getString(
-      HBaseConstants.KEY_INSTALL_FUNCTION, HBaseConstants.FUNCTION_INSTALL);
-
     String tarurl = prepareRemoteFileUrl(event,
       getConfiguration(clusterSpec).getString(HBaseConstants.KEY_TARBALL_URL));
 
-    addStatement(event, call(hbaseInstallFunction,
+    addStatement(event, call(
+      getInstallFunction(getConfiguration(clusterSpec)),
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider(),
-      HBaseConstants.PARAM_TARBALL_URL, tarurl));
+      HBaseConstants.PARAM_TARBALL_URL, tarurl)
+    );
   }
 
   @Override
@@ -87,22 +87,21 @@ public class BasicServerClusterActionHan
       Rule.create().destination(instance).port(port)
     );
 
-    String hbaseConfigureFunction = getConfiguration(clusterSpec).getString(
-      HBaseConstants.KEY_CONFIGURE_FUNCTION,
-      HBaseConstants.FUNCTION_POST_CONFIGURE);
-
     String master = masterPublicAddress.getHostName();
     String quorum = ZooKeeperCluster.getHosts(cluster);
 
     String tarurl = prepareRemoteFileUrl(event,
       getConfiguration(clusterSpec).getString(HBaseConstants.KEY_TARBALL_URL));
 
-    addStatement(event, call(hbaseConfigureFunction, role,
+    addStatement(event, call(
+      getConfigureFunction(getConfiguration(clusterSpec)),
+      role,
       HBaseConstants.PARAM_MASTER, master,
       HBaseConstants.PARAM_QUORUM, quorum,
       HBaseConstants.PARAM_PORT, Integer.toString(port),
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider(),
-      HBaseConstants.PARAM_TARBALL_URL, tarurl));
+      HBaseConstants.PARAM_TARBALL_URL, tarurl)
+    );
   }
 
 }

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -40,4 +40,13 @@ public abstract class HBaseClusterAction
     return getConfiguration(clusterSpec,
       HBaseConstants.FILE_HBASE_DEFAULT_PROPERTIES);
   }
+
+  protected String getInstallFunction(Configuration config) {
+    return getInstallFunction(config, "hbase", HBaseConstants.FUNCTION_INSTALL);
+  }
+
+  protected String getConfigureFunction(Configuration config) {
+    return getConfigureFunction(config, "hbase", HBaseConstants.FUNCTION_CONFIGURE);
+  }
+
 }

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConstants.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConstants.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConstants.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConstants.java Mon Jul 25 20:26:50 2011
@@ -20,12 +20,10 @@ package org.apache.whirr.service.hbase;
 
 public final class HBaseConstants {
 
-  public static final String KEY_INSTALL_FUNCTION = "whirr.hbase.install-function";
-  public static final String KEY_CONFIGURE_FUNCTION = "whirr.hbase.configure-function";
   public static final String KEY_TARBALL_URL = "whirr.hbase.tarball.url";
 
   public static final String FUNCTION_INSTALL = "install_hbase";
-  public static final String FUNCTION_POST_CONFIGURE = "configure_hbase";
+  public static final String FUNCTION_CONFIGURE = "configure_hbase";
 
   public static final String PARAM_PROVIDER = "-c";
   public static final String PARAM_MASTER = "-m";

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -62,18 +62,18 @@ public class HBaseMasterClusterActionHan
 
     addStatement(event, call("configure_hostnames",
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider()));
+
     addStatement(event, call("install_java"));
     addStatement(event, call("install_tarball"));
 
-    String hbaseInstallFunction = getConfiguration(clusterSpec).getString(
-      HBaseConstants.KEY_INSTALL_FUNCTION, HBaseConstants.FUNCTION_INSTALL);
-
     String tarurl = prepareRemoteFileUrl(event,
       getConfiguration(clusterSpec).getString(HBaseConstants.KEY_TARBALL_URL));
 
-    addStatement(event, call(hbaseInstallFunction,
+    addStatement(event, call(
+      getInstallFunction(getConfiguration(clusterSpec)),
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider(),
-      HBaseConstants.PARAM_TARBALL_URL, tarurl));
+      HBaseConstants.PARAM_TARBALL_URL, tarurl)
+    );
   }
 
   @Override
@@ -91,20 +91,20 @@ public class HBaseMasterClusterActionHan
         .ports(MASTER_WEB_UI_PORT, MASTER_PORT)
     );
 
-    String hbaseConfigureFunction = getConfiguration(clusterSpec).getString(
-      HBaseConstants.KEY_CONFIGURE_FUNCTION, HBaseConstants.FUNCTION_POST_CONFIGURE);
-
     String master = masterPublicAddress.getHostName();
     String quorum = ZooKeeperCluster.getHosts(cluster);
 
     String tarurl = prepareRemoteFileUrl(event,
       getConfiguration(clusterSpec).getString(HBaseConstants.KEY_TARBALL_URL));
 
-    addStatement(event, call(hbaseConfigureFunction, ROLE,
+    addStatement(event, call(
+      getConfigureFunction(getConfiguration(clusterSpec)),
+      ROLE,
       HBaseConstants.PARAM_MASTER, master,
       HBaseConstants.PARAM_QUORUM, quorum,
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider(),
-      HBaseConstants.PARAM_TARBALL_URL, tarurl));
+      HBaseConstants.PARAM_TARBALL_URL, tarurl)
+    );
   }
 
   @Override

Modified: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -49,18 +49,18 @@ public class HBaseRegionServerClusterAct
 
     addStatement(event, call("configure_hostnames",
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider()));
+
     addStatement(event, call("install_java"));
     addStatement(event, call("install_tarball"));
 
-    String hbaseInstallFunction = getConfiguration(clusterSpec).getString(
-      HBaseConstants.KEY_INSTALL_FUNCTION, HBaseConstants.FUNCTION_INSTALL);
-
     String tarurl = prepareRemoteFileUrl(event,
       getConfiguration(clusterSpec).getString(HBaseConstants.KEY_TARBALL_URL));
 
-    addStatement(event, call(hbaseInstallFunction,
+    addStatement(event, call(
+      getInstallFunction(getConfiguration(clusterSpec)),
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider(),
-      HBaseConstants.PARAM_TARBALL_URL, tarurl));
+      HBaseConstants.PARAM_TARBALL_URL, tarurl)
+    );
   }
 
   @Override
@@ -79,21 +79,20 @@ public class HBaseRegionServerClusterAct
         .ports(REGIONSERVER_WEB_UI_PORT, REGIONSERVER_PORT)
     );
 
-    String hbaseConfigureFunction = getConfiguration(clusterSpec).getString(
-      HBaseConstants.KEY_CONFIGURE_FUNCTION,
-      HBaseConstants.FUNCTION_POST_CONFIGURE);
-
     String master = masterPublicAddress.getHostName();
     String quorum = ZooKeeperCluster.getHosts(cluster);
 
     String tarurl = prepareRemoteFileUrl(event,
       getConfiguration(clusterSpec).getString(HBaseConstants.KEY_TARBALL_URL));
 
-    addStatement(event, call(hbaseConfigureFunction, ROLE,
+    addStatement(event, call(
+      getConfigureFunction(getConfiguration(clusterSpec)),
+      ROLE,
       HBaseConstants.PARAM_MASTER, master,
       HBaseConstants.PARAM_QUORUM, quorum,
       HBaseConstants.PARAM_PROVIDER, clusterSpec.getProvider(),
-      HBaseConstants.PARAM_TARBALL_URL, tarurl));
+      HBaseConstants.PARAM_TARBALL_URL, tarurl)
+    );
   }
 
 }

Modified: incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties (original)
+++ incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties Mon Jul 25 20:26:50 2011
@@ -16,8 +16,10 @@
 # limitations under the License.
 #
 whirr.cluster-name=hbaseclustertest
-whirr.instance-templates=1 zookeeper,1 hadoop-namenode+hadoop-jobtracker+hbase-master,1 hbase-thriftserver,1 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
+whirr.instance-templates=1 hadoop-namenode+hadoop-jobtracker+hbase-master,1 zookeeper+hbase-thriftserver,1 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
+
 whirr.provider=${sys:whirr.test.provider}
 whirr.identity=${sys:whirr.test.identity}
 whirr.credential=${sys:whirr.test.credential}
+
 whirr.hbase.tarball.url=http://archive.apache.org/dist/hbase/hbase-0.89.20100924/hbase-0.89.20100924-bin.tar.gz

Modified: incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperClusterActionHandler.java?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperClusterActionHandler.java (original)
+++ incubator/whirr/trunk/services/zookeeper/src/main/java/org/apache/whirr/service/zookeeper/ZooKeeperClusterActionHandler.java Mon Jul 25 20:26:50 2011
@@ -56,42 +56,61 @@ public class ZooKeeperClusterActionHandl
     return getConfiguration(spec, "whirr-zookeeper-default.properties");
   }
 
+  protected String getInstallFunction(Configuration config) {
+    return getInstallFunction(config, getRole(), "install_" + getRole());
+  }
+
+  protected String getConfigureFunction(Configuration config) {
+    return getConfigureFunction(config, getRole(), "configure_" + getRole());
+  }
+
+  protected String getStartFunction(Configuration config) {
+    return getStartFunction(config, getRole(), "start_" + getRole());
+  }
+
   @Override
   protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
     ClusterSpec clusterSpec = event.getClusterSpec();
-    String zookeeperInstallFunction = clusterSpec.getConfiguration().getString(
-        "whirr.zookeeper-install-function", "install_zookeeper");
     Configuration config = getConfiguration(clusterSpec);
 
     addStatement(event, call("install_java"));
     addStatement(event, call("install_tarball"));
-
-    /* register utility functions for managing init scripts */
     addStatement(event, call("install_service"));
-    addStatement(event, call("remove_service"));
 
     String tarurl = config.getString("whirr.zookeeper.tarball.url");
-    addStatement(event, call(zookeeperInstallFunction,
-      prepareRemoteFileUrl(event, tarurl)));
+    addStatement(event, call(getInstallFunction(config),
+      "-c", clusterSpec.getProvider(),
+      "-u", prepareRemoteFileUrl(event, tarurl))
+    );
   }
 
   @Override
   protected void beforeConfigure(ClusterActionEvent event) throws IOException, InterruptedException {
     ClusterSpec clusterSpec = event.getClusterSpec();
     Cluster cluster = event.getCluster();
-    Set<Instance> ensemble = cluster.getInstancesMatching(role(ZOOKEEPER_ROLE));
+
     event.getFirewallManager().addRule(
         Rule.create().destination(role(ZOOKEEPER_ROLE)).port(CLIENT_PORT)
     );
     
     // Pass list of all servers in ensemble to configure script.
     // Position is significant: i-th server has id i.
+
+    Set<Instance> ensemble = cluster.getInstancesMatching(role(ZOOKEEPER_ROLE));
     String servers = Joiner.on(' ').join(getPrivateIps(ensemble));
-    String zookeeperConfigureFunction = clusterSpec.getConfiguration().getString(
-        "whirr.zookeeper-configure-function", "configure_zookeeper");
-    addStatement(event, call(zookeeperConfigureFunction, "-c",
+
+    Configuration config = getConfiguration(clusterSpec);
+    String configureFunction = getConfigureFunction(config);
+
+    addStatement(event, call(configureFunction, "-c",
         clusterSpec.getProvider(), servers));
-    addStatement(event, call("start_zookeeper"));
+
+    if (configureFunction.equals("configure_zookeeper")) { //default zookeeper.configure-function
+      addStatement(event, call(getStartFunction(config)));
+    }
+    else {
+      //don't call start_zookeeper, because the CDH config starts the CDH version of zookeeper on its own
+    }
   }
   
   @Override

Modified: incubator/whirr/trunk/services/zookeeper/src/main/resources/functions/install_zookeeper.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/zookeeper/src/main/resources/functions/install_zookeeper.sh?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/services/zookeeper/src/main/resources/functions/install_zookeeper.sh (original)
+++ incubator/whirr/trunk/services/zookeeper/src/main/resources/functions/install_zookeeper.sh Mon Jul 25 20:26:50 2011
@@ -19,11 +19,15 @@ function install_zookeeper() {
   local OPTARG
   
   CLOUD_PROVIDER=
-  while getopts "c:" OPTION; do
+  ZK_TARBALL_URL=
+  while getopts "c:u:" OPTION; do
     case $OPTION in
     c)
       CLOUD_PROVIDER="$OPTARG"
       ;;
+    u)
+      ZK_TARBALL_URL="$OPTARG"
+      ;;
     esac
   done
   
@@ -36,7 +40,6 @@ function install_zookeeper() {
       ;;
   esac
 
-  ZK_TARBALL_URL=$1
   ZOOKEEPER_HOME=/usr/local/$(basename $ZK_TARBALL_URL '.tar.gz')
 
   ZK_CONF_DIR=/etc/zookeeper/conf
@@ -50,8 +53,10 @@ function install_zookeeper() {
   
   mkdir -p /mnt/zookeeper/logs
   ln -s /mnt/zookeeper/logs $ZK_LOG_DIR
+
   mkdir -p $ZK_LOG_DIR/txlog
   mkdir -p $ZK_CONF_DIR
+
   cp $ZOOKEEPER_HOME/conf/log4j.properties $ZK_CONF_DIR
   
   sed -i -e "s|log4j.rootLogger=INFO, CONSOLE|log4j.rootLogger=INFO, ROLLINGFILE|" \

Modified: incubator/whirr/trunk/src/site/xdoc/faq.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/src/site/xdoc/faq.xml?rev=1150885&r1=1150884&r2=1150885&view=diff
==============================================================================
--- incubator/whirr/trunk/src/site/xdoc/faq.xml (original)
+++ incubator/whirr/trunk/src/site/xdoc/faq.xml Mon Jul 25 20:26:50 2011
@@ -161,8 +161,8 @@ above for details on how to override the
 <subsection name="How do I run Cloudera's Distribution for Hadoop?"></subsection>
 
 <p>You can run CDH rather than Apache Hadoop by running the Hadoop service and setting the 
-<tt>whirr.hadoop-install-function</tt> and 
-<tt>whirr.hadoop-configure-function</tt> properties. See the 
+<tt>whirr.hadoop.install-function</tt> and
+<tt>whirr.hadoop.configure-function</tt> properties. See the
 <i>recipes</i>directory in the distribution for samples.</p>
 <p>
 <a name="other-services"></a>