You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by rv...@apache.org on 2013/10/04 22:33:25 UTC

[1/2] git commit: BIGTOP-1108. Solr 4.5.0 has been formally released and we should start using the official release bits

Updated Branches:
  refs/heads/master a34142a7e -> 5de186bc9


BIGTOP-1108. Solr 4.5.0 has been formally released and we should start using the official release bits


Project: http://git-wip-us.apache.org/repos/asf/bigtop/repo
Commit: http://git-wip-us.apache.org/repos/asf/bigtop/commit/822a6ea1
Tree: http://git-wip-us.apache.org/repos/asf/bigtop/tree/822a6ea1
Diff: http://git-wip-us.apache.org/repos/asf/bigtop/diff/822a6ea1

Branch: refs/heads/master
Commit: 822a6ea1ac7cd31368900758a4041a9a63531de5
Parents: a34142a
Author: Roman Shaposhnik <rv...@cloudera.com>
Authored: Thu Oct 3 17:46:28 2013 -0700
Committer: Roman Shaposhnik <rv...@cloudera.com>
Committed: Fri Oct 4 13:18:52 2013 -0700

----------------------------------------------------------------------
 .../puppet/modules/solr/manifests/init.pp       | 45 ++++++++++----------
 .../puppet/modules/solr/templates/jaas.conf     | 24 +++++++++++
 .../puppet/modules/solr/templates/solr          | 19 +++++++--
 .../puppet/modules/solr/templates/solr.xml      | 26 -----------
 bigtop-deploy/puppet/modules/solr/tests/init.pp |  3 +-
 bigtop-packages/src/common/hadoop/init-hdfs.sh  |  4 +-
 bigtop.mk                                       |  6 +--
 7 files changed, 70 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop-deploy/puppet/modules/solr/manifests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/solr/manifests/init.pp b/bigtop-deploy/puppet/modules/solr/manifests/init.pp
