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 2011/01/03 05:33:32 UTC

svn commit: r1054522 - in /incubator/whirr/trunk: ./ cli/ scripts/apache/hbase/ services/hbase/ services/hbase/src/ services/hbase/src/main/ services/hbase/src/main/java/ services/hbase/src/main/java/org/ services/hbase/src/main/java/org/apache/ servic...

Author: tomwhite
Date: Mon Jan  3 04:33:30 2011
New Revision: 1054522

URL: http://svn.apache.org/viewvc?rev=1054522&view=rev
Log:
WHIRR-25. Add HBase service. Contributed by Lars George.

Added:
    incubator/whirr/trunk/scripts/apache/hbase/
    incubator/whirr/trunk/scripts/apache/hbase/install   (with props)
    incubator/whirr/trunk/scripts/apache/hbase/post-configure
    incubator/whirr/trunk/services/hbase/
    incubator/whirr/trunk/services/hbase/pom.xml   (with props)
    incubator/whirr/trunk/services/hbase/src/
    incubator/whirr/trunk/services/hbase/src/main/
    incubator/whirr/trunk/services/hbase/src/main/java/
    incubator/whirr/trunk/services/hbase/src/main/java/org/
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseAvroServerClusterActionHandler.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseCluster.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRestServerClusterActionHandler.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseTemplateBuilderStrategy.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseThriftServerClusterActionHandler.java   (with props)
    incubator/whirr/trunk/services/hbase/src/main/resources/
    incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/
    incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/services/
    incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler
    incubator/whirr/trunk/services/hbase/src/test/
    incubator/whirr/trunk/services/hbase/src/test/java/
    incubator/whirr/trunk/services/hbase/src/test/java/org/
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java   (with props)
    incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java   (with props)
    incubator/whirr/trunk/services/hbase/src/test/resources/
    incubator/whirr/trunk/services/hbase/src/test/resources/log4j.xml   (with props)
    incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties   (with props)
Modified:
    incubator/whirr/trunk/.gitignore
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/cli/pom.xml
    incubator/whirr/trunk/pom.xml

Modified: incubator/whirr/trunk/.gitignore
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/.gitignore?rev=1054522&r1=1054521&r2=1054522&view=diff
==============================================================================
--- incubator/whirr/trunk/.gitignore (original)
+++ incubator/whirr/trunk/.gitignore Mon Jan  3 04:33:30 2011
@@ -3,3 +3,5 @@
 .project
 .settings
 target
+.idea/
+*.iml

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1054522&r1=1054521&r2=1054522&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Mon Jan  3 04:33:30 2011
@@ -9,6 +9,8 @@ Trunk (unreleased changes)
     WHIRR-176. Set AWS credentials in the local site file for Hadoop S3 access.
     (Lars George via tomwhite)
 
+    WHIRR-25. Add HBase service. (Lars George via tomwhite)
+
   IMPROVEMENTS
 
     WHIRR-87. Parallelize Hadoop cluster creation. (tomwhite)

Modified: incubator/whirr/trunk/cli/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/cli/pom.xml?rev=1054522&r1=1054521&r2=1054522&view=diff
==============================================================================
--- incubator/whirr/trunk/cli/pom.xml (original)
+++ incubator/whirr/trunk/cli/pom.xml Mon Jan  3 04:33:30 2011
@@ -50,6 +50,11 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-hbase</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
       <groupId>org.jclouds</groupId>
       <artifactId>jclouds-allcompute</artifactId>
     </dependency>

Modified: incubator/whirr/trunk/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/pom.xml?rev=1054522&r1=1054521&r2=1054522&view=diff
==============================================================================
--- incubator/whirr/trunk/pom.xml (original)
+++ incubator/whirr/trunk/pom.xml Mon Jan  3 04:33:30 2011
@@ -43,6 +43,7 @@
     <module>services/cdh</module>
     <module>services/hadoop</module>
     <module>services/zookeeper</module>
+    <module>services/hbase</module>
   </modules>
 
   <properties>

