You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by co...@apache.org on 2016/02/25 23:48:58 UTC

[7/7] bigtop git commit: BIGTOP-2325. Deployment recipes for HAWQ

BIGTOP-2325. Deployment recipes for HAWQ

Initial draft for puppet recipes


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

Branch: refs/heads/BIGTOP-2320
Commit: 421027ffea3923316a5ea52556e2c94b622de712
Parents: a5243a0
Author: Konstantin Boudnik <co...@apache.org>
Authored: Tue Feb 16 21:27:27 2016 +0300
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Thu Feb 25 14:48:35 2016 -0800

----------------------------------------------------------------------
 .../puppet/hieradata/bigtop/cluster.yaml        |   4 +
 bigtop-deploy/puppet/manifests/cluster.pp       |   4 +
 .../puppet/modules/hawq/manifests/init.pp       | 112 +++++++
 .../puppet/modules/hawq/templates/gpcheck.cnf   |  58 ++++
 .../puppet/modules/hawq/templates/hawq-site.xml | 158 +++++++++
 .../puppet/modules/hawq/templates/hawq.default  |  32 ++
 .../modules/hawq/templates/hdfs-client.xml      | 331 +++++++++++++++++++
 .../modules/hawq/templates/yarn-client.xml      | 123 +++++++
 bigtop-deploy/puppet/modules/hawq/tests/init.pp |  16 +
 .../src/common/hadoop/init-hcfs.json            |   1 +
 bigtop-packages/src/common/hawq/hawq.svc        |   7 +-
 bigtop-packages/src/common/hawq/install_hawq.sh |   2 +
 bigtop-packages/src/deb/hawq/rules              |   4 +-
 bigtop.bom                                      |   2 +-
 14 files changed, 848 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/hieradata/bigtop/cluster.yaml
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/hieradata/bigtop/cluster.yaml b/bigtop-deploy/puppet/hieradata/bigtop/cluster.yaml
index de98502..eb604ca 100644
--- a/bigtop-deploy/puppet/hieradata/bigtop/cluster.yaml
+++ b/bigtop-deploy/puppet/hieradata/bigtop/cluster.yaml
@@ -174,3 +174,7 @@ zeppelin::server::spark_master_url: "yarn-client"
 zeppelin::server::hiveserver2_url: "jdbc:hive2://%{hiera('hadoop-hive::common::hiveserver2_host')}:%{hiera('hadoop-hive::common::hiveserver2_port')}"
 zeppelin::server::hiveserver2_user: "%{hiera('bigtop::hiveserver2_user')}"
 zeppelin::server::hiveserver2_password: "%{hiera('bigtop::hiveserver2_password')}"