index 9163ce9..22c4d9e 100644
--- a/bigtop-deploy/puppet/modules/solr/manifests/init.pp
+++ b/bigtop-deploy/puppet/modules/solr/manifests/init.pp
@@ -14,15 +14,7 @@
 # limitations under the License.
 
 class solr {
-  define solrcloud_config($confdir, $zk) {
-    exec { "ZK $title config upload":
-      command => "/bin/bash -c \"java -classpath '/usr/lib/solr/server/webapps/solr/WEB-INF/lib/*' org.apache.solr.cloud.ZkCLI -cmd makepath /solr -zkhost ${zk} ; java -classpath '/usr/lib/solr/server/webapps/solr/WEB-INF/lib/*' org.apache.solr.cloud.ZkCLI -cmd upconfig  -confdir ${confdir}/${title}/conf -confname $title -zkhost ${zk}/solr\"",
-      logoutput => true,
-    }
-  }
-  
-
-  define server($collections = ["solrcloud"], $port = "1978", $port_admin = "1979", $zk = "localhost:2181") {
+  define server($port = "1978", $port_admin = "1979", $zk = "localhost:2181", $root_url = "hdfs://localhost:8020/solr", $kerberos_realm = "") {
     package { "solr-server":
       ensure => latest,
     }
@@ -33,25 +25,34 @@ class solr {
         require => [Package["solr-server"]],
     }
 
-    file {
-      "/etc/solr/conf/solr.xml":
-        content => template("solr/solr.xml"),
-        require => [Package["solr-server"]],
-    }
-
-    # FIXME: perhap we have to provide a way to manage collection configs
-    solrcloud_config { $collections:
-      zk      => $zk,
-      confdir => "/etc/solr/conf",
-      require => [Package["solr-server"]],
+    exec { "solr init":
+      command => "/bin/bash -c '/usr/bin/solrctl debug-dump | grep -q solr.xml || /usr/bin/solrctl init'",
+      require => [ Package["solr-server"], File["/etc/default/solr"] ],
+      logoutput => true,
     }
 
     service { "solr-server":
       ensure => running,
-      require => [ Package["solr-server"], File["/etc/default/solr"], File["/etc/solr/conf/solr.xml"], Solrcloud_config[$collections] ],
-      subscribe => [Package["solr-server"], File["/etc/default/solr"], File["/etc/solr/conf/solr.xml"] ],
+      require => [ Package["solr-server"], File["/etc/default/solr"], Exec["solr init"] ],
+      subscribe => [Package["solr-server"], File["/etc/default/solr"] ],
       hasrestart => true,
       hasstatus => true,
     } 
+
+    if ($kerberos_realm) {
+      require kerberos::client
+
+      kerberos::host_keytab { "solr":
+        spnego => true,
+        require => Package["solr-server"],
+      }
+
+      file { "/etc/solr/conf/jaas.conf":
+          content => template("solr/jaas.conf"),
+          require => [Package["solr-server"]],
+      }
+
+      Kerberos::Host_keytab <| title == "solr" |> -> Service["solr-server"]
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop-deploy/puppet/modules/solr/templates/jaas.conf
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/solr/templates/jaas.conf b/bigtop-deploy/puppet/modules/solr/templates/jaas.conf
new file mode 100644
index 0000000..aff0d5c
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/solr/templates/jaas.conf
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ * <p/>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p/>
+ * 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.
+ */
+Client {
+     com.sun.security.auth.module.Krb5LoginModule required
+     useKeyTab=true
+     useTicketCache=false
+     keyTab="/etc/solr.keytab"
+     principal="solr/<%= fqdn %>@<%= kerberos_realm %>";
+};

http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop-deploy/puppet/modules/solr/templates/solr
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/solr/templates/solr b/bigtop-deploy/puppet/modules/solr/templates/solr
index ffab877..0f17ee9 100644
--- a/bigtop-deploy/puppet/modules/solr/templates/solr
+++ b/bigtop-deploy/puppet/modules/solr/templates/solr
@@ -12,9 +12,22 @@
 # 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.
+SOLR_LOG=/var/log/solr
 SOLR_PORT="<%= port %>"
 SOLR_ADMIN_PORT="<%= port_admin %>"
 SOLR_ZK_ENSEMBLE="<%= zk %>/solr"
-SOLR_LOG=/var/log/solr
-SOLR_DATA_DIR=/var/lib/solr/index
-# CATALINA_OPTS="-Dfoo=bar"
+SOLR_HDFS_HOME=<%= root_url %>/solr
+SOLR_HDFS_CONFIG=/etc/hadoop/conf
+
+<% if @kerberos_realm == "" %>
+SOLR_AUTHENTICATION_TYPE=simple
+SOLR_AUTHENTICATION_SIMPLE_ALLOW_ANON=true
+<% else %>
+SOLR_KERBEROS_ENABLED=true
+SOLR_KERBEROS_KEYTAB=/etc/solr.keytab
+SOLR_KERBEROS_PRINCIPAL="solr/<%= fqdn %>@<%= kerberos_realm %>"
+SOLR_AUTHENTICATION_KERBEROS_KEYTAB=/etc/solr.keytab
+SOLR_AUTHENTICATION_KERBEROS_PRINCIPAL="HTTP/<%= fqdn %>@<%= kerberos_realm %>"
+SOLR_AUTHENTICATION_KERBEROS_NAME_RULES=DEFAULT
+SOLR_AUTHENTICATION_JAAS_CONF=/etc/solr/conf/jaas.conf
+<% end %>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop-deploy/puppet/modules/solr/templates/solr.xml
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/solr/templates/solr.xml b/bigtop-deploy/puppet/modules/solr/templates/solr.xml
deleted file mode 100644
index 66309f3..0000000
--- a/bigtop-deploy/puppet/modules/solr/templates/solr.xml
+++ /dev/null
@@ -1,26 +0,0 @@
-<?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.
--->
-<solr persistent="false">
-  <cores adminPath="/admin/cores" host="<%= fqdn %>" hostPort="<%= port %>" hostContext="${hostContext:}" zkClientTimeout="${zkClientTimeout:15000}">
-<% collections.each do |collection| %>
-    <core name="<%= collection %>_<%= fqdn.to_s.tr('.', '_') %>" shard="<%= collection %>_<%= fqdn.to_s.tr('.', '_') %>" collection="<%= collection %>"
-          instanceDir="<%= fqdn.to_s.tr('.', '_') %>/" schema="schema.xml" config="solrconfig.xml"/>
-<% end %>
-  </cores>
-</solr>
-

http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop-deploy/puppet/modules/solr/tests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/solr/tests/init.pp b/bigtop-deploy/puppet/modules/solr/tests/init.pp
index 643abb5..cc3f73c 100644
--- a/bigtop-deploy/puppet/modules/solr/tests/init.pp
+++ b/bigtop-deploy/puppet/modules/solr/tests/init.pp
@@ -14,8 +14,9 @@
 # limitations under the License.
 
 solr::server { "test-solr-server": 
-  collections => ["collection1", "solrcloud", "a", "b", "c"],
   port        => "1978",
   port_admin  => "1979",
   zk          => "localhost:2181",
+  root_url    => "hdfs://localhost:8020/solr",
+  kerberos_realm => "KRB.YOU.ORG",
 }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop-packages/src/common/hadoop/init-hdfs.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hadoop/init-hdfs.sh b/bigtop-packages/src/common/hadoop/init-hdfs.sh
index bc96761..3a5fe36 100755
--- a/bigtop-packages/src/common/hadoop/init-hdfs.sh
+++ b/bigtop-packages/src/common/hadoop/init-hdfs.sh
@@ -34,6 +34,8 @@ su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -chmod -R 1777 /var/log/hadoop-yarn/
 su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -chown yarn:mapred /var/log/hadoop-yarn/apps'
 su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -mkdir /hbase'
 su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -chown hbase:hbase /hbase'
+su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -mkdir /solr'
+su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -chown solr:solr /solr'
 su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -mkdir /benchmarks'
 su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -chmod -R 777 /benchmarks'
 su -s /bin/bash hdfs -c '/usr/bin/hadoop fs -mkdir /user'
@@ -94,4 +96,4 @@ if [ "$1" = "-u" ] ; then
     su -s /bin/bash hdfs -c "/usr/bin/hadoop fs -chmod -R 755 /user/${USER}"
     su -s /bin/bash hdfs -c "/usr/bin/hadoop fs -chown ${USER} /user/${USER}"
   fi
-fi
\ No newline at end of file
+fi

http://git-wip-us.apache.org/repos/asf/bigtop/blob/822a6ea1/bigtop.mk
----------------------------------------------------------------------
diff --git a/bigtop.mk b/bigtop.mk
index f7235d5..502573d 100644
--- a/bigtop.mk
+++ b/bigtop.mk
@@ -219,10 +219,8 @@ SOLR_RELEASE_VERSION=1
 SOLR_TARBALL_DST=solr-$(SOLR_BASE_VERSION)-src.tgz
 SOLR_TARBALL_SRC=$(SOLR_TARBALL_DST)
 SOLR_DOWNLOAD_PATH=/lucene/solr/$(SOLR_BASE_VERSION)
-#SOLR_SITE=$(APACHE_MIRROR)$(SOLR_DOWNLOAD_PATH)
-#SOLR_ARCHIVE=$(APACHE_ARCHIVE)$(SOLR_DOWNLOAD_PATH)
-SOLR_SITE=http://people.apache.org/~jpountz/staging_area/lucene-solr-4.5.0-RC3-rev1526423/solr/
-SOLR_ARCHIVE=$(SOLR_SITE)
+SOLR_SITE=$(APACHE_MIRROR)$(SOLR_DOWNLOAD_PATH)
+SOLR_ARCHIVE=$(APACHE_ARCHIVE)$(SOLR_DOWNLOAD_PATH)
 $(eval $(call PACKAGE,solr,SOLR))
 
 # Crunch


[2/2] git commit: BIGTOP-1106. update our puppet deployment topology

Posted by rv...@apache.org.
BIGTOP-1106. update our puppet deployment topology


Project: http://git-wip-us.apache.org/repos/asf/bigtop/repo
Commit: http://git-wip-us.apache.org/repos/asf/bigtop/commit/5de186bc
Tree: http://git-wip-us.apache.org/repos/asf/bigtop/tree/5de186bc
Diff: http://git-wip-us.apache.org/repos/asf/bigtop/diff/5de186bc

Branch: refs/heads/master
Commit: 5de186bc956b0f19ec9a000b9a343baaa249b915
Parents: 822a6ea
Author: Roman Shaposhnik <rv...@cloudera.com>
Authored: Thu Oct 3 17:50:05 2013 -0700
Committer: Roman Shaposhnik <rv...@cloudera.com>
Committed: Fri Oct 4 13:21:20 2013 -0700

----------------------------------------------------------------------
 bigtop-deploy/puppet/manifests/cluster.pp       | 108 +++++++++++--------
 .../modules/hadoop-hbase/manifests/init.pp      |  25 ++++-
 .../puppet/modules/hadoop-hbase/tests/init.pp   |   5 +-
 3 files changed, 86 insertions(+), 52 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/5de186bc/bigtop-deploy/puppet/manifests/cluster.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/manifests/cluster.pp b/bigtop-deploy/puppet/manifests/cluster.pp
index 303cc74..cec9cec 100644
--- a/bigtop-deploy/puppet/manifests/cluster.pp
+++ b/bigtop-deploy/puppet/manifests/cluster.pp
@@ -59,36 +59,44 @@ class hadoop_cluster_node {
   $hadoop_jobtracker_thrift_port     = extlookup("hadoop_jobtracker_thrift_port", "9290")
   $hadoop_mapred_jobtracker_plugins  = extlookup("hadoop_mapred_jobtracker_plugins", "")
   $hadoop_mapred_tasktracker_plugins = extlookup("hadoop_mapred_tasktracker_plugins", "")
-  $hadoop_ha_zookeeper_quorum        = "${hadoop_head_node}:2181"
-  # $hadoop_mapred_jobtracker_plugins="org.apache.hadoop.thriftfs.ThriftJobTrackerPlugin"
-  # $hadoop_mapred_tasktracker_plugins="org.apache.hadoop.mapred.TaskTrackerCmonInst"
+
+  $hadoop_zookeeper_port             = extlookup("hadoop_zookeeper_port", "2181")
+  $solrcloud_port                    = extlookup("solrcloud_port", "1978")
+  $solrcloud_admin_port              = extlookup("solrcloud_admin_port", "1979")
+  $hadoop_oozie_port                 = extlookup("hadoop_oozie_port", "11000")
+  $hadoop_httpfs_port                = extlookup("hadoop_httpfs_port", "14000")
+  $hadoop_rm_http_port               = extlookup("hadoop_rm_http_port", "8088")
+  $hadoop_rm_proxy_port              = extlookup("hadoop_rm_proxy_port", "8088")
+  $hadoop_history_server_port        = extlookup("hadoop_history_server_port", "19888")
+  $hbase_thrift_port                 = extlookup("hbase_thrift_port", "9090")
+
+  $hadoop_ha_zookeeper_quorum        = "${hadoop_head_node}:${hadoop_zookeeper_port}"
+  $solrcloud_zk                      = "${hadoop_head_node}:${hadoop_zookeeper_port}"
+  $hbase_thrift_address              = "${hadoop_head_node}:${hbase_thrift_port}"
+  $hadoop_oozie_url                  = "http://${hadoop_head_node}:${hadoop_oozie_port}/oozie"
+  $hadoop_httpfs_url                 = "http://${hadoop_head_node}:${hadoop_httpfs_port}/webhdfs/v1"
+  $sqoop_server_url                  = "http://${hadoop_head_node}:${sqoop_server_port}/sqoop"
+  $solrcloud_url                     = "http://${hadoop_head_node}:${solrcloud_port}/solr/"
+  $hadoop_rm_url                     = "http://${hadoop_head_node}:${hadoop_rm_http_port}"
+  $hadoop_rm_proxy_url               = "http://${hadoop_head_node}:${hadoop_rm_proxy_port}"
+  $hadoop_history_server_url         = "http://${hadoop_head_node}:${hadoop_history_server_port}"
 
   $bigtop_real_users = [ 'jenkins', 'testuser', 'hudson' ]
 
-  $hadoop_core_proxyusers = { oozie => { groups => 'hudson,testuser,root,hadoop,jenkins,oozie,httpfs,hue,users', hosts => "${hadoop_head_node},localhost,127.0.0.1" },
-                                hue => { groups => 'hudson,testuser,root,hadoop,jenkins,oozie,httpfs,hue,users', hosts => "${hadoop_head_node},localhost,127.0.0.1" },
-                             httpfs => { groups => 'hudson,testuser,root,hadoop,jenkins,oozie,httpfs,hue,users', hosts => "${hadoop_head_node},localhost,127.0.0.1" } }
+  $hadoop_core_proxyusers = { oozie => { groups => 'hudson,testuser,root,hadoop,jenkins,oozie,httpfs,hue,users', hosts => "*" },
+                                hue => { groups => 'hudson,testuser,root,hadoop,jenkins,oozie,httpfs,hue,users', hosts => "*" },
+                             httpfs => { groups => 'hudson,testuser,root,hadoop,jenkins,oozie,httpfs,hue,users', hosts => "*" } }
 
   $hbase_relative_rootdir        = extlookup("hadoop_hbase_rootdir", "/hbase")
   $hadoop_hbase_rootdir = "$hadoop_namenode_uri$hbase_relative_rootdir"
   $hadoop_hbase_zookeeper_quorum = $hadoop_head_node
   $hbase_heap_size               = extlookup("hbase_heap_size", "1024")
+  $hbase_thrift_server           = $hadoop_head_node
 
   $giraph_zookeeper_quorum       = $hadoop_head_node
 
   $hadoop_zookeeper_ensemble = ["$hadoop_head_node:2888:3888"]
 
-  $hadoop_oozie_url  = "http://${hadoop_head_node}:11000/oozie"
-  $hadoop_httpfs_url = "http://${hadoop_head_node}:14000/webhdfs/v1"
-  $hadoop_rm_url             = "http://${hadoop_head_node}:8088"
-  $hadoop_rm_proxy_url       = "http://${hadoop_head_node}:8088"
-  $hadoop_history_server_url = "http://${hadoop_head_node}:19888"
-
-  $solrcloud_collections = ["collection1"]
-  $solrcloud_port        = "1978"
-  $solrcloud_port_admin  = "1979"
-  $solrcloud_zk          = "${hadoop_head_node}:2181"
-
   # Set from facter if available
   $roots              = extlookup("hadoop_storage_dirs",       split($hadoop_storage_dirs, ";"))
   $namenode_data_dirs = extlookup("hadoop_namenode_data_dirs", append_each("/namenode", $roots))
@@ -157,10 +165,11 @@ class hadoop_worker_node inherits hadoop_cluster_node {
   }
 
   solr::server { "solrcloud server":
-       collections => $solrcloud_collections,
        port        => $solrcloud_port,
-       port_admin  => $solrcloud_port_admin,
+       port_admin  => $solrcloud_admin_port,
        zk          => $solrcloud_zk,
+       root_url    => $hadoop_namenode_uri,
+       kerberos_realm => $kerberos_realm,
   }
 }
 
@@ -212,12 +221,6 @@ class hadoop_head_node inherits hadoop_worker_node {
         auth => $hadoop_security_authentication,
   }
 
-  hadoop::httpfs { "httpfs":
-        namenode_host => $hadoop_namenode_host,
-        namenode_port => $hadoop_namenode_port,
-        auth => $hadoop_security_authentication,
-  }
-
   hadoop-hbase::master { "hbase master":
         rootdir => $hadoop_hbase_rootdir,
         heap_size => $hbase_heap_size,
@@ -229,11 +232,6 @@ class hadoop_head_node inherits hadoop_worker_node {
         kerberos_realm => $kerberos_realm, 
   }
 
-  hadoop-sqoop::server { "sqoop server":
-  }
-  hadoop-sqoop::client { "sqoop client":
-  }
-
   hcatalog::server { "hcatalog server":
         kerberos_realm => $kerberos_realm,
   }
@@ -241,20 +239,6 @@ class hadoop_head_node inherits hadoop_worker_node {
         kerberos_realm => $kerberos_realm,
   }
 
-  hue::server { "hue server":
-        rm_url      => $hadoop_rm_url,
-        rm_proxy_url => $hadoop_rm_proxy_url,
-        history_server_url => $hadoop_history_server_url,
-        webhdfs_url => $hadoop_httpfs_url,
-        rm_host     => $hadoop_rm_host,
-        rm_port     => $hadoop_rm_port,
-        oozie_url   => $hadoop_oozie_url,
-        default_fs  => $hadoop_namenode_uri,
-        kerberos_realm => $kerberos_realm,
-  }
-  Hadoop::Httpfs<||> -> Hue::Server<||>
-  Hadoop-sqoop::Client<||> -> Hue::Server<||>
-
   hadoop-zookeeper::server { "zookeeper":
         myid => "0",
         ensemble => $hadoop_zookeeper_ensemble,
@@ -270,7 +254,6 @@ class hadoop_head_node inherits hadoop_worker_node {
   Exec<| title == "init hdfs" |> -> Hadoop-hbase::Master<||>
   Exec<| title == "init hdfs" |> -> Hadoop::Resourcemanager<||>
   Exec<| title == "init hdfs" |> -> Hadoop::Historyserver<||>
-  Exec<| title == "init hdfs" |> -> Hadoop::Httpfs<||>
   Exec<| title == "init hdfs" |> -> Hadoop::Rsync_hdfs<||>
   Exec<| title == "init hdfs" |> -> Hadoop-oozie::Server<||>
 
@@ -290,6 +273,37 @@ class standby_head_node inherits hadoop_cluster_node {
 }
 
 class hadoop_gateway_node inherits hadoop_cluster_node {
+  $hbase_thrift_address              = "${fqdn}:${hbase_thrift_port}"
+  $hadoop_httpfs_url                 = "http://${fqdn}:${hadoop_httpfs_port}/webhdfs/v1"
+  $sqoop_server_url                  = "http://${fqdn}:${sqoop_server_port}/sqoop"
+  $solrcloud_url                     = "http://${fqdn}:${solrcloud_port}/solr/"
+
+  hadoop-sqoop::server { "sqoop server":
+  }
+
+  hadoop::httpfs { "httpfs":
+        namenode_host => $hadoop_namenode_host,
+        namenode_port => $hadoop_namenode_port,
+        auth => $hadoop_security_authentication,
+  }
+
+  hue::server { "hue server":
+        rm_url      => $hadoop_rm_url,
+        rm_proxy_url => $hadoop_rm_proxy_url,
+        history_server_url => $hadoop_history_server_url,
+        webhdfs_url => $hadoop_httpfs_url,
+        sqoop_url   => $sqoop_server_url,
+        solr_url    => $solrcloud_url,
+        hbase_thrift_url => $hbase_thrift_address, 
+        rm_host     => $hadoop_rm_host,
+        rm_port     => $hadoop_rm_port,
+        oozie_url   => $hadoop_oozie_url,
+        default_fs  => $hadoop_namenode_uri,
+        kerberos_realm => $kerberos_realm,
+  }
+  Hadoop::Httpfs<||> -> Hue::Server<||>
+  Hadoop-hbase::Client<||> -> Hue::Server<||>
+
   hadoop::client { "hadoop client":
     namenode_host => $hadoop_namenode_host,
     namenode_port => $hadoop_namenode_port,
@@ -313,7 +327,9 @@ class hadoop_gateway_node inherits hadoop_cluster_node {
   }
   hadoop-oozie::client { "oozie client":
   }
-  hadoop-hbase::client { "hbase client":
+  hadoop-hbase::client { "hbase thrift client":
+    thrift => true,
+    kerberos_realm => $kerberos_realm,
   }
   hadoop-zookeeper::client { "zookeeper client":
   }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/5de186bc/bigtop-deploy/puppet/modules/hadoop-hbase/manifests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hadoop-hbase/manifests/init.pp b/bigtop-deploy/puppet/modules/hadoop-hbase/manifests/init.pp
index e102986..3bbaa8a 100644
--- a/bigtop-deploy/puppet/modules/hadoop-hbase/manifests/init.pp
+++ b/bigtop-deploy/puppet/modules/hadoop-hbase/manifests/init.pp
@@ -20,7 +20,7 @@ class hadoop-hbase {
     } 
   }
 
-  class common-server-config {
+  class common-config {
     include client-package
     if ($kerberos_realm) {
       require kerberos::client
@@ -45,12 +45,27 @@ class hadoop-hbase {
     }
   }
 
-  define client {
-    include client-package
+  define client($thrift = false, $kerberos_realm = "") {
+    include common-config
+
+    if ($thrift) {
+      package { "hbase-thrift":
+        ensure => latest,
+      }
+
+      service { "hbase-thrift":
+        ensure => running,
+        require => Package["hbase-thrift"],
+        subscribe => File["/etc/hbase/conf/hbase-site.xml", "/etc/hbase/conf/hbase-env.sh"],
+        hasrestart => true,
+        hasstatus => true,
+      }
+      Kerberos::Host_keytab <| title == "hbase" |> -> Service["hbase-thrift"]
+    }
   }
 
   define server($rootdir, $zookeeper_quorum, $kerberos_realm = "", $heap_size="1024") {
-    include common-server-config
+    include common-config
 
     package { "hbase-regionserver":
       ensure => latest,
@@ -67,7 +82,7 @@ class hadoop-hbase {
   }
 
   define master($rootdir, $zookeeper_quorum, $kerberos_realm = "", $heap_size="1024") {
-    include common-server-config
+    include common-config
 
     package { "hbase-master":
       ensure => latest,

http://git-wip-us.apache.org/repos/asf/bigtop/blob/5de186bc/bigtop-deploy/puppet/modules/hadoop-hbase/tests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hadoop-hbase/tests/init.pp b/bigtop-deploy/puppet/modules/hadoop-hbase/tests/init.pp
index f25c08a..c6fc7c0 100644
--- a/bigtop-deploy/puppet/modules/hadoop-hbase/tests/init.pp
+++ b/bigtop-deploy/puppet/modules/hadoop-hbase/tests/init.pp
@@ -13,7 +13,10 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-hadoop-hbase::client { "hbase shell": } 
+hadoop-hbase::client { "hbase thrift client":
+  thrift => true,
+  kerberos_realm => "KRB.YOU.ORG",
+}
 hadoop-hbase::master { "test-hbase-master": 
   rootdir => "hdfs://localhost:17020/hbase",
   zookeeper_quorum => "localhost",