Added: incubator/whirr/trunk/scripts/apache/hbase/install
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/scripts/apache/hbase/install?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/scripts/apache/hbase/install (added)
+++ incubator/whirr/trunk/scripts/apache/hbase/install Mon Jan  3 04:33:30 2011
@@ -0,0 +1,131 @@
+#!/usr/bin/env bash
+#
+# 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.
+#
+# Install Apache HBase.
+#
+
+set -x
+set -e
+
+################################################################################
+# Initialize variables
+################################################################################
+
+CLOUD_PROVIDER=
+while getopts "c:" OPTION; do
+  case $OPTION in
+  c)
+    CLOUD_PROVIDER="$OPTARG"
+    ;;
+  esac
+done
+
+HBASE_VERSION=${HBASE_VERSION:-0.89.20100924}
+HBASE_HOME=/usr/local/hbase-$HBASE_VERSION
+
+function update_repo() {
+  if which dpkg &> /dev/null; then
+    sudo apt-get update
+  elif which rpm &> /dev/null; then
+    yum update -y yum
+  fi
+}
+
+# Install a list of packages on debian or redhat as appropriate
+function install_packages() {
+  if which dpkg &> /dev/null; then
+    apt-get update
+    apt-get -y install $@
+  elif which rpm &> /dev/null; then
+    yum install -y $@
+  else
+    echo "No package manager found."
+  fi
+}
+
+function install_hbase() {
+  set +e
+  useradd hadoop
+  set -e
+
+  # up file-max
+  sysctl -w fs.file-max=65535
+  # up ulimits
+  echo "root soft nofile 65535" >> /etc/security/limits.conf
+  echo "root hard nofile 65535" >> /etc/security/limits.conf
+  ulimit -n 65535
+  # up epoll limits; ok if this fails, only valid for kernels 2.6.27+
+  set +e
+  sysctl -w fs.epoll.max_user_instances=65535 > /dev/null 2>&1
+  set -e
+  # if there is no hosts file then provide a minimal one
+  [ ! -f /etc/hosts ] &&  echo "127.0.0.1 localhost" > /etc/hosts
+
+  # Reformat sdb as xfs
+  #umount /mnt
+  #mkfs.xfs -f /dev/sdb
+  #mount -o noatime /dev/sdb /mnt
+  # Probe for additional instance volumes
+  # /dev/sdb as /mnt is always set up by base image
+  #DFS_NAME_DIR="/mnt/hadoop/dfs/name"
+  #DFS_DATA_DIR="/mnt/hadoop/dfs/data"
+  #i=2
+  #for d in c d e f g h i j k l m n o p q r s t u v w x y z; do
+  # m="/mnt${i}"
+  # mkdir -p $m
+  # mkfs.xfs -f /dev/sd${d}
+  # if [ $? -eq 0 ] ; then
+  #  mount -o noatime /dev/sd${d} $m > /dev/null 2>&1
+  #  if [ $i -lt 3 ] ; then # no more than two namedirs
+  #   DFS_NAME_DIR="${DFS_NAME_DIR},${m}/hadoop/dfs/name"
+  #  fi
+  #  DFS_DATA_DIR="${DFS_DATA_DIR},${m}/hadoop/dfs/data"
+  #  i=$(( i + 1 ))
+  # fi
+  #done
+
+  # install HBase tarball
+  hbase_tar_url=http://archive.apache.org/dist/hbase/hbase-$HBASE_VERSION/hbase-${HBASE_VERSION}-bin.tar.gz
+  hbase_tar_file=`basename $hbase_tar_url`
+  hbase_tar_md5_file=`basename $hbase_tar_url.md5`
+
+  curl="curl --retry 3 --silent --show-error --fail"
+  for i in `seq 1 3`;
+  do
+    $curl -O $hbase_tar_url
+    $curl -O $hbase_tar_url.md5
+    if md5sum -c $hbase_tar_md5_file; then
+      break;
+    else
+      rm -f $hbase_tar_file $hbase_tar_md5_file
+    fi
+  done
+
+  if [ ! -e $hbase_tar_file ]; then
+    echo "Failed to download $hbase_tar_url. Aborting."
+    exit 1
+  fi
+
+  tar zxf $hbase_tar_file -C /usr/local
+  rm -f $hbase_tar_file $hbase_tar_md5_file
+
+  echo "export HBASE_HOME=$HBASE_HOME" >> ~root/.bashrc
+  echo 'export PATH=$JAVA_HOME/bin:$HBASE_HOME/bin:$PATH' >> ~root/.bashrc
+}
+
+update_repo
+install_hbase

