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/08/08 00:04:55 UTC

svn commit: r1154780 - in /incubator/whirr/trunk: ./ recipes/ services/cdh/src/main/resources/functions/ services/hbase/src/main/java/org/apache/whirr/service/hbase/ services/hbase/src/main/resources/ services/hbase/src/main/resources/functions/

Author: asavu
Date: Sun Aug  7 22:04:54 2011
New Revision: 1154780

URL: http://svn.apache.org/viewvc?rev=1154780&view=rev
Log:
WHIRR-339. Allow to specify hbase-site.xml properties through cluster configuration file (Bruno Dumon via asavu)

Added:
    incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConfigurationBuilder.java
Modified:
    incubator/whirr/trunk/CHANGES.txt
    incubator/whirr/trunk/recipes/hbase-ec2-0.90.properties
    incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties
    incubator/whirr/trunk/services/cdh/src/main/resources/functions/configure_cdh_hbase.sh
    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/main/resources/functions/configure_hbase.sh
    incubator/whirr/trunk/services/hbase/src/main/resources/whirr-hbase-default.properties

Modified: incubator/whirr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/CHANGES.txt?rev=1154780&r1=1154779&r2=1154780&view=diff
==============================================================================
--- incubator/whirr/trunk/CHANGES.txt (original)
+++ incubator/whirr/trunk/CHANGES.txt Sun Aug  7 22:04:54 2011
@@ -45,6 +45,9 @@ Trunk (unreleased changes)
 
     WHIRR-257. Remove outdated Python contrib (Tom White via asavu)
 
+    WHIRR-339. Allow to specify hbase-site.xml properties through 
+    cluster configuration file (Bruno Dumon via asavu)
+
   BUG FIXES
 
     WHIRR-315. Temporary override Providers#withIds until jclouds 

Modified: incubator/whirr/trunk/recipes/hbase-ec2-0.90.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/hbase-ec2-0.90.properties?rev=1154780&r1=1154779&r2=1154780&view=diff
==============================================================================
--- incubator/whirr/trunk/recipes/hbase-ec2-0.90.properties (original)
+++ incubator/whirr/trunk/recipes/hbase-ec2-0.90.properties Sun Aug  7 22:04:54 2011
@@ -28,6 +28,9 @@ whirr.cluster-name=hbase-0.90
 # Change the number of machines in the cluster here
 whirr.instance-templates=1 zookeeper+hadoop-namenode+hadoop-jobtracker+hbase-master,3 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
 
+# replication level should not be higher than number of data nodes
+hbase-site.dfs.replication=2
+
 # For EC2 set AWS_ACCESS_KEY_ID and AWS_SECRET_ACCESS_KEY environment variables.
 whirr.provider=aws-ec2
 whirr.identity=${env:AWS_ACCESS_KEY_ID}

Modified: incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties?rev=1154780&r1=1154779&r2=1154780&view=diff
==============================================================================
--- incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties (original)
+++ incubator/whirr/trunk/recipes/hbase-ec2-cdh.properties Sun Aug  7 22:04:54 2011
@@ -23,6 +23,9 @@ whirr.cluster-name=hbase-cdh
 
 whirr.instance-templates=1 zookeeper+hadoop-namenode+hadoop-jobtracker+hbase-master,1 hadoop-datanode+hadoop-tasktracker+hbase-regionserver
 