+
+# hawq
+bigtop::hawq_master_node:	"%{hiera('bigtop::hawq_master_node')}"
+bigtop::hawq_master_port:	"5432"

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/manifests/cluster.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/manifests/cluster.pp b/bigtop-deploy/puppet/manifests/cluster.pp
index a0be567..f80ef5a 100644
--- a/bigtop-deploy/puppet/manifests/cluster.pp
+++ b/bigtop-deploy/puppet/manifests/cluster.pp
@@ -105,6 +105,9 @@ $roles_map = {
   zeppelin => {
     master => ["zeppelin-server"],
   },
+  hawq => {
+    master => ["hawq"],
+  },
 }
 
 class hadoop_cluster_node (
@@ -159,6 +162,7 @@ class node_with_roles ($roles = hiera("bigtop::roles")) inherits hadoop_cluster_
     "hadoop_hive",
     "hadoop_oozie",
     "hadoop_pig",
+    "hawq",
     "sqoop2",
     "hadoop_zookeeper",
     "hcatalog",

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/manifests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/manifests/init.pp b/bigtop-deploy/puppet/modules/hawq/manifests/init.pp
new file mode 100644
index 0000000..8ba6adb
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/manifests/init.pp
@@ -0,0 +1,112 @@
+# 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.
+
+class hawq {
+  class deploy ($roles) {
+    if ("hawq" in $roles) {
+      hawq::cluster_node { "hawq-node": }
+    }
+  }
+
+  define cluster_node() {
+    $hadoop_head_node = hiera("bigtop::hadoop_head_node")
+    $hadoop_namenode_port = hiera("hadoop::common_hdfs::hadoop_namenode_port", "8020")
+    $hawq_head = hiera("bigtop::hawq_master_node", "localhost")
+    $hawq_head_port = hiera('bigtop::hawq_master_port', "5432")
+    $hawq_yarn_rm_host = hiera('hadoop::common_yarn::hadoop_rm_host')
+    $hawq_yarn_rm_port = hiera('hadoop::common_yarn::hadoop_rm_port')
+
+    package { "hawq":
+      ensure  => latest,
+      require => Package["libhdfs3-devel"],
+      ## require => for centos this crap needs epel-release
+    }
+
+    file { "/etc/default/hawq":
+      content => template("hawq/hawq.default"),
+      require => Package["hawq"],
+    }
+
+    file { "/etc/hawq/conf":
+      ensure  => directory,
+      owner   => 'hawq',
+      group   => 'hawq',
+      mode    => '0755',
+      require => Package["hawq"],
+    }
+    file { "/etc/hawq/conf/hawq-site.xml":
+        content => template('hawq/hawq-site.xml'),
+        require => [File["/etc/hawq/conf"]],
+    }
+    file { "/etc/hawq/conf/gpcheck.cnf":
+        content => template('hawq/gpcheck.cnf'),
+        require => [File["/etc/hawq/conf"]],
+    }
+    file { "/etc/hawq/conf/hdfs-client.xml":
+        content => template('hawq/hdfs-client.xml'),
+        require => [File["/etc/hawq/conf"]],
+    }
+    file { "/etc/hawq/conf/yarn-client.xml":
+        content => template('hawq/yarn-client.xml'),
+        require => [File["/etc/hawq/conf"]],
+    }
+    file { "/etc/hawq/conf/slaves":
+        ensure  => file,
+        content => "localhost", ## TODO - this has to be dynamic
+    }
+
+    exec { "install pygresql modules1":
+      path 	 => ['/usr/bin'],
+      command	 => 'pip --retries=50 --timeout=300 install pg8000 simplejson unittest2 pycrypto pygresql pyyaml lockfile paramiko psi',
+      require	 => [ Package['python-pip', 'postgresql-devel'] ],
+    }
+    exec { "install pygresql modules2":
+      path 	 => ['/usr/bin'],
+      command	 => 'pip --retries=50 --timeout=300 install http://darcs.idyll.org/~t/projects/figleaf-0.6.1.tar.gz',
+      require	 => [ Package['python-pip', 'pychecker'], Exec ['install pygresql modules1'] ],
+      ## HAWQ install instructions are suggesting to
+      ## uninstall postgresql postgresql-libs postgresql-devel at this point
+      ## but I don't think it matter, and for sure looks ugly
+    }
+
+    package { "python-pip":
+      ensure 	 => latest,
+    }
+    package { "pychecker":
+      ensure 	 => latest,
+    }
+    package { "postgresql-devel":
+      ensure 	 => latest,
+    }
+    package { "libhdfs3-devel":
+      ensure     => latest,
+    }
+
+### TODO init require hdfs to be running. Need to test this
+    exec { "hawk init":
+      path 	 => ['/usr/bin'],
+      # Silly init will ask if I am really sure I want to init the cluster
+      command	 => 'echo y | bash -x /usr/bin/hawq init cluster',
+      require	 => [ Package['hawq'], Exec ['install pygresql modules2'] ],
+    }
+
+### TODO The expectation is that init will start the service. I don't think so...
+#    service { "hawq":
+#      ensure  => running,
+#      require => [ Package["hawq"], File["/etc/default/hawq"], Exec["hawk init"] ],
+#     subscribe => [ Package["hawq"], File["/etc/default/hawq", "/etc/hawq/conf/hawq-site.xml"] ]
+#    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/templates/gpcheck.cnf
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/templates/gpcheck.cnf b/bigtop-deploy/puppet/modules/hawq/templates/gpcheck.cnf
new file mode 100644
index 0000000..11ae02f
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/templates/gpcheck.cnf
@@ -0,0 +1,58 @@
+[global]
+configfile_version = 4
+
+[linux.mount]
+mount.points = /
+
+[linux.sysctl]
+sysctl.kernel.shmmax = 500000000
+sysctl.kernel.shmmni = 4096
+sysctl.kernel.shmall = 4000000000
+sysctl.kernel.sem = 250 512000 100 2048
+sysctl.kernel.sysrq = 1
+sysctl.kernel.core_uses_pid = 1
+sysctl.kernel.msgmnb = 65536
+sysctl.kernel.msgmax = 65536
+sysctl.kernel.msgmni = 2048
+sysctl.net.ipv4.tcp_syncookies = 0
+sysctl.net.ipv4.ip_forward = 0
+sysctl.net.ipv4.conf.default.accept_source_route = 0
+sysctl.net.ipv4.tcp_tw_recycle = 1
+sysctl.net.ipv4.tcp_max_syn_backlog = 200000
+sysctl.net.ipv4.conf.all.arp_filter = 1
+sysctl.net.ipv4.ip_local_port_range = 1281 65535
+sysctl.net.core.netdev_max_backlog = 200000
+sysctl.vm.overcommit_memory = 2
+sysctl.fs.nr_open = 3000000
+sysctl.kernel.threads-max = 798720
+sysctl.kernel.pid_max = 798720
+# increase network
+sysctl.net.core.rmem_max = 2097152
+sysctl.net.core.wmem_max = 2097152
+
+[linux.limits]
+soft.nofile = 2900000
+hard.nofile = 2900000
+soft.nproc  = 131072
+hard.nproc  = 131072
+
+[linux.diskusage]
+diskusage.monitor.mounts = /
+diskusage.monitor.usagemax = 90%
+
+[hdfs]
+dfs.mem.namenode.heap = 40960
+dfs.mem.datanode.heap = 6144
+# in hdfs-site.xml
+dfs.support.append = true
+dfs.client.enable.read.from.local = true
+dfs.block.local-path-access.user = gpadmin
+dfs.datanode.max.transfer.threads = 40960
+dfs.client.socket-timeout = 300000000
+dfs.datanode.socket.write.timeout = 7200000
+dfs.namenode.handler.count = 60
+ipc.server.handler.queue.size = 3300
+dfs.datanode.handler.count = 60
+ipc.client.connection.maxidletime = 3600000
+dfs.namenode.accesstime.precision = -1
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/templates/hawq-site.xml
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/templates/hawq-site.xml b/bigtop-deploy/puppet/modules/hawq/templates/hawq-site.xml
new file mode 100644
index 0000000..f3ec4ec
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/templates/hawq-site.xml
@@ -0,0 +1,158 @@
+<?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.
+-->
+
+<configuration>
+  <property>
+    <name>hawq_master_address_host</name>
+    <value><%= @hawq_head %></value>
+    <description>The host name of hawq master.</description>
+  </property>
+
+  <property>
+    <name>hawq_master_address_port</name>
+    <value><%= @hawq_head_port %></value>
+    <description>The port of hawq master.</description>
+  </property>
+
+  <property>
+    <name>hawq_standby_address_host</name>
+    <value>none</value>
+    <description>The host name of hawq standby master.</description>
+  </property>
+
+  <property>
+    <name>hawq_segment_address_port</name>
+    <value>40000</value>
+    <description>The port of hawq segment.</description>
+  </property>
+
+  <property>
+    <name>hawq_dfs_url</name>
+    <value><%= @hadoop_head_node %>:<%= @hadoop_namenode_port %>/hawq_default</value>
+    <description>URL for accessing HDFS.</description>
+  </property>
+
+  <property>
+    <name>hawq_master_directory</name>
+    <value>~/hawq-data-directory/masterdd</value>
+    <description>The directory of hawq master.</description>
+  </property>
+
+  <property>
+    <name>hawq_segment_directory</name>
+    <value>~/hawq-data-directory/segmentdd</value>
+    <description>The directory of hawq segment.</description>
+  </property>
+
+  <property>
+    <name>hawq_master_temp_directory</name>
+    <value>/tmp</value>
+    <description>The temporary directory reserved for hawq master.</description>
+  </property>
+
+  <property>
+    <name>hawq_segment_temp_directory</name>
+    <value>/tmp</value>
+    <description>The temporary directory reserved for hawq segment.</description>
+  </property>
+
+  <!-- HAWQ resource manager parameters -->
+  <property>
+    <name>hawq_global_rm_type</name>
+    <value>none</value>
+    <description>The resource manager type to start for allocating resource.
+    'none' means hawq resource manager exclusively uses whole
+    cluster; 'yarn' means hawq resource manager contacts YARN
+    resource manager to negotiate resource.
+    </description>
+  </property>
+
+  <property>
+    <name>hawq_rm_memory_limit_perseg</name>
+    <value>64GB</value>
+    <description>The limit of memory usage in a hawq segment when
+    hawq_global_rm_type is set 'none'.
+    </description>
+  </property>
+
+  <property>
+    <name>hawq_rm_nvcore_limit_perseg</name>
+    <value>16</value>
+    <description>The limit of virtual core usage in a hawq segment when
+    hawq_global_rm_type is set 'none'.
+    </description>
+  </property>
+
+  <property>
+    <name>hawq_rm_yarn_address</name>
+    <value><%= @hawq_yarn_rm_host %>:<%= @hawq_yarn_rm_port %></value>
+    <description>The address of YARN resource manager server.</description>
+  </property>
+
+  <property>
+    <name>hawq_rm_yarn_scheduler_address</name>
+    <value>localhost:8030</value>
+    <description>The address of YARN scheduler server.</description>
+  </property>
+
+  <property>
+    <name>hawq_rm_yarn_queue_name</name>
+    <value>default</value>
+    <description>The YARN queue name to register hawq resource manager.</description>
+  </property>
+
+  <property>
+    <name>hawq_rm_yarn_app_name</name>
+    <value>hawq</value>
+    <description>The application name to register hawq resource manager in YARN.</description>
+  </property>
+  <!-- HAWQ resource manager parameters end here. -->
+
+  <!-- HAWQ resource enforcement parameters -->
+  <property>
+    <name>hawq_re_cpu_enable</name>
+    <value>false</value>
+    <description>The control to enable/disable CPU resource enforcement.</description>
+  </property>
+
+  <property>
+    <name>hawq_re_cgroup_mount_point</name>
+    <value>/sys/fs/cgroup</value>
+    <description>The mount point of CGroup file system for resource enforcement.
+    For example, /sys/fs/cgroup/cpu/hawq for CPU sub-system.
+    </description>
+  </property>
+
+  <property>
+    <name>hawq_re_cgroup_hierarchy_name</name>
+    <value>hawq</value>
+    <description>The name of the hierarchy to accomodate CGroup directories/files for resource enforcement.
+    For example, /sys/fs/cgroup/cpu/hawq for CPU sub-system.
+    </description>
+  </property>
+
+  <property>
+    <name>hawq_rm_nvseg_perquery_perseg_limit</name>
+    <value>8</value>
+    <description>This is something that init tries to push in</description>
+  </property>
+  <!-- HAWQ resource enforcement parameters end here. -->
+</configuration>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/templates/hawq.default
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/templates/hawq.default b/bigtop-deploy/puppet/modules/hawq/templates/hawq.default
new file mode 100644
index 0000000..19da99e
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/templates/hawq.default
@@ -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.
+
+export HAWQ_HOME="/usr/lib/hawq"
+## Due to some weird scripting, hawq doesn't work without GPHOME
+export GPHOME=$HAWQ_HOME
+export HAWQ_CONF_DIR="/etc/hawq/conf"
+
+export HAWQ_PID_DIR="/var/run/hawq"
+export HAWQ_LOG_DIR="/var/log/hawq"
+export HAWQ_WORK_DIR="/var/run/hawq/work"
+export HAWQ_IDENT_STRING=hawq
+
+export PATH=$HAWQ_HOME/bin:$HAWQ_HOME/ext/python/bin:$PATH
+export LD_LIBRARY_PATH=$HAWQ_HOME/lib:$HAWQ_HOME/ext/python/lib:$LD_LIBRARY_PATH
+export PYTHONPATH=$HAWQ_HOME/lib/python:$HAWQ_HOME/lib/python/pygresql:$PYTHONPATH
+export OPENSSL_CONF=$HAWQ_CONF_DIR/openssl.cnf
+export LIBHDFS3_CONF=$HAWQ_CONF_DIR/hdfs-client.xml
+export LIBYARN_CONF=$HAWQ_CONF_DIR/yarn-client.xml
+export HAWQSITE_CONF=$HAWQ_CONF_DIR/hawq-site.xml

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/templates/hdfs-client.xml
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/templates/hdfs-client.xml b/bigtop-deploy/puppet/modules/hawq/templates/hdfs-client.xml
new file mode 100644
index 0000000..3f08696
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/templates/hdfs-client.xml
@@ -0,0 +1,331 @@
+<?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.
+-->
+
+
+<configuration>
+
+  <!-- KDC
+       <property>
+       <name>hadoop.security.authentication</name>
+       <value>kerberos</value>
+       </property>
+       KDC -->
+
+  <!-- HA
+       <property>
+       <name>dfs.nameservices</name>
+       <value>phdcluster</value>
+       </property>
+
+       <property>
+       <name>dfs.ha.namenodes.phdcluster</name>
+       <value>nn1,nn2</value>
+       </property>
+
+       <property>
+       <name>dfs.namenode.rpc-address.phdcluster.nn1</name>
+       <value>mdw:9000</value>
+       </property>
+
+       <property>
+       <name>dfs.namenode.rpc-address.phdcluster.nn2</name>
+       <value>smdw:9000</value>
+       </property>
+
+<property>
+<name>dfs.namenode.http-address.phdcluster.nn1</name>
+<value>mdw:50070</value>
+</property>
+
+<property>
+<name>dfs.namenode.http-address.phdcluster.nn2</name>
+<value>smdw:50070</value>
+</property>
+
+HA -->
+
+  <!-- RPC client configuration -->
+  <property>
+    <name>rpc.client.timeout</name>
+    <value>3600000</value>
+    <description>
+      timeout interval of a RPC invocation in millisecond. default is 3600000.
+    </description>
+  </property>
+  <property>
+    <name>rpc.client.connect.tcpnodelay</name>
+    <value>true</value>
+    <description>
+      whether set socket TCP_NODELAY to true when connect to RPC server. default is true.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.max.idle</name>
+    <value>10000</value>
+    <description>
+      the max idle time of a RPC connection in millisecond. default is 10000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.ping.interval</name>
+    <value>10000</value>
+    <description>
+      the interval which the RPC client send a heart beat to server. 0 means disable, default is 10000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.connect.timeout</name>
+    <value>600000</value>
+    <description>
+      the timeout interval in millisecond when the RPC client is trying to setup the connection. default is 600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.connect.retry</name>
+    <value>10</value>
+    <description>
+      the max retry times if the RPC client fail to setup the connection to server. default is 10.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.read.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the RPC client is trying to read from server. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.write.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the RPC client is trying to write to server. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.socket.linger.timeout</name>
+    <value>-1</value>
+    <description>
+      set value to socket SO_LINGER when connect to RPC server. -1 means default OS value. default is -1.
+    </description>
+  </property>
+
+  <!-- dfs client configuration -->
+  <property>
+    <name>dfs.client.read.shortcircuit</name>
+    <value>true</value>
+    <description>
+      whether reading block file bypass datanode if the block and the client are on the same node. default is true.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.default.replica</name>
+    <value>3</value>
+    <description>
+      the default number of replica. default is 3.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.prefetchsize</name>
+    <value>10</value>
+    <description>
+      the default number of blocks which information will be prefetched. default is 10.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.client.failover.max.attempts</name>
+    <value>15</value>
+    <description>
+      if multiply namenodes are configured, it is the max retry times when the dfs client try to issue a RPC call. default is 15.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.default.blocksize</name>
+    <value>134217728</value>
+    <description>
+      default block size. default is 134217728.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.client.log.severity</name>
+    <value>INFO</value>
+    <description>
+      the minimal log severity level, valid values include FATAL, ERROR, INFO, DEBUG1, DEBUG2, DEBUG3. default is INFO.
+    </description>
+  </property>
+
+  <!-- input client configuration -->
+  <property>
+    <name>input.connect.timeout</name>
+    <value>600000</value>
+    <description>
+      the timeout interval in millisecond when the input stream is trying to setup the connection to datanode. default is 600000.
+    </description>
+  </property>
+
+  <property>
+    <name>input.read.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the input stream is trying to read from datanode. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>input.write.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the input stream is trying to write to datanode. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>input.localread.default.buffersize</name>
+    <value>2097152</value>
+    <description>
+      number of bytes of the buffer which is used to hold the data from block file and verify checksum.
+      it is only used when "dfs.client.read.shortcircuit" is set to true. default is 1048576.
+    </description>
+  </property>
+
+  <property>
+    <name>input.localread.blockinfo.cachesize</name>
+    <value>1000</value>
+    <description>
+      the size of block file path information cache. default is 1000.
+    </description>
+  </property>
+
+  <property>
+    <name>input.read.getblockinfo.retry</name>
+    <value>3</value>
+    <description>
+      the max retry times when the client fail to get block information from namenode. default is 3.
+    </description>
+  </property>
+
+  <!-- output client configuration -->
+  <property>
+    <name>output.replace-datanode-on-failure</name>
+    <value>false</value>
+    <description>
+      whether the client add new datanode into pipeline if the number of nodes in pipeline is less the specified number of replicas. default is true.
+    </description>
+  </property>
+
+  <property>
+    <name>output.default.chunksize</name>
+    <value>512</value>
+    <description>
+      the number of bytes of a chunk in pipeline. default is 512.
+    </description>
+  </property>
+
+  <property>
+    <name>output.default.packetsize</name>
+    <value>65536</value>
+    <description>
+      the number of bytes of a packet in pipeline. default is 65536.
+    </description>
+  </property>
+
+  <property>
+    <name>output.default.write.retry</name>
+    <value>10</value>
+    <description>
+      the max retry times when the client fail to setup the pipeline. default is 10.
+    </description>
+  </property>
+
+  <property>
+    <name>output.connect.timeout</name>
+    <value>600000</value>
+    <description>
+      the timeout interval in millisecond when the output stream is trying to setup the connection to datanode. default is 600000.
+    </description>
+  </property>
+
+  <property>
+    <name>output.read.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the output stream is trying to read from datanode. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>output.write.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the output stream is trying to write to datanode. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>output.packetpool.size</name>
+    <value>1024</value>
+    <description>
+      the max number of packets in a file's packet pool. default is 1024.
+    </description>
+  </property>
+
+  <property>
+    <name>output.close.timeout</name>
+    <value>900000</value>
+    <description>
+      the timeout interval in millisecond when close an output stream. default is 900000.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.domain.socket.path</name>
+    <value>/var/lib/hadoop-hdfs/dn_socket</value>
+    <description>
+      Optional.  This is a path to a UNIX domain socket that will be used for
+      communication between the DataNode and local HDFS clients.
+      If the string "_PORT" is present in this path, it will be replaced by the
+      TCP port of the DataNode.
+    </description>
+  </property>
+
+  <property>
+    <name>dfs.client.use.legacy.blockreader.local</name>
+    <value>false</value>
+    <description>
+      Legacy short-circuit reader implementation based on HDFS-2246 is used
+      if this configuration parameter is true.
+      This is for the platforms other than Linux
+      where the new implementation based on HDFS-347 is not available.
+    </description>
+  </property>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/templates/yarn-client.xml
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/templates/yarn-client.xml b/bigtop-deploy/puppet/modules/hawq/templates/yarn-client.xml
new file mode 100644
index 0000000..a5f9df7
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/templates/yarn-client.xml
@@ -0,0 +1,123 @@
+<?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.
+-->
+
+
+<configuration>
+
+  <!-- KDC
+       <property>
+       <name>hadoop.security.authentication</name>
+       <value>kerberos</value>
+       </property>
+       KDC -->
+
+  <!-- HA
+       <property>
+       <name>yarn.resourcemanager.ha</name>
+       <value>%RESOURCEMANAGER%:8032,%RESOURCEMANAGER2%:8032</value>
+       </property>
+       <property>
+       <name>yarn.resourcemanager.scheduler.ha</name>
+       <value>%RESOURCEMANAGER%:8030,%RESOURCEMANAGER2%:8030</value>
+       </property>
+       HA -->
+
+  <!-- RPC client configuration -->
+  <property>
+    <name>rpc.client.timeout</name>
+    <value>3600000</value>
+    <description>
+      timeout interval of a RPC invocation in millisecond. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.connect.tcpnodelay</name>
+    <value>true</value>
+    <description>
+      whether set socket TCP_NODELAY to true when connect to RPC server. default is true.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.max.idle</name>
+    <value>10000</value>
+    <description>
+      the max idle time of a RPC connection in millisecond. default is 10000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.ping.interval</name>
+    <value>10000</value>
+    <description>
+      the interval which the RPC client send a heart beat to server. 0 means disable, default is 10000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.connect.timeout</name>
+    <value>600000</value>
+    <description>
+      the timeout interval in millisecond when the RPC client is trying to setup the connection. default is 600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.connect.retry</name>
+    <value>10</value>
+    <description>
+      the max retry times if the RPC client fail to setup the connection to server. default is 10.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.read.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the RPC client is trying to read from server. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.write.timeout</name>
+    <value>3600000</value>
+    <description>
+      the timeout interval in millisecond when the RPC client is trying to write to server. default is 3600000.
+    </description>
+  </property>
+
+  <property>
+    <name>rpc.client.socket.linger.timeout</name>
+    <value>-1</value>
+    <description>
+      set value to socket SO_LINGER when connect to RPC server. -1 means default OS value. default is -1.
+    </description>
+  </property>
+
+  <property>
+    <name>yarn.client.failover.max.attempts</name>
+    <value>15</value>
+    <description>
+      if multiply resource managers are configured, it is the max retry times when the yarn client try to issue a RPC call. default is 15.
+    </description>
+  </property>
+</configuration>

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-deploy/puppet/modules/hawq/tests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/hawq/tests/init.pp b/bigtop-deploy/puppet/modules/hawq/tests/init.pp
new file mode 100644
index 0000000..37a5a08
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/tests/init.pp
@@ -0,0 +1,16 @@
+# 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.
+
+hawq::cluster_node { "test-hawq-node": }

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-packages/src/common/hadoop/init-hcfs.json
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hadoop/init-hcfs.json b/bigtop-packages/src/common/hadoop/init-hcfs.json
index 7f9b6cc..a9e0674 100644
--- a/bigtop-packages/src/common/hadoop/init-hcfs.json
+++ b/bigtop-packages/src/common/hadoop/init-hcfs.json
@@ -64,6 +64,7 @@
     ["/tmp/hadoop-yarn","777","mapred","mapred"],
     ["/var/log/hadoop-yarn/apps","1777","yarn","mapred"],
     ["/hbase",null,"hbase","hbase"],
+    ["/hawq_default","755","hawq","hawq"],
     ["/solr",null,"solr","solr"],
     ["/benchmarks","777",null,null],
     ["/user","755","HCFS_SUPER_USER",null],

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-packages/src/common/hawq/hawq.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/hawq.svc b/bigtop-packages/src/common/hawq/hawq.svc
index c3ddb27..735c82e 100644
--- a/bigtop-packages/src/common/hawq/hawq.svc
+++ b/bigtop-packages/src/common/hawq/hawq.svc
@@ -15,9 +15,10 @@
 
 DAEMON="hawq"
 DESC="hawq daemon"
-EXEC_PATH="\${HAWQ_HOME}/bin/include/service.sh"
+EXEC_PATH="\${HAWQ_HOME}/bin/hawq"
 SVC_USER="hawq"
 PIDFILE="\${HAWQ_PID_DIR}/${DAEMON}.pid"
+SVC_OPTS="-l \${HAWQ_LOG_DIR}"
 
 generate_start() {
 
@@ -33,7 +34,7 @@ start() {
         exit 0
     fi
 
-    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} start ${DAEMON}"
+    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} start master $SVC_OPTS"
 
     checkstatusofproc
     RETVAL=$?
@@ -50,7 +51,7 @@ cat <<'__EOT__'
 stop() {
     log_success_msg "Stopping $DESC (${DAEMON}): "
 
-    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} stop ${DAEMON}"
+    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} stop master $SVC_OPTS"
 
     sleep 3
     RETVAL=$?

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-packages/src/common/hawq/install_hawq.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/install_hawq.sh b/bigtop-packages/src/common/hawq/install_hawq.sh
index d1b7a59..af58ec6 100755
--- a/bigtop-packages/src/common/hawq/install_hawq.sh
+++ b/bigtop-packages/src/common/hawq/install_hawq.sh
@@ -130,6 +130,8 @@ sed -i -e 's#source %s/greenplum_path.sh" % hawq_home#source /etc/default/hawq#'
 ln -s $ETC_DIR/conf $PREFIX/$LIB_DIR/config
 ## HAWQ-422 configs are expected to sit in this hardcoded locations ;(
 ln -s $PREFIX/$LIB_DIR/config $PREFIX/$LIB_DIR/etc
+# HAWQ-421
+ln -s $ETC_DIR/default/hawq $PREFIX/$LIB_DIR/greenplum_path.sh
 
 wrapper=$PREFIX/usr/bin/hawq
 mkdir -p `dirname $wrapper`

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop-packages/src/deb/hawq/rules
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/rules b/bigtop-packages/src/deb/hawq/rules
index 21d9f00..7a7640b 100755
--- a/bigtop-packages/src/deb/hawq/rules
+++ b/bigtop-packages/src/deb/hawq/rules
@@ -30,7 +30,7 @@ hawq_pkg_name=hawq
 	dh $@
 
 override_dh_auto_build:
-	bash debian/do-component-build 
+	bash debian/do-component-build
 
 
 override_dh_auto_install:
@@ -45,7 +45,7 @@ override_dh_auto_install:
 	ln -s /var/run/${hawq_pkg_name} debian/tmp/usr/lib/${hawq_pkg_name}/pids
 	bash debian/init.d.tmpl debian/hawq.svc deb debian/${hawq_pkg_name}-service.init
 
-## Let's override the auto_configure and auto_clean to make sure existing 
+## Let's override the auto_configure and auto_clean to make sure existing
 ## top-level Makefile doesn't interfere with the package creation
 override_dh_auto_configure:
 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/421027ff/bigtop.bom
----------------------------------------------------------------------
diff --git a/bigtop.bom b/bigtop.bom
index 59fdfbc..f0066c3 100644
--- a/bigtop.bom
+++ b/bigtop.bom
@@ -90,7 +90,7 @@ bigtop {
     zookeeper:['hadoop', 'hbase'],
     hadoop:['ignite-hadoop', 'hbase', 'crunch', 'pig', 'hive', 'tez', 'sqoop', 'sqoop2',
       'oozie', 'mahout', 'flume', 'giraph', 'solr', 'crunch', 'spark',
-      'phoenix', 'tachyon', 'kafka', 'ycsb', 'kite', 'hama', 'zeppelin',
+      'phoenix', 'tachyon', 'kafka', 'ycsb', 'kite', 'hama', 'zeppelin', 'hawq',
       'tajo', 'apex'
     ],
     hbase:['phoenix','giraph','ycsb'],