Propchange: incubator/whirr/trunk/scripts/apache/hbase/install
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/scripts/apache/hbase/post-configure
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/scripts/apache/hbase/post-configure?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/scripts/apache/hbase/post-configure (added)
+++ incubator/whirr/trunk/scripts/apache/hbase/post-configure Mon Jan  3 04:33:30 2011
@@ -0,0 +1,214 @@
+#!/usr/bin/env bash
+#
+# 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.
+#
+# Configure Apache HBase after the cluster has started.
+#
+# Call with the following arguments
+# -m <master>
+# -q <zookeeper quorum>
+# -p <port>
+# -c <cloud provider>
+
+set -x
+set -e
+
+################################################################################
+# Initialize variables
+################################################################################
+
+ROLES=$1
+shift
+
+MASTER_HOST=
+ZOOKEEKER_QUORUM=
+CLOUD_PROVIDER=
+PORT=
+while getopts "m:q:p:c:" OPTION; do
+  case $OPTION in
+  m)
+    MASTER_HOST="$OPTARG"
+    ;;
+  q)
+    ZOOKEEPER_QUORUM="$OPTARG"
+    ;;
+  p)
+    PORT="$OPTARG"
+    ;;
+  c)
+    CLOUD_PROVIDER="$OPTARG"
+    ;;
+  esac
+done
+
+case $CLOUD_PROVIDER in
+  ec2)
+    # Use public hostname for EC2
+    SELF_HOST=`wget -q -O - http://169.254.169.254/latest/meta-data/public-hostname`
+    ;;
+  *)
+    SELF_HOST=`/sbin/ifconfig eth0 | grep 'inet addr:' | cut -d: -f2 | awk '{ print $1}'`
+    ;;
+esac
+
+HBASE_VERSION=${HBASE_VERSION:-0.89.20100924}
+HBASE_HOME=/usr/local/hbase-$HBASE_VERSION
+HBASE_CONF_DIR=$HBASE_HOME/conf
+
+# Configure HBase by setting up disks and site file
+function configure_hbase() {
+  case $CLOUD_PROVIDER in
+  ec2)
+    MOUNT=/mnt
+    ;;
+  *)
+    MOUNT=/data
+    ;;
+  esac
+  mkdir -p $MOUNT/hbase
+  chown hadoop:hadoop $MOUNT/hbase
+  mkdir $MOUNT/tmp
+  chmod a+rwxt $MOUNT/tmp
+
+  mkdir /etc/hbase
+  ln -s $HBASE_CONF_DIR /etc/hbase/conf
+
+  ##############################################################################
+  # Modify this section to customize your HBase cluster.
+  ##############################################################################
+cat > $HBASE_HOME/conf/hbase-site.xml <<EOF
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<configuration>
+<property>
+ <name>hbase.rootdir</name>
+ <value>hdfs://$MASTER_HOST:8020/hbase</value>
+</property>
+<property>
+ <name>hbase.cluster.distributed</name>
+ <value>true</value>
+</property>
+<property>
+ <name>hbase.zookeeper.quorum</name>
+ <value>$ZOOKEEPER_QUORUM</value>
+</property>
+<property>
+ <name>hbase.regionserver.handler.count</name>
+ <value>100</value>
+</property>
+<property>
+ <name>dfs.replication</name>
+ <value>3</value>
+</property>
+<property>
+ <name>zookeeper.session.timeout</name>
+ <value>60000</value>
+</property>
+<property>
+ <name>hbase.tmp.dir</name>
+ <value>/mnt/hbase</value>
+</property>
+</configuration>
+EOF
+# Override JVM options
+cat >> $HBASE_HOME/conf/hbase-env.sh <<EOF
+export HBASE_MASTER_OPTS="-Xms1000m -Xmx1000m -Xmn256m -XX:+UseConcMarkSweepGC -XX:+AggressiveOpts -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -Xloggc:/mnt/hbase/logs/hbase-master-gc.log"
+export HBASE_REGIONSERVER_OPTS="-Xms2000m -Xmx2000m -Xmn256m -XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=88 -XX:+AggressiveOpts -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps -Xloggc:/mnt/hbase/logs/hbase-regionserver-gc.log"
+EOF
+# Configure HBase for Ganglia
+cat > $HBASE_HOME/conf/hadoop-metrics.properties <<EOF
+dfs.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+dfs.period=10
+dfs.servers=$MASTER_HOST:8649
+hbase.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+hbase.period=10
+hbase.servers=$MASTER_HOST:8649
+jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+jvm.period=10
+jvm.servers=$MASTER_HOST:8649
+EOF
+
+  # Update classpath to include HBase jars and config
+#  cat >> $HADOOP_HOME/conf/hadoop-env.sh <<EOF
+#HADOOP_CLASSPATH="$HBASE_HOME/hbase-${HBASE_VERSION}.jar:$HBASE_HOME/lib/zookeeper-3.3.1.jar:$HBASE_HOME/conf"
+#EOF
+  # Configure Hadoop for Ganglia
+#  cat > $HADOOP_HOME/conf/hadoop-metrics.properties <<EOF
+#dfs.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+#dfs.period=10
+#dfs.servers=$MASTER_HOST:8649
+#jvm.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+#jvm.period=10
+#jvm.servers=$MASTER_HOST:8649
+#mapred.class=org.apache.hadoop.metrics.ganglia.GangliaContext
+#mapred.period=10
+#mapred.servers=$MASTER_HOST:8649
+#EOF
+
+  # Keep PID files in a non-temporary directory
+  sed -i -e "s|# export HBASE_PID_DIR=.*|export HBASE_PID_DIR=/var/run/hbase|" \
+    $HBASE_CONF_DIR/hbase-env.sh
+  mkdir -p /var/run/hbase
+  chown -R hadoop:hadoop /var/run/hbase
+
+  # Set SSH options within the cluster
+  sed -i -e 's|# export HBASE_SSH_OPTS=.*|export HBASE_SSH_OPTS="-o StrictHostKeyChecking=no"|' \
+    $HBASE_CONF_DIR/hbase-env.sh
+
+  # Disable IPv6
+  sed -i -e 's|# export HBASE_OPTS=.*|export HBASE_OPTS="-Djava.net.preferIPv4Stack=true"|' \
+    $HBASE_CONF_DIR/hbase-env.sh
+
+  # HBase logs should be on the /mnt partition
+  sed -i -e 's|# export HBASE_LOG_DIR=.*|export HBASE_LOG_DIR=/var/log/hbase/logs|' \
+    $HBASE_CONF_DIR/hbase-env.sh
+  rm -rf /var/log/hbase
+  mkdir $MOUNT/hbase/logs
+  chown hadoop:hadoop $MOUNT/hbase/logs
+  ln -s $MOUNT/hbase/logs /var/log/hbase
+  chown -R hadoop:hadoop /var/log/hbase
+}
+
+function start_daemon() {
+  if which dpkg &> /dev/null; then
+    AS_HADOOP="su -s /bin/bash - hadoop -c"
+  elif which rpm &> /dev/null; then
+    AS_HADOOP="/sbin/runuser -s /bin/bash - hadoop -c"
+  fi
+  $AS_HADOOP "$HBASE_HOME/bin/hbase-daemon.sh start $1"
+}
+
+configure_hbase
+
+for role in $(echo "$ROLES" | tr "," "\n"); do
+  case $role in
+  hbase-master)
+    start_daemon master
+    ;;
+  hbase-regionserver)
+    start_daemon regionserver
+    ;;
+  hbase-restserver)
+    start_daemon rest -p $PORT
+    ;;
+  hbase-avroserver)
+    start_daemon avro -p $PORT
+    ;;
+  hbase-thriftserver)
+    start_daemon thrift -p $PORT
+    ;;
+  esac
+done