+# replication level should not be higher than number of data nodes
+hbase-site.dfs.replication=1
+
 #custom CDH zookeeper install and configure functions
 whirr.zookeeper.install-function=install_cdh_zookeeper
 whirr.zookeeper.configure-function=configure_cdh_zookeeper

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=1154780&r1=1154779&r2=1154780&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 Sun Aug  7 22:04:54 2011
@@ -77,51 +77,8 @@ function configure_cdh_hbase() {
     chmod a+rwxt /data/tmp
   fi
 
-  ##############################################################################
-  # Modify this section to customize your HBase cluster.
-  ##############################################################################
-  cat > $HBASE_CONF_DIR/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>/data/tmp/hbase-\${user.name}</value>
-</property>
-<property>
- <name>hbase.client.retries.number</name>
- <value>100</value>
-</property>
-<property>
- <name>hbase.zookeeper.recoverable.waittime</name>
- <value>600000</value>
-</property>
-</configuration>
-EOF
+  # Copy generated configuration file in place
+  cp /tmp/hbase-site.xml $HBASE_CONF_DIR
 
   # override JVM options
   cat >> $HBASE_CONF_DIR/hbase-env.sh <<EOF

Added: incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConfigurationBuilder.java
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConfigurationBuilder.java?rev=1154780&view=auto
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConfigurationBuilder.java (added)
+++ incubator/whirr/trunk/services/hbase/src/main/java/org/apache/whirr/service/hbase/HBaseConfigurationBuilder.java Sun Aug  7 22:04:54 2011
@@ -0,0 +1,65 @@
+/**
+ * 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.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.whirr.Cluster;
+import org.apache.whirr.ClusterSpec;
+import org.apache.whirr.service.hadoop.HadoopConfigurationConverter;
+import org.apache.whirr.service.zookeeper.ZooKeeperCluster;
+import org.jclouds.scriptbuilder.domain.Statement;
+
+import java.io.IOException;
+
+import static org.apache.whirr.RolePredicates.role;
+
+public class HBaseConfigurationBuilder {
+  public static Statement buildHBaseSite(String path, ClusterSpec clusterSpec, Cluster cluster)
+      throws ConfigurationException, IOException {
+    Configuration config = buildHBaseSiteConfiguration(clusterSpec, cluster,
+        new PropertiesConfiguration(HBaseConstants.FILE_HBASE_DEFAULT_PROPERTIES));
+    return HadoopConfigurationConverter.asCreateFileStatement(path, config);
+  }
+
+  static Configuration buildHBaseSiteConfiguration(ClusterSpec clusterSpec, Cluster cluster, Configuration defaults)
+      throws ConfigurationException, IOException {
+    Configuration config = build(clusterSpec, cluster, defaults, "hbase-site");
+
+    Cluster.Instance master = cluster.getInstanceMatching(
+      role(HBaseMasterClusterActionHandler.ROLE));
+    String masterHostName = master.getPublicHostName();
+
+    config.setProperty("hbase.rootdir", String.format("hdfs://%s:8020/hbase", masterHostName));
+    config.setProperty("hbase.zookeeper.quorum", ZooKeeperCluster.getHosts(cluster));
+
+    return config;
+  }
+
+  private static Configuration build(ClusterSpec clusterSpec, Cluster cluster, Configuration defaults, String prefix)
+      throws ConfigurationException {
+    CompositeConfiguration config = new CompositeConfiguration();
+    Configuration sub = clusterSpec.getConfigurationForKeysWithPrefix(prefix);
+    config.addConfiguration(sub.subset(prefix)); // remove prefix
+    config.addConfiguration(defaults.subset(prefix));
+    return config;
+  }
+}

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=1154780&r1=1154779&r2=1154780&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 Sun Aug  7 22:04:54 2011
@@ -20,6 +20,7 @@ package org.apache.whirr.service.hbase;
 
 import static org.apache.whirr.RolePredicates.role;
 import static org.apache.whirr.service.FirewallManager.Rule;
+import static org.apache.whirr.service.hbase.HBaseConfigurationBuilder.buildHBaseSite;
 import static org.jclouds.scriptbuilder.domain.Statements.call;
 
 import com.google.common.base.Charsets;
@@ -32,6 +33,7 @@ import java.net.InetAddress;
 import java.util.Map.Entry;
 import java.util.Properties;
 
+import org.apache.commons.configuration.ConfigurationException;
 import org.apache.whirr.Cluster;
 import org.apache.whirr.Cluster.Instance;
 import org.apache.whirr.ClusterSpec;
@@ -91,6 +93,14 @@ public class HBaseMasterClusterActionHan
         .ports(MASTER_WEB_UI_PORT, MASTER_PORT)
     );
 
+    try {
+      event.getStatementBuilder().addStatement(
+        buildHBaseSite("/tmp/hbase-site.xml", clusterSpec, cluster)
+      );
+    } catch (ConfigurationException e) {
+      throw new IOException(e);
+    }
+
     String master = masterPublicAddress.getHostName();
     String quorum = ZooKeeperCluster.getHosts(cluster);
 

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=1154780&r1=1154779&r2=1154780&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 Sun Aug  7 22:04:54 2011
@@ -20,11 +20,13 @@ package org.apache.whirr.service.hbase;
 
 import static org.apache.whirr.RolePredicates.role;
 import static org.apache.whirr.service.FirewallManager.Rule;
+import static org.apache.whirr.service.hbase.HBaseConfigurationBuilder.buildHBaseSite;
 import static org.jclouds.scriptbuilder.domain.Statements.call;
 
 import java.io.IOException;
 import java.net.InetAddress;
 
+import org.apache.commons.configuration.ConfigurationException;
 import org.apache.whirr.Cluster;
 import org.apache.whirr.Cluster.Instance;
 import org.apache.whirr.ClusterSpec;
@@ -79,6 +81,14 @@ public class HBaseRegionServerClusterAct
         .ports(REGIONSERVER_WEB_UI_PORT, REGIONSERVER_PORT)
     );
 
+    try {
+      event.getStatementBuilder().addStatement(
+        buildHBaseSite("/tmp/hbase-site.xml", clusterSpec, cluster)
+      );
+    } catch (ConfigurationException e) {
+      throw new IOException(e);
+    }
+
     String master = masterPublicAddress.getHostName();
     String quorum = ZooKeeperCluster.getHosts(cluster);
 

Modified: incubator/whirr/trunk/services/hbase/src/main/resources/functions/configure_hbase.sh
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/resources/functions/configure_hbase.sh?rev=1154780&r1=1154779&r2=1154780&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/resources/functions/configure_hbase.sh (original)
+++ incubator/whirr/trunk/services/hbase/src/main/resources/functions/configure_hbase.sh Sun Aug  7 22:04:54 2011
@@ -91,51 +91,8 @@ function configure_hbase() {
   mkdir /etc/hbase
   ln -s $HBASE_CONF_DIR /etc/hbase/conf
 
-  ##############################################################################
-  # Modify this section to customize your HBase cluster.
-  ##############################################################################
-  cat > $HBASE_CONF_DIR/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>$MOUNT/tmp/hbase-\${user.name}</value>
-</property>
-<property>
- <name>hbase.client.retries.number</name>
- <value>100</value>
-</property>
-<property>
- <name>hbase.zookeeper.recoverable.waittime</name>
- <value>600000</value>
-</property>
-</configuration>
-EOF
+  # Copy generated configuration file in place
+  cp /tmp/hbase-site.xml $HBASE_CONF_DIR
 
   # override JVM options
   cat >> $HBASE_CONF_DIR/hbase-env.sh <<EOF

Modified: incubator/whirr/trunk/services/hbase/src/main/resources/whirr-hbase-default.properties
URL: http://svn.apache.org/viewvc/incubator/whirr/trunk/services/hbase/src/main/resources/whirr-hbase-default.properties?rev=1154780&r1=1154779&r2=1154780&view=diff
==============================================================================
--- incubator/whirr/trunk/services/hbase/src/main/resources/whirr-hbase-default.properties (original)
+++ incubator/whirr/trunk/services/hbase/src/main/resources/whirr-hbase-default.properties Sun Aug  7 22:04:54 2011
@@ -10,4 +10,25 @@
 #   See the License for the specific language governing permissions and
 #   limitations under the License.
 
-whirr.hbase.tarball.url=http://archive.apache.org/dist/hbase/hbase-0.89.20100924/hbase-0.89.20100924-bin.tar.gz
\ No newline at end of file
+whirr.hbase.tarball.url=http://archive.apache.org/dist/hbase/hbase-0.89.20100924/hbase-0.89.20100924-bin.tar.gz
+
+hbase-site.hbase.cluster.distributed=true
+hbase-site.hbase.regionserver.handler.count=100
+hbase-site.dfs.replication=3
+hbase-site.zookeeper.session.timeout=60000
+hbase-site.hbase.tmp.dir=/data/tmp/hbase-\${user.name}
+
+# This is so that thriftserver (which is an HBase client) would would keep
+# trying to connect for an extended amount of time.
+# It can take some time for HBase to be functional since it requires
+# both master and regionservers to be up, and currently this can take
+# quite some time because the configuration phase does not run concurrently
+# for different templates, and to some extent also because services
+# are not started in order. (WHIRR-314)
+hbase-site.hbase.client.retries.number=100
+
+# This makes HBase cope with the fact that ZooKeeper might be started
+# (a lot) later than HBase. (for similar reasons as
+# hbase.client.retries.number)
+hbase-site.hbase.zookeeper.recoverable.waittime=600000
+