Added: incubator/whirr/trunk/services/hbase/pom.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/pom.xml?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/pom.xml (added)
+++ incubator/whirr/trunk/services/hbase/pom.xml Mon Jan  3 04:33:30 2011
@@ -0,0 +1,137 @@
+<!--
+   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.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+  xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <groupId>org.apache.whirr</groupId>
+    <artifactId>whirr</artifactId>
+    <version>0.3.0-incubating-SNAPSHOT</version>
+    <relativePath>../../pom.xml</relativePath>
+  </parent>
+  <groupId>org.apache.whirr</groupId>
+  <artifactId>whirr-hbase</artifactId>
+  <packaging>jar</packaging>
+  <version>0.3.0-incubating-SNAPSHOT</version>
+  <name>Apache Whirr HBase</name>
+  <properties>
+    <hadoop.version>0.20.2</hadoop.version>
+    <hbase.version>0.89.20100924-28</hbase.version>
+  </properties>
+  <dependencies>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-core</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-core</artifactId>
+      <version>${project.version}</version>
+      <type>test-jar</type>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-hadoop</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>${project.groupId}</groupId>
+      <artifactId>whirr-zookeeper</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.jclouds</groupId>
+      <artifactId>jclouds-allcompute</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.jclouds</groupId>
+      <artifactId>jclouds-compute</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.jclouds</groupId>
+      <artifactId>jclouds-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.jclouds</groupId>
+      <artifactId>jclouds-enterprise</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.jclouds</groupId>
+      <artifactId>jclouds-jsch</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.jclouds</groupId>
+      <artifactId>jclouds-log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.hamcrest</groupId>
+      <artifactId>hamcrest-all</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>commons-configuration</groupId>
+      <artifactId>commons-configuration</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.slf4j</groupId>
+      <artifactId>slf4j-log4j12</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>zookeeper</artifactId>
+      <version>3.3.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>dnsjava</groupId>
+      <artifactId>dnsjava</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-core</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.cloudera.hbase</groupId>
+      <artifactId>hbase</artifactId>
+      <version>${hbase.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>com.cloudera.hbase</groupId>
+      <artifactId>hbase</artifactId>
+      <version>${hbase.version}</version>
+      <classifier>tests</classifier>
+    </dependency>
+  </dependencies>
+</project>
+
+
+

Propchange: incubator/whirr/trunk/services/hbase/pom.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 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=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,91 @@
+/**
+ * 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.service.hbase;
+
+import org.apache.whirr.net.DnsUtil;
+import org.apache.whirr.service.*;
+import org.apache.whirr.service.jclouds.FirewallSettings;
+import org.apache.whirr.service.zookeeper.ZooKeeperCluster;
+import org.jclouds.compute.ComputeServiceContext;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+import static org.apache.whirr.service.RolePredicates.role;
+
+/**
+ * Provides a base class for servers like REST or Avro.
+ */
+public class BasicServerClusterActionHandler extends ClusterActionHandlerSupport {
+
+  private final String role;
+  private final int defaultPort;
+  private final String configKeyPort;
+
+  public BasicServerClusterActionHandler(String role, int port, String configKeyPort) {
+    this.role = role;
+    this.defaultPort = port;
+    this.configKeyPort = configKeyPort;
+  }
+
+  @Override
+  public String getRole() {
+    return role;
+  }
+
+  @Override
+  protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    addRunUrl(event, "util/configure-hostnames", "-c", clusterSpec.getProvider());
+    addRunUrl(event, "sun/java/install");
+    String hbaseInstallRunUrl = clusterSpec.getConfiguration().getString(
+      "whirr.hbase-install-runurl", "apache/hbase/install");
+    addRunUrl(event, hbaseInstallRunUrl, "-c", clusterSpec.getProvider());
+    event.setTemplateBuilderStrategy(new HBaseTemplateBuilderStrategy());
+  }
+
+  @Override
+  protected void beforeConfigure(ClusterActionEvent event)
+      throws IOException, InterruptedException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+    int port = defaultPort;
+    if (configKeyPort != null) {
+      port = clusterSpec.getConfiguration().getInt(configKeyPort, defaultPort);
+    }
+
+    Cluster.Instance instance = cluster.getInstanceMatching(
+      role(HBaseMasterClusterActionHandler.ROLE));
+    InetAddress masterPublicAddress = instance.getPublicAddress();
+
+    ComputeServiceContext computeServiceContext =
+      ComputeServiceContextBuilder.build(clusterSpec);
+    FirewallSettings.authorizeIngress(computeServiceContext, instance,
+      clusterSpec, port);
+
+    String hbaseConfigureRunUrl = clusterSpec.getConfiguration().getString(
+      "whirr.hbase-configure-runurl", "apache/hbase/post-configure");
+    String quorum = ZooKeeperCluster.getHosts(cluster);
+    addRunUrl(event, hbaseConfigureRunUrl, role,
+      "-m", DnsUtil.resolveAddress(masterPublicAddress.getHostAddress()),
+      "-q", quorum,
+      "-p", Integer.toString(port),
+      "-c", clusterSpec.getProvider());
+  }
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/BasicServerClusterActionHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseAvroServerClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseAvroServerClusterActionHandler.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseAvroServerClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseAvroServerClusterActionHandler.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,30 @@
+/**
+ * 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.service.hbase;
+
+public class HBaseAvroServerClusterActionHandler extends BasicServerClusterActionHandler {
+
+  public static final String ROLE = "hbase-avroserver";
+  public static final int PORT = 9090;
+  private static final String CFG_KEY_PORT = "hbase.avroserver.port";
+
+  public HBaseAvroServerClusterActionHandler() {
+    super(ROLE, PORT, CFG_KEY_PORT);
+  }
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseAvroServerClusterActionHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseCluster.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseCluster.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseCluster.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseCluster.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,32 @@
+/**
+ * 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.service.hbase;
+
+import org.apache.whirr.service.Cluster;
+import org.apache.whirr.service.RolePredicates;
+
+import java.net.InetAddress;
+
+public class HBaseCluster {
+  public static InetAddress getMasterPublicAddress(Cluster cluster) {
+    return cluster.getInstanceMatching(
+        RolePredicates.role(HBaseMasterClusterActionHandler.ROLE))
+        .getPublicAddress();
+  }
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseCluster.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 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=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,169 @@
+/**
+ * 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.service.hbase;
+
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.io.Files;
+import org.apache.whirr.net.DnsUtil;
+import org.apache.whirr.service.*;
+import org.apache.whirr.service.Cluster.Instance;
+import org.apache.whirr.service.hadoop.HadoopProxy;
+import org.apache.whirr.service.jclouds.FirewallSettings;
+import org.apache.whirr.service.zookeeper.ZooKeeperCluster;
+import org.jclouds.compute.ComputeServiceContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.util.Map.Entry;
+import java.util.Properties;
+
+import static org.apache.whirr.service.RolePredicates.role;
+
+public class HBaseMasterClusterActionHandler extends ClusterActionHandlerSupport {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(HBaseMasterClusterActionHandler.class);
+
+  public static final String ROLE = "hbase-master";
+
+  public static final int MASTER_PORT = 60000;
+  public static final int MASTER_WEB_UI_PORT = 60010;
+
+  @Override
+  public String getRole() {
+    return ROLE;
+  }
+
+  @Override
+  protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    addRunUrl(event, "util/configure-hostnames", "-c", clusterSpec.getProvider());
+    addRunUrl(event, "sun/java/install");
+    String hadoopInstallRunUrl = clusterSpec.getConfiguration().getString(
+      "whirr.hbase-install-runurl", "apache/hbase/install");
+    addRunUrl(event, hadoopInstallRunUrl, "-c", clusterSpec.getProvider());
+    event.setTemplateBuilderStrategy(new HBaseTemplateBuilderStrategy());
+  }
+
+  @Override
+  protected void beforeConfigure(ClusterActionEvent event) throws IOException, InterruptedException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    LOG.info("Authorizing firewall");
+    Instance instance = cluster.getInstanceMatching(role(ROLE));
+    InetAddress masterPublicAddress = instance.getPublicAddress();
+
+    ComputeServiceContext computeServiceContext =
+      ComputeServiceContextBuilder.build(clusterSpec);
+    FirewallSettings.authorizeIngress(computeServiceContext, instance, clusterSpec,
+        MASTER_WEB_UI_PORT);
+    FirewallSettings.authorizeIngress(computeServiceContext, instance, clusterSpec,
+      masterPublicAddress.getHostAddress(), MASTER_PORT);
+
+    String hbaseConfigureRunUrl = clusterSpec.getConfiguration().getString(
+      "whirr.hbase-configure-runurl", "apache/hbase/post-configure");
+    String quorum = ZooKeeperCluster.getHosts(cluster);
+    addRunUrl(event, hbaseConfigureRunUrl, ROLE,
+      "-m", DnsUtil.resolveAddress(masterPublicAddress.getHostAddress()),
+      "-q", quorum,
+      "-c", clusterSpec.getProvider());
+  }
+
+  @Override
+  protected void afterConfigure(ClusterActionEvent event) throws IOException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    // TODO: wait for regionservers to come up?
+
+    LOG.info("Completed configuration of {}", clusterSpec.getClusterName());
+    Instance instance = cluster.getInstanceMatching(role(ROLE));
+    InetAddress masterPublicAddress = instance.getPublicAddress();
+
+    LOG.info("Web UI available at http://{}",
+      DnsUtil.resolveAddress(masterPublicAddress.getHostAddress()));
+    String quorum = ZooKeeperCluster.getHosts(cluster);
+    Properties config = createClientSideProperties(masterPublicAddress, quorum);
+    createClientSideHadoopSiteFile(clusterSpec, config);
+    createProxyScript(clusterSpec, cluster);
+    event.setCluster(new Cluster(cluster.getInstances(), config));
+  }
+
+  private Properties createClientSideProperties(InetAddress master, String quorum) throws IOException {
+    Properties config = new Properties();
+    config.setProperty("hbase.zookeeper.quorum", quorum);
+    return config;
+  }
+
+  private void createClientSideHadoopSiteFile(ClusterSpec clusterSpec, Properties config) {
+    File configDir = getConfigDir(clusterSpec);
+    File hbaseSiteFile = new File(configDir, "hbase-site.xml");
+    try {
+      Files.write(generateHBaseConfigurationFile(config), hbaseSiteFile, Charsets.UTF_8);
+      LOG.info("Wrote HBase site file {}", hbaseSiteFile);
+    } catch (IOException e) {
+      LOG.error("Problem writing HBase site file {}", hbaseSiteFile, e);
+    }
+  }
+
+  private File getConfigDir(ClusterSpec clusterSpec) {
+    File configDir = new File(new File(System.getProperty("user.home")), ".whirr");
+    configDir = new File(configDir, clusterSpec.getClusterName());
+    configDir.mkdirs();
+    return configDir;
+  }
+
+  private CharSequence generateHBaseConfigurationFile(Properties config) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("<?xml version=\"1.0\"?>\n");
+    sb.append("<?xml-stylesheet type=\"text/xsl\" href=\"configuration.xsl\"?>\n");
+    sb.append("<configuration>\n");
+    for (Entry<Object, Object> entry : config.entrySet()) {
+      sb.append("  <property>\n");
+      sb.append("    <name>").append(entry.getKey()).append("</name>\n");
+      sb.append("    <value>").append(entry.getValue()).append("</value>\n");
+      sb.append("  </property>\n");
+    }
+    sb.append("</configuration>\n");
+    return sb;
+  }
+
+  private void createProxyScript(ClusterSpec clusterSpec, Cluster cluster) {
+    File configDir = getConfigDir(clusterSpec);
+    File hbaseProxyFile = new File(configDir, "hbase-proxy.sh");
+    try {
+      HadoopProxy proxy = new HadoopProxy(clusterSpec, cluster);
+      InetAddress master = HBaseCluster.getMasterPublicAddress(cluster);
+      String script = String.format("echo 'Running proxy to HBase cluster at %s. " +
+        "Use Ctrl-c to quit.'\n",
+        DnsUtil.resolveAddress(master.getHostAddress()))
+        + Joiner.on(" ").join(proxy.getProxyCommand());
+      Files.write(script, hbaseProxyFile, Charsets.UTF_8);
+      LOG.info("Wrote HBase proxy script {}", hbaseProxyFile);
+    } catch (IOException e) {
+      LOG.error("Problem writing HBase proxy script {}", hbaseProxyFile, e);
+    }
+  }
+
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseMasterClusterActionHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 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=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,82 @@
+/**
+ * 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.service.hbase;
+
+import org.apache.whirr.net.DnsUtil;
+import org.apache.whirr.service.*;
+import org.apache.whirr.service.Cluster.Instance;
+import org.apache.whirr.service.jclouds.FirewallSettings;
+import org.apache.whirr.service.zookeeper.ZooKeeperCluster;
+import org.jclouds.compute.ComputeServiceContext;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+import static org.apache.whirr.service.RolePredicates.role;
+
+public class HBaseRegionServerClusterActionHandler extends ClusterActionHandlerSupport {
+
+  public static final String ROLE = "hbase-regionserver";
+
+  public static final int REGIONSERVER_PORT = 60020;
+  public static final int REGIONSERVER_WEB_UI_PORT = 60030;
+
+  @Override
+  public String getRole() {
+    return ROLE;
+  }
+
+  @Override
+  protected void beforeBootstrap(ClusterActionEvent event) throws IOException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    addRunUrl(event, "util/configure-hostnames", "-c", clusterSpec.getProvider());
+    String hbaseInstallRunUrl = clusterSpec.getConfiguration().getString(
+        "whirr.hbase-install-runurl", "apache/hbase/install");
+    addRunUrl(event, "sun/java/install");
+    addRunUrl(event, hbaseInstallRunUrl, "-c", clusterSpec.getProvider());
+    event.setTemplateBuilderStrategy(new HBaseTemplateBuilderStrategy());
+  }
+
+  @Override
+  protected void beforeConfigure(ClusterActionEvent event)
+      throws IOException, InterruptedException {
+    ClusterSpec clusterSpec = event.getClusterSpec();
+    Cluster cluster = event.getCluster();
+
+    Instance instance = cluster.getInstanceMatching(
+      role(HBaseMasterClusterActionHandler.ROLE));
+    InetAddress masterPublicAddress = instance.getPublicAddress();
+
+    ComputeServiceContext computeServiceContext =
+      ComputeServiceContextBuilder.build(clusterSpec);
+    FirewallSettings.authorizeIngress(computeServiceContext, instance, clusterSpec,
+      REGIONSERVER_WEB_UI_PORT);
+    FirewallSettings.authorizeIngress(computeServiceContext, instance, clusterSpec,
+        masterPublicAddress.getHostAddress(), REGIONSERVER_PORT);
+
+    String hbaseConfigureRunUrl = clusterSpec.getConfiguration().getString(
+      "whirr.hbase-configure-runurl", "apache/hbase/post-configure");
+    String quorum = ZooKeeperCluster.getHosts(cluster);
+    addRunUrl(event, hbaseConfigureRunUrl, ROLE,
+      "-m", DnsUtil.resolveAddress(masterPublicAddress.getHostAddress()),
+      "-q", quorum,
+      "-c", clusterSpec.getProvider());
+  }
+
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRegionServerClusterActionHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRestServerClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRestServerClusterActionHandler.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRestServerClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRestServerClusterActionHandler.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,30 @@
+/**
+ * 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.service.hbase;
+
+public class HBaseRestServerClusterActionHandler extends BasicServerClusterActionHandler {
+
+  public static final String ROLE = "hbase-restserver";
+  public static final int PORT = 8080;
+  private static final String CFG_KEY_PORT = "hbase.restserver.port";
+
+  public HBaseRestServerClusterActionHandler() {
+    super(ROLE, PORT, CFG_KEY_PORT);
+  }
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseRestServerClusterActionHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseTemplateBuilderStrategy.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseTemplateBuilderStrategy.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseTemplateBuilderStrategy.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseTemplateBuilderStrategy.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,43 @@
+/**
+ * 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.service.hbase;
+
+import org.apache.whirr.service.ClusterSpec;
+import org.apache.whirr.service.jclouds.TemplateBuilderStrategy;
+import org.jclouds.aws.ec2.compute.domain.EC2Hardware;
+import org.jclouds.compute.domain.OsFamily;
+import org.jclouds.compute.domain.TemplateBuilder;
+
+public class HBaseTemplateBuilderStrategy extends TemplateBuilderStrategy {
+
+  public void configureTemplateBuilder(ClusterSpec clusterSpec,
+      TemplateBuilder templateBuilder) {
+    super.configureTemplateBuilder(clusterSpec, templateBuilder);
+
+    if ("ec2".equals(clusterSpec.getProvider())
+        && clusterSpec.getImageId() == null) {
+      templateBuilder.osFamily(OsFamily.AMZN_LINUX);
+    }
+    if ("ec2".equals(clusterSpec.getProvider())
+        && clusterSpec.getHardwareId() == null) {
+      // micro is too small for Hadoop (even for testing)
+      templateBuilder.fromHardware(EC2Hardware.M1_SMALL);
+    }
+  }
+}

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseTemplateBuilderStrategy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseThriftServerClusterActionHandler.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseThriftServerClusterActionHandler.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseThriftServerClusterActionHandler.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseThriftServerClusterActionHandler.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,31 @@
+/**
+ * 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.service.hbase;
+
+public class HBaseThriftServerClusterActionHandler extends BasicServerClusterActionHandler {
+
+  public static final String ROLE = "hbase-thriftserver";
+  public static final int PORT = 9090;
+  private static final String CFG_KEY_PORT = "hbase.thriftserver.port";
+
+  public HBaseThriftServerClusterActionHandler() {
+    super(ROLE, PORT, CFG_KEY_PORT);
+  }
+}
+

Propchange: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseThriftServerClusterActionHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler (added)
+++ incubator/whirr/trunk/services/hbase/src/main/resources/META-INF/services/org.apache.whirr.service.ClusterActionHandler Mon Jan  3 04:33:30 2011
@@ -0,0 +1,5 @@
+org.apache.whirr.service.hbase.HBaseMasterClusterActionHandler
+org.apache.whirr.service.hbase.HBaseRegionServerClusterActionHandler
+org.apache.whirr.service.hbase.HBaseRestServerClusterActionHandler
+org.apache.whirr.service.hbase.HBaseAvroServerClusterActionHandler
+org.apache.whirr.service.hbase.HBaseThriftServerClusterActionHandler

Added: incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java (added)
+++ incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,126 @@
+/**
+ * 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.service.hbase.integration;
+
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.HTable;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.whirr.service.Cluster;
+import org.apache.whirr.service.ClusterSpec;
+import org.apache.whirr.service.Service;
+import org.apache.whirr.service.hadoop.HadoopProxy;
+import org.apache.whirr.ssh.KeyPair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
+
+public class HBaseServiceController {
+
+  private static final Logger LOG =
+    LoggerFactory.getLogger(HBaseServiceController.class);
+
+  private static final HBaseServiceController INSTANCE =
+    new HBaseServiceController();
+
+  public static HBaseServiceController getInstance() {
+    return INSTANCE;
+  }
+
+  private boolean running;
+  private ClusterSpec clusterSpec;
+  private Service service;
+  private HadoopProxy proxy;
+  private Cluster cluster;
+
+  private HBaseServiceController() {
+  }
+
+  public synchronized boolean ensureClusterRunning() throws Exception {
+    if (running) {
+      LOG.info("Cluster already running.");
+      return false;
+    } else {
+      startup();
+      return true;
+    }
+  }
+
+  public synchronized void startup() throws Exception {
+    LOG.info("Starting up cluster...");
+    CompositeConfiguration config = new CompositeConfiguration();
+    if (System.getProperty("config") != null) {
+      config.addConfiguration(new PropertiesConfiguration(System.getProperty("config")));
+    }
+    config.addConfiguration(new PropertiesConfiguration("whirr-hbase-test.properties"));
+    clusterSpec = new ClusterSpec(config);
+    if (clusterSpec.getPrivateKey() == null) {
+      Map<String, String> pair = KeyPair.generate();
+      clusterSpec.setPublicKey(pair.get("public"));
+      clusterSpec.setPrivateKey(pair.get("private"));
+    }
+    service = new Service();
+
+    cluster = service.launchCluster(clusterSpec);
+    proxy = new HadoopProxy(clusterSpec, cluster);
+    proxy.start();
+
+    Configuration conf = getConfiguration();
+    waitForMaster(conf);
+    running = true;
+  }
+
+  public Cluster getCluster() {
+    return cluster;
+  }
+
+  public Configuration getConfiguration() {
+    Configuration conf = new Configuration();
+    for (Entry<Object, Object> entry : cluster.getConfiguration().entrySet()) {
+      conf.set(entry.getKey().toString(), entry.getValue().toString());
+    }
+    return conf;
+  }
+
+  private static void waitForMaster(Configuration conf) throws IOException {
+    LOG.info("Waiting for master...");
+    HTable t = new HTable(conf, HConstants.META_TABLE_NAME);
+    ResultScanner s = t.getScanner(new Scan());
+    while (s.next() != null) {
+      continue;
+    }
+    LOG.info("Master reported in. Continuing.");
+  }
+
+  public synchronized void shutdown() throws IOException, InterruptedException {
+    LOG.info("Shutting down cluster...");
+    if (proxy != null) {
+      proxy.stop();
+    }
+    service.destroyCluster(clusterSpec);
+    running = false;
+  }
+
+}

Propchange: incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceController.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java (added)
+++ incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java Mon Jan  3 04:33:30 2011
@@ -0,0 +1,71 @@
+/**
+ * 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.service.hbase.integration;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.client.*;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static junit.framework.Assert.assertEquals;
+import static junit.framework.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+public class HBaseServiceTest {
+
+  private static byte [] ROW = Bytes.toBytes("testRow");
+  private static byte [] FAMILY = Bytes.toBytes("testFamily");
+  private static byte [] QUALIFIER = Bytes.toBytes("testQualifier");
+  private static byte [] VALUE = Bytes.toBytes("testValue");
+
+  private static HBaseServiceController controller =
+    HBaseServiceController.getInstance();
+
+  @BeforeClass
+  public static void setUp() throws Exception {
+    controller.ensureClusterRunning();
+  }
+
+  @AfterClass
+  public static void tearDown() throws Exception {
+    controller.shutdown();
+  }
+
+  @Test
+  public void test() throws Exception {
+    Configuration conf = controller.getConfiguration();
+    HBaseTestingUtility testUtil = new HBaseTestingUtility(conf);
+    byte [] TABLE = Bytes.toBytes("testtable");
+    HTable ht = testUtil.createTable(TABLE, FAMILY);
+    Put put = new Put(ROW);
+    put.add(FAMILY, QUALIFIER, VALUE);
+    ht.put(put);
+    Scan scan = new Scan();
+    scan.addColumn(FAMILY, TABLE);
+    ResultScanner scanner = ht.getScanner(scan);
+    Result result = scanner.next();
+    assertTrue("Expected null result", result == null);
+    scanner.close();
+    System.out.println("Done.");
+  }
+
+}

Propchange: incubator/whirr/trunk/services/hbase/src/test/java/org/apache/whirr/service/hbase/integration/HBaseServiceTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/whirr/trunk/services/hbase/src/test/resources/log4j.xml
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/test/resources/log4j.xml?rev=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/resources/log4j.xml (added)
+++ incubator/whirr/trunk/services/hbase/src/test/resources/log4j.xml Mon Jan  3 04:33:30 2011
@@ -0,0 +1,196 @@
+<?xml version="1.0" encoding="UTF-8"?>
+    <!--
+   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.
+    -->
+<!DOCTYPE log4j:configuration SYSTEM "log4j.dtd">
+
+    <!--
+        For more configuration infromation and examples see the Apache
+        Log4j website: http://logging.apache.org/log4j/
+    -->
+<log4j:configuration xmlns:log4j="http://jakarta.apache.org/log4j/"
+    debug="false">
+    
+    <appender name="CONSOLE" class="org.apache.log4j.ConsoleAppender">
+        <param name="Threshold" value="INFO" />
+        <layout class="org.apache.log4j.PatternLayout"> 
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n"/> 
+        </layout> 
+    </appender>
+
+    <!-- A time/date based rolling appender -->
+    <appender name="WIREFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds-wire.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+
+    <!-- A time/date based rolling appender -->
+    <appender name="FILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+
+    <!-- A time/date based rolling appender -->
+    <appender name="BLOBSTOREFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds-blobstore.log" />
+        <param name="Append" value="true" />
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+        <param name="Threshold" value="TRACE" />
+        <layout class="org.apache.log4j.PatternLayout">
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+        </layout>
+    </appender>
+    
+
+    <!-- A time/date based rolling appender -->
+    <appender name="COMPUTEFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/jclouds-compute.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+    
+    <!-- A time/date based rolling appender -->
+    <appender name="WHIRRFILE" class="org.apache.log4j.DailyRollingFileAppender">
+        <param name="File" value="target/test-data/whirr.log" />
+        <param name="Append" value="true" />
+
+        <!-- Rollover at midnight each day -->
+        <param name="DatePattern" value="'.'yyyy-MM-dd" />
+
+        <param name="Threshold" value="TRACE" />
+
+        <layout class="org.apache.log4j.PatternLayout">
+            <!-- The default pattern: Date Priority [Category] Message\n -->
+            <param name="ConversionPattern" value="%d %-5p [%c] (%t) %m%n" />
+
+            <!--
+                The full pattern: Date MS Priority [Category]
+                (Thread:NDC) Message\n <param name="ConversionPattern"
+                value="%d %-5r %-5p [%c] (%t:%x) %m%n"/>
+            -->
+        </layout>
+    </appender>
+
+    <appender name="ASYNCCOMPUTE" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="COMPUTEFILE" />
+    </appender>
+    <appender name="ASYNC" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="FILE" />
+    </appender>
+
+    <appender name="ASYNCWIRE" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="WIREFILE" />
+    </appender>
+
+    <appender name="ASYNCBLOBSTORE" class="org.apache.log4j.AsyncAppender">
+        <appender-ref ref="BLOBSTOREFILE" />
+    </appender>
+    <!-- ================ -->
+    <!-- Limit categories -->
+    <!-- ================ -->
+    <category name="jclouds.blobstore">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCBLOBSTORE" />
+    </category>
+
+    <category name="org.jclouds">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNC" />
+    </category>
+    
+    <category name="org.apache.whirr">
+        <priority value="DEBUG" />
+        <appender-ref ref="WHIRRFILE" />
+    </category>
+
+    <category name="jclouds.headers">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCWIRE" />
+    </category>
+    <category name="jclouds.compute">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCCOMPUTE" />
+    </category>
+
+    <category name="jclouds.wire">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCWIRE" />
+    </category><!--
+    
+       <category name="jclouds.signature">
+        <priority value="DEBUG" />
+        <appender-ref ref="ASYNCWIRE" />
+    </category>
+    
+    
+    --><!--  ======================= -->
+    <!-- Setup the Root category -->
+    <!-- ======================= -->
+
+    <root>
+        <priority value="WARN" />
+        <appender-ref ref="CONSOLE" />
+    </root>
+
+</log4j:configuration>
\ No newline at end of file

Propchange: incubator/whirr/trunk/services/hbase/src/test/resources/log4j.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: 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=1054522&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties (added)
+++ incubator/whirr/trunk/services/hbase/src/test/resources/whirr-hbase-test.properties Mon Jan  3 04:33:30 2011
@@ -0,0 +1,23 @@
+#
+# 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.cluster-name=hbaseclustertest
+whirr.instance-templates=1 zk,1 nn+jt+hbase-master,1 dn+tt+hbase-regionserver
+whirr.provider=ec2
+whirr.identity=${sys:whirr.test.identity}
+whirr.credential=${sys:whirr.test.credential}
+whirr.private-key-file=${sys:user.home}/.ssh/id_rsa

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