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/03/18 03:22:43 UTC

[11/13] bigtop git commit: BIGTOP-2321. Add packaging code for HAWQ

BIGTOP-2321. Add packaging code for HAWQ


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

Branch: refs/heads/BIGTOP-2320
Commit: 8a946b7050db980cc36344fe4035bd7b32d11df4
Parents: 374cf9b
Author: Konstantin Boudnik <co...@apache.org>
Authored: Tue Feb 9 08:27:17 2016 -0800
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Thu Mar 17 18:52:42 2016 -0700

----------------------------------------------------------------------
 .../puppet/modules/hawq/manifests/init.pp       | 132 +++++++++++++
 .../src/common/hawq/do-component-build          |  73 +++++++
 bigtop-packages/src/common/hawq/hawq-master.svc |  64 +++++++
 .../src/common/hawq/hawq-segment.svc            |  65 +++++++
 bigtop-packages/src/common/hawq/hawq.1          |  59 ++++++
 bigtop-packages/src/common/hawq/hawq.default    |  30 +++
 bigtop-packages/src/common/hawq/install_hawq.sh | 157 ++++++++++++++++
 bigtop-packages/src/deb/hawq/compat             |   1 +
 bigtop-packages/src/deb/hawq/control            |  32 ++++
 bigtop-packages/src/deb/hawq/copyright          |  15 ++
 bigtop-packages/src/deb/hawq/hawq-doc.dirs      |   1 +
 bigtop-packages/src/deb/hawq/hawq-doc.install   |   1 +
 bigtop-packages/src/deb/hawq/hawq.dirs          |   4 +
 bigtop-packages/src/deb/hawq/hawq.install       |   3 +
 bigtop-packages/src/deb/hawq/hawq.manpages      |   1 +
 bigtop-packages/src/deb/hawq/hawq.postinst      |  47 +++++
 bigtop-packages/src/deb/hawq/hawq.preinst       |  61 ++++++
 bigtop-packages/src/deb/hawq/hawq.prerm         |  55 ++++++
 bigtop-packages/src/deb/hawq/rules              |  55 ++++++
 bigtop-packages/src/deb/hawq/source/format      |   1 +
 bigtop-packages/src/rpm/hawq/BUILD/.gitignore   |   0
 bigtop-packages/src/rpm/hawq/RPMS/.gitignore    |   0
 bigtop-packages/src/rpm/hawq/SOURCES/.gitignore |   0
 bigtop-packages/src/rpm/hawq/SPECS/hawq.spec    | 188 +++++++++++++++++++
 bigtop-packages/src/rpm/hawq/SRPMS/.gitignore   |   0
 bigtop.bom                                      |  10 +
 26 files changed, 1055 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/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..3d3d207
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/hawq/manifests/init.pp
@@ -0,0 +1,132 @@
+# 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')
+    $hawq_masterdata_dir = hiera('bigtop::hawq_master_datadir', "/var/run/hawq/work/hawq-data-directory/masterdd")
+    $hawq_segmentdata_dir = hiera('bigtop::hawq_segment_datadir', "/hawq/work/hawq-data-directory/segmentdd")
+
+    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"]],
+        owner   => 'hawq',
+        group   => 'hawq',
+        mode    => '0755',
+    }
+    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
+    }
+
+    file { "/etc/sysctl.conf":
+	# TODO overriding sysctl might be a somewhat dangerous, let's figure something better
+        content => template('hawq/sysctl.conf'),
+    }
+    exec { "sysctl reset":
+      path 	 => ['/usr/sbin'],
+      command	 => 'sysctl -p',
+      require	 => [ File['/etc/sysctl.conf'] ],
+    }
+
+    exec { "install pygresql modules1":
+      path 	 => ['/usr/bin'],
+      command	 => 'pip --retries=50 --timeout=300 install pg8000 simplejson unittest2 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 { "epel-release":
+      ensure 	 => latest,
+    }
+
+    package { "python-pip":
+      ensure 	 => latest,
+      require	 => [ Package['epel-release'] ],
+    }
+    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 master":
+      path 	 => ['/usr/bin', '/usr/lib/hawq/bin/lib'],
+      # Silly init will ask if I am really sure I want to init the cluster
+      command	 => 'bash -x run-init.sh master $GPHOME',
+      require	 => [ Package['hawq'], Exec ['sysctl reset', '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 master"] ],
+      subscribe => [ Package["hawq"], File["/etc/default/hawq", "/etc/hawq/conf/hawq-site.xml"] ]
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/common/hawq/do-component-build
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/do-component-build b/bigtop-packages/src/common/hawq/do-component-build
new file mode 100755
index 0000000..1c98dd4
--- /dev/null
+++ b/bigtop-packages/src/common/hawq/do-component-build
@@ -0,0 +1,73 @@
+#!/bin/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.
+
+set -ex
+
+. `dirname $0`/bigtop.bom
+CWD=`pwd`
+
+# Why? A single script with ksh, which isn't even ksh specific?
+sed -i -e 's#/bin/ksh#/bin/bash#' tools/bin/diskqual.ksh
+
+#docker run --rm=true -v `pwd`:/data -u root rlei/mydocker:latest /bin/sh -c "
+## build libyarn first
+cd depends/libyarn
+mkdir -p build
+cd build
+../bootstrap --prefix=${CWD}/libyarn
+make -j8
+make install
+cd ${CWD}
+ldconfig $CWD/libyarn/lib
+## libyarn should be available in the system at this point
+
+## build libhdfs3
+#wget https://github.com/PivotalRD/libhdfs3/archive/v2.2.31.tar.gz
+#tar zxf v2.2.31.tar.gz
+#cd libhdfs3-2.2.31
+#mkdir -p build
+#cd build
+#../bootstrap
+#make rpm-package
+## need to run diff target for deb and rpm. Figure out how
+
+cd ${CWD}
+# now something like
+# rpm -ivh rpms/RPMS/`uname -p`/*rpm
+## libhdfs3 libhdfs3-devel should be available now
+
+echo "Running the native build"
+date;
+./configure --prefix=${CWD}/hawq \
+    --with-libs=$CWD/libyarn/lib \
+    --with-includes=$CWD/libyarn/include
+if [ $? != 0 ]; then
+    echo HAWQ configure failed.
+    exit 1
+fi
+make -j8
+if [ $? != 0 ]; then
+    echo HAWQ compile failed.
+    exit 1
+fi
+make install
+if [ $? != 0 ]; then
+    echo HAWQ make install failed.
+    exit 1
+fi
+
+cd pxf
+./gradlew distTar

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/common/hawq/hawq-master.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/hawq-master.svc b/bigtop-packages/src/common/hawq/hawq-master.svc
new file mode 100644
index 0000000..4fa5097
--- /dev/null
+++ b/bigtop-packages/src/common/hawq/hawq-master.svc
@@ -0,0 +1,64 @@
+# 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.
+
+DAEMON="hawq-master"
+DESC="master daemon"
+EXEC_PATH="\${HAWQ_HOME}/bin/pg_ctL"
+SVC_USER="hawq"
+PIDFILE="\${HAWQ_PID_DIR}/${DAEMON}.pid"
+SVC_OPTS="-w -t \${HAWQ_TIMEOUT} -m \${HAWQ_SHUTDOWN_MODE} -D \${HAWQ_MASTERDATA_DIR} -l \${HAWQ_LOG_DIR}"
+
+generate_start() {
+
+cat <<'__EOT__'
+start() {
+    [ -x $EXEC_PATH ] || exit $ERROR_PROGRAM_NOT_INSTALLED
+    log_success_msg "Starting $DESC (${DAEMON}): "
+
+    checkstatusofproc
+    status=$?
+    if [ "$status" -eq "$STATUS_RUNNING" ]; then
+        log_success_msg "${DESC} is running"
+        exit 0
+    fi
+
+    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} start $SVC_OPTS"
+
+    checkstatusofproc
+    RETVAL=$?
+    [ $RETVAL -eq $STATUS_RUNNING ] && touch $LOCKFILE
+    return $RETVAL
+}
+__EOT__
+
+}
+
+generate_stop() {
+
+cat <<'__EOT__'
+stop() {
+    log_success_msg "Stopping $DESC (${DAEMON}): "
+
+    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} stop $SVC_OPTS"
+
+    sleep 3
+    RETVAL=$?
+
+    [ $RETVAL -eq $RETVAL_SUCCESS ] && rm -f $LOCKFILE $PIDFILE
+    return $RETVAL
+}
+__EOT__
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/common/hawq/hawq-segment.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/hawq-segment.svc b/bigtop-packages/src/common/hawq/hawq-segment.svc
new file mode 100644
index 0000000..b88dc12
--- /dev/null
+++ b/bigtop-packages/src/common/hawq/hawq-segment.svc
@@ -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.
+
+DAEMON="hawq-segment"
+DESC="segment daemon"
+EXEC_PATH="\${HAWQ_HOME}/bin/pg_ctl"
+SVC_USER="hawq"
+PIDFILE="\${HAWQ_PID_DIR}/${DAEMON}.pid"
+SVC_OPTS="-D \${HAWQ_SEGMENTDATA_DIR} -l \${HAWQ_LOG_DIR} -p \${HAWQ_SEGMENT_PORT} \
+	--silent-mode=true -M segment -i"
+
+generate_start() {
+
+cat <<'__EOT__'
+start() {
+    [ -x $EXEC_PATH ] || exit $ERROR_PROGRAM_NOT_INSTALLED
+    log_success_msg "Starting $DESC (${DAEMON}): "
+
+    checkstatusofproc
+    status=$?
+    if [ "$status" -eq "$STATUS_RUNNING" ]; then
+        log_success_msg "${DESC} is running"
+        exit 0
+    fi
+
+    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} start $SVC_OPTS"
+
+    checkstatusofproc
+    RETVAL=$?
+    [ $RETVAL -eq $STATUS_RUNNING ] && touch $LOCKFILE
+    return $RETVAL
+}
+__EOT__
+
+}
+
+generate_stop() {
+
+cat <<'__EOT__'
+stop() {
+    log_success_msg "Stopping $DESC (${DAEMON}): "
+
+    su -s /bin/bash ${SVC_USER} -c "${EXEC_PATH} stop $SVC_OPTS"
+
+    sleep 3
+    RETVAL=$?
+
+    [ $RETVAL -eq $RETVAL_SUCCESS ] && rm -f $LOCKFILE $PIDFILE
+    return $RETVAL
+}
+__EOT__
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/common/hawq/hawq.1
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/hawq.1 b/bigtop-packages/src/common/hawq/hawq.1
new file mode 100644
index 0000000..96c62c0
--- /dev/null
+++ b/bigtop-packages/src/common/hawq/hawq.1
@@ -0,0 +1,59 @@
+.\" 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.
+.\"
+.\" Process this file with
+.\" groff -man -Tascii ignite.1
+.\"
+.TH ignite 1 "Mar 2015 " Linux "User Manuals"
+
+.SH NAME
+Ignite \- Ignite is a filesystem and MR accelerator for Hadoop.
+
+.SH SYNOPSIS
+
+.B ignite
+\fICOMMAND\fR
+
+.SH DESCRIPTION
+
+Ignite is a filesystem and MR accelerator for Hadoop. Use it when you
+you need predictable file system caching, fast MR processing...
+
+Ignite is an open-source, distributed blah-blah
+
+For more information about Ignite, see http://ignite.apache.org/
+
+\fICOMMAND\fR may be one of the following:
+  something        do something
+  else             do something else
+
+Most commands print help when invoked w/o parameters or with --help.
+
+.SH ENVIRONMENT
+
+.IP JAVA_HOME
+The java implementation to use.  Overrides JAVA_HOME.
+
+.IP IGNITE_HOME
+Extra Java CLASSPATH entries.
+
+.IP IGNITE_CONF_DIR
+Alternate conf dir. Default is ${IGNITE_HOME}/conf.
+
+.IP HADOOP_HOME
+Optionally, the Hadoop home to run with.
+
+.SH COPYRIGHT
+2014 - 2015, The Apache Software Foundation

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/common/hawq/hawq.default
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/hawq/hawq.default b/bigtop-packages/src/common/hawq/hawq.default
new file mode 100644
index 0000000..f8cf6e1
--- /dev/null
+++ b/bigtop-packages/src/common/hawq/hawq.default
@@ -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.
+
+export HAWQ_HOME="/usr/lib/hawq"
+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/8a946b70/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
new file mode 100755
index 0000000..c12f924
--- /dev/null
+++ b/bigtop-packages/src/common/hawq/install_hawq.sh
@@ -0,0 +1,157 @@
+#!/bin/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.
+
+set -ex
+
+usage() {
+  echo "
+usage: $0 <options>
+  Required not-so-options:
+     --build-dir=DIR             path to hawq dist.dir
+     --prefix=PREFIX             path to install into
+
+  Optional options:
+     --doc-dir=DIR               path to install docs into [/usr/share/doc/hawq]
+     --lib-dir=DIR               path to install hawq home [/usr/lib/hawq]
+     --installed-lib-dir=DIR     path where lib-dir will end up on target system
+     --bin-dir=DIR               path to install bins [/usr/bin]
+     --examples-dir=DIR          path to install examples [doc-dir/examples]
+     ... [ see source for more similar options ]
+  "
+  exit 1
+}
+
+OPTS=$(getopt \
+  -n $0 \
+  -o '' \
+  -l 'prefix:' \
+  -l 'doc-dir:' \
+  -l 'lib-dir:' \
+  -l 'installed-lib-dir:' \
+  -l 'bin-dir:' \
+  -l 'examples-dir:' \
+  -l 'conf-dir:' \
+  -l 'build-dir:' -- "$@")
+
+if [ $? != 0 ] ; then
+    usage
+fi
+
+eval set -- "$OPTS"
+while true ; do
+    case "$1" in
+        --prefix)
+        PREFIX=$2 ; shift 2
+        ;;
+        --build-dir)
+        BUILD_DIR=$2 ; shift 2
+        ;;
+        --doc-dir)
+        DOC_DIR=$2 ; shift 2
+        ;;
+        --lib-dir)
+        LIB_DIR=$2 ; shift 2
+        ;;
+        --installed-lib-dir)
+        INSTALLED_LIB_DIR=$2 ; shift 2
+        ;;
+        --bin-dir)
+        BIN_DIR=$2 ; shift 2
+        ;;
+        --examples-dir)
+        EXAMPLES_DIR=$2 ; shift 2
+        ;;
+        --conf-dir)
+        CONF_DIR=$2 ; shift 2
+        ;;
+        --)
+        shift ; break
+        ;;
+        *)
+        echo "Unknown option: $1"
+        usage
+        exit 1
+        ;;
+    esac
+done
+
+for var in PREFIX BUILD_DIR ; do
+  if [ -z "$(eval "echo \$$var")" ]; then
+    echo Missing param: $var
+    usage
+  fi
+done
+
+MAN_DIR=${MAN_DIR:-/usr/share/man/man1}
+DOC_DIR=${DOC_DIR:-/usr/share/doc/hawq}
+LIB_DIR=${LIB_DIR:-/usr/lib/hawq}
+BIN_DIR=${BIN_DIR:-/usr/lib/hawq/bin}
+ETC_DIR=${ETC_DIR:-/etc/hawq}
+CONF_DIR=${CONF_DIR:-${ETC_DIR}/conf.dist}
+
+install -d -m 0755 $PREFIX/$LIB_DIR
+install -d -m 0755 $PREFIX/$LIB_DIR/lib
+install -d -m 0755 $PREFIX/$LIB_DIR/include
+install -d -m 0755 $PREFIX/$LIB_DIR/sbin
+install -d -m 0755 $PREFIX/$LIB_DIR/share
+install -d -m 0755 $PREFIX/$DOC_DIR
+install -d -m 0755 $PREFIX/$BIN_DIR
+install -d -m 0755 $PREFIX/$ETC_DIR
+install -d -m 0755 $PREFIX/$CONF_DIR
+install -d -m 0755 $PREFIX/$MAN_DIR
+install -d -m 0755 $PREFIX/var/run/hawq/work/
+
+UNZIP_DIR=hawq
+cp -ar $UNZIP_DIR/lib $PREFIX/$LIB_DIR
+cp -a $UNZIP_DIR/etc/* $PREFIX/$CONF_DIR
+cp -ra $UNZIP_DIR/bin/* $PREFIX/$BIN_DIR
+cp -ra $UNZIP_DIR/sbin $PREFIX/$LIB_DIR
+cp -ra $UNZIP_DIR/share $PREFIX/$LIB_DIR
+cp -ar $UNZIP_DIR/include $PREFIX/$LIB_DIR
+cp -ar $UNZIP_DIR/docs $PREFIX/$DOC_DIR
+
+cp -ar $UNZIP_DIR/../libyarn/* $PREFIX/$LIB_DIR
+
+#sed -i -e 's#source %s/greenplum_path.sh" % hawq_home#source /etc/default/hawq#' $PREFIX/$BIN_DIR/hawq
+ln -s $ETC_DIR/conf $PREFIX/$LIB_DIR/config
+## HAWQ-422 configs are expected to sit in this hardcoded locations ;(
+ln -s $ETC_DIR/conf $PREFIX/$LIB_DIR/etc
+# HAWQ-421
+ln -s /etc/default/hawq $PREFIX/$LIB_DIR/greenplum_path.sh
+
+## Put a wrapper to source the defaults and then run init script for particular object
+wrapper=$PREFIX/usr/lib/hawq/bin/lib/run-init.sh
+mkdir -p `dirname $wrapper`
+cat > $wrapper <<EOF
+#!/bin/bash
+
+BIGTOP_DEFAULTS_DIR=\${BIGTOP_DEFAULTS_DIR-/etc/default}
+[ -n "\${BIGTOP_DEFAULTS_DIR}" -a -r \${BIGTOP_DEFAULTS_DIR}/hawq ] && .  \${BIGTOP_DEFAULTS_DIR}/hawq
+
+bash -x /usr/lib/hawq/bin/lib/hawqinit.sh $* $HAWQ_HOME
+
+EOF
+chmod 755 $wrapper
+
+install -d -m 0755 $PREFIX/usr/bin
+
+install -d -m 0755 $PREFIX/$LIB_DIR/pxf
+UNZIP_DIR=pxf
+PXF_MODULES="pxf-hive pxf-hdfs pxf-service pxf-hbase pxf-api"
+for mod in $PXF_MODULES; do
+  tar zxf  $UNZIP_DIR/$mod/build/distributions/*tar.gz -C $PREFIX/$LIB_DIR/pxf/
+done

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/compat
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/compat b/bigtop-packages/src/deb/hawq/compat
new file mode 100644
index 0000000..ec63514
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/compat
@@ -0,0 +1 @@
+9

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/control
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/control b/bigtop-packages/src/deb/hawq/control
new file mode 100644
index 0000000..269a809
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/control
@@ -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.
+Source: hawq
+Section: misc
+Priority: extra
+Maintainer: Bigtop <de...@bigtop.apache.org>
+Build-Depends: debhelper (>= 9)
+Standards-Version: 3.9.4
+Homepage: http://hawq.incubator.apache.org/
+
+Package: hawq
+Architecture: all
+Depends: adduser, hadoop-hdfs, bigtop-utils (>= 1.1.0), bigtop-tomcat
+Description: HAWQ (incubating) is an advanced analytics MPP database
+
+Package: hawq-doc
+Architecture: all
+Section: doc
+Description: HAWQ Documentation
+ Documentation for Apache HAWQ (incubating)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/copyright
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/copyright b/bigtop-packages/src/deb/hawq/copyright
new file mode 100644
index 0000000..89a5120
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/copyright
@@ -0,0 +1,15 @@
+Format: http://dep.debian.net/deps/dep5
+Source: https://git-wip-us.apache.org/repos/asf/incubator-ignite
+Upstream-Name: none
+
+Files: *
+Copyright: 2014 - 2015, The Apache Software Foundation
+License: Apache-2.0
+
+Files debian/*
+Copyright: 2011, The Apache Software Foundation
+License: Apache-2.0
+
+License: Apache-2.0
+ On Debian systems, the complete text of the Apache 2.0 license
+ can be found in "/usr/share/common-licenses/Apache-2.0".

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq-doc.dirs
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq-doc.dirs b/bigtop-packages/src/deb/hawq/hawq-doc.dirs
new file mode 100644
index 0000000..61fe83a
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq-doc.dirs
@@ -0,0 +1 @@
+/usr/share/doc/hawq-docs

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq-doc.install
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq-doc.install b/bigtop-packages/src/deb/hawq/hawq-doc.install
new file mode 100644
index 0000000..99376a2
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq-doc.install
@@ -0,0 +1 @@
+/usr/share/doc/hawq-doc

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq.dirs
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq.dirs b/bigtop-packages/src/deb/hawq/hawq.dirs
new file mode 100644
index 0000000..3487c0d
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq.dirs
@@ -0,0 +1,4 @@
+/usr/bin
+/usr/lib/hawq
+/var/log/hawq
+/etc/default

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq.install
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq.install b/bigtop-packages/src/deb/hawq/hawq.install
new file mode 100644
index 0000000..59d4005
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq.install
@@ -0,0 +1,3 @@
+/usr/lib/hawq
+/usr/bin/hawq
+/etc/hawq

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq.manpages
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq.manpages b/bigtop-packages/src/deb/hawq/hawq.manpages
new file mode 100644
index 0000000..e5d94ec
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq.manpages
@@ -0,0 +1 @@
+debian/hawq.1

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq.postinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq.postinst b/bigtop-packages/src/deb/hawq/hawq.postinst
new file mode 100644
index 0000000..916a8a4
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq.postinst
@@ -0,0 +1,47 @@
+#!/bin/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.
+
+# postinst script for hawq
+
+hawq_pkg_name=hawq
+
+set -e
+
+case "$1" in
+    configure)
+        # Install config alternatives
+        update-alternatives --install /etc/${hawq_pkg_name}/conf ${hawq_pkg_name}-conf \
+        /etc/${hawq_pkg_name}/conf.dist 30
+
+        mkdir -p /var/log/${hawq_pkg_name} /var/run/${hawq_pkg_name} /var/run/${hawq_pkg_name}/work || :
+        chown hawq:hawq /var/log/${hawq_pkg_name} /var/run/${hawq_pkg_name}
+        chmod g+w /var/log/${hawq_pkg_name} /var/run/${hawq_pkg_name}
+        chown hawq:hawq /var/run/${hawq_pkg_name} /var/run/${hawq_pkg_name}/work
+        chmod 0755 /var/run/${hawq_pkg_name}
+        chmod 1777 /var/run/${hawq_pkg_name}/work
+    ;;
+
+    abort-upgrade|abort-remove|abort-deconfigure)
+    ;;
+
+    *)
+        echo "postinst called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+#DEBHELPER#

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq.preinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq.preinst b/bigtop-packages/src/deb/hawq/hawq.preinst
new file mode 100644
index 0000000..0b36a7a
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq.preinst
@@ -0,0 +1,61 @@
+#!/bin/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.
+
+# preinst script for hawq
+#
+# see: dh_installdeb(1)
+
+set -e
+
+# summary of how this script can be called:
+#        * <new-preinst> `install'
+#        * <new-preinst> `install' <old-version>
+#        * <new-preinst> `upgrade' <old-version>
+#        * <old-preinst> `abort-upgrade' <new-version>
+# for details, see http://www.debian.org/doc/debian-policy/ or
+# the debian-policy package
+
+
+case "$1" in
+    install|upgrade)
+        if ! getent passwd hawq >/dev/null; then
+                # Adding system user: hawq
+                adduser \
+                  --system \
+                  --group \
+                  --home /var/run/hawq \
+                  --gecos "HAWQ User" \
+                  --shell /bin/bash \
+                  hawq >/dev/null
+        fi
+    ;;
+
+    abort-upgrade)
+    ;;
+
+    *)
+        echo "preinst called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+# dh_installdeb will replace this with shell code automatically
+# generated by other debhelper scripts.
+
+#DEBHELPER#
+
+exit 0

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/hawq.prerm
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/hawq.prerm b/bigtop-packages/src/deb/hawq/hawq.prerm
new file mode 100644
index 0000000..0bcaf3c
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/hawq.prerm
@@ -0,0 +1,55 @@
+#!/bin/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.
+
+# prerm script for hawq
+#
+# see: dh_installdeb(1)
+
+set -e
+
+# summary of how this script can be called:
+#        * <prerm> `remove'
+#        * <old-prerm> `upgrade' <new-version>
+#        * <new-prerm> `failed-upgrade' <old-version>
+#        * <conflictor's-prerm> `remove' `in-favour' <package> <new-version>
+#        * <deconfigured's-prerm> `deconfigure' `in-favour'
+#          <package-being-installed> <version> `removing'
+#          <conflicting-package> <version>
+# for details, see http://www.debian.org/doc/debian-policy/ or
+# the debian-policy package
+
+
+case "$1" in
+    remove|upgrade|deconfigure)
+      update-alternatives --remove hawq-conf /etc/hawq/conf.dist || :
+    ;;
+
+    failed-upgrade)
+    ;;
+
+    *)
+        echo "prerm called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+# dh_installdeb will replace this with shell code automatically
+# generated by other debhelper scripts.
+
+#DEBHELPER#
+
+exit 0

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/rules
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/rules b/bigtop-packages/src/deb/hawq/rules
new file mode 100755
index 0000000..b4a3c9a
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/rules
@@ -0,0 +1,55 @@
+#!/usr/bin/make -f
+#
+# 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.
+#
+# -*- makefile -*-
+SHELL=/bin/bash
+
+# Uncomment this to turn on verbose mode.
+export DH_VERBOSE=1
+
+# This has to be exported to make some magic below work.
+export DH_OPTIONS
+
+hawq_pkg_name=hawq
+
+%:
+	dh $@
+
+override_dh_auto_build:
+	bash debian/do-component-build 
+
+
+override_dh_auto_install:
+	cp debian/hawq.default debian/${hawq_pkg_name}/etc/default/${hawq_pkg_name}
+	env HADOOP_VERSION=${HAWQ_BASE_VERSION} \
+	bash -x debian/install_hawq.sh \
+		--build-dir=target/bin \
+		--conf-dir=/etc/${hawq_pkg_name}/conf.dist \
+		--doc-dir=usr/share/doc/${hawq_pkg_name}-doc \
+		--prefix=debian/tmp
+	ln -s /var/log/${hawq_pkg_name} debian/tmp/usr/lib/${hawq_pkg_name}/logs
+	ln -s /var/run/${hawq_pkg_name} debian/tmp/usr/lib/${hawq_pkg_name}/pids
+	bash debian/init.d.tmpl debian/hawq-master.svc deb debian/${hawq_pkg_name}-master-service.init
+	bash debian/init.d.tmpl debian/hawq-segment.svc deb debian/${hawq_pkg_name}-segment-service.init
+
+## 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:
+
+override_dh_auto_clean:
+
+override_dh_auto_test:

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/deb/hawq/source/format
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/hawq/source/format b/bigtop-packages/src/deb/hawq/source/format
new file mode 100644
index 0000000..163aaf8
--- /dev/null
+++ b/bigtop-packages/src/deb/hawq/source/format
@@ -0,0 +1 @@
+3.0 (quilt)

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/rpm/hawq/BUILD/.gitignore
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/hawq/BUILD/.gitignore b/bigtop-packages/src/rpm/hawq/BUILD/.gitignore
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/rpm/hawq/RPMS/.gitignore
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/hawq/RPMS/.gitignore b/bigtop-packages/src/rpm/hawq/RPMS/.gitignore
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/rpm/hawq/SOURCES/.gitignore
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/hawq/SOURCES/.gitignore b/bigtop-packages/src/rpm/hawq/SOURCES/.gitignore
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/rpm/hawq/SPECS/hawq.spec
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/hawq/SPECS/hawq.spec b/bigtop-packages/src/rpm/hawq/SPECS/hawq.spec
new file mode 100644
index 0000000..0b17e84
--- /dev/null
+++ b/bigtop-packages/src/rpm/hawq/SPECS/hawq.spec
@@ -0,0 +1,188 @@
+# 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.
+%define etc_hawq_conf %{_sysconfdir}/%{name}/conf
+%define etc_hawq_conf_dist %{etc_hawq_conf}.dist
+%define hawq_home /usr/lib/%{name}
+%define bin_hawq %{hawq_home}/bin
+%define lib_hawq %{hawq_home}/lib
+%define conf_hawq %{hawq_home}/config
+%define logs_hawq %{hawq_home}/logs
+%define pids_hawq %{hawq_home}/pids
+%define man_dir %{_mandir}
+%define hawq_username hawq
+%define vcs_tag incubator-%{name}-HAWQ-307
+
+%define pxf %{hawq_home}/pxf
+
+%if  %{?suse_version:1}0
+
+# Only tested on openSUSE 11.4. le'ts update it for previous release when confirmed
+%if 0%{suse_version} > 1130
+%define suse_check \# Define an empty suse_check for compatibility with older sles
+%endif
+
+# SLES is more strict anc check all symlinks point to valid path
+# But we do point to a hadoop jar which is not there at build time
+# (but would be at install time).
+# Since our package build system does not handle dependencies,
+# these symlink checks are deactivated
+%define __os_install_post \
+    %{suse_check} ; \
+    /usr/lib/rpm/brp-compress ; \
+    %{nil}
+
+%define doc_hawq %{_docdir}/%{name}
+%global initd_dir %{_sysconfdir}/rc.d
+%define alternatives_cmd update-alternatives
+
+%else
+
+# CentOS 5 does not have any dist macro
+# So I will suppose anything that is not Mageia or a SUSE will be a RHEL/CentOS/Fedora
+%if %{!?mgaversion:1}0
+
+# FIXME: brp-repack-jars uses unzip to expand jar files
+# Unfortunately guice-2.0.jar pulled by ivy contains some files and directories without any read permission
+# and make whole process to fail.
+# So for now brp-repack-jars is being deactivated until this is fixed.
+# See BIGTOP-294
+#%define __os_install_post \
+#    %{_rpmconfigdir}/brp-compress ; \
+#    %{_rpmconfigdir}/brp-strip-static-archive %{__strip} ; \
+#   %{_rpmconfigdir}/brp-strip-comment-note %{__strip} %{__objdump} ; \
+#   /usr/lib/rpm/brp-python-bytecompile ; \
+#   %{nil}
+%endif
+
+%define doc_hawq %{_docdir}/%{name}-%{hawq_version}
+%global initd_dir %{_sysconfdir}/rc.d/init.d
+%define alternatives_cmd alternatives
+
+%endif
+
+
+Name: hawq
+Version: %{hawq_version}
+Release: %{hawq_release}
+Summary: Apache Hawq (incubating) is an advanced analytics MPP database
+URL: http://hawq.incubator.apache.org/
+Group: Development/Libraries
+Buildroot: %{_topdir}/INSTALL/%{name}-%{version}
+License: APL2
+Source0: %{vcs_tag}.tar.gz
+Source1: do-component-build
+Source2: install_hawq.sh
+Source3: hawq-master.svc
+Source4: init.d.tmpl
+Source5: hawq.default
+Source6: hawq-segment.svc
+Requires: coreutils, /usr/sbin/useradd, /sbin/chkconfig, /sbin/service
+Requires: hadoop-hdfs, bigtop-utils >= 1.0, bigtop-tomcat
+
+%if  0%{?mgaversion}
+Requires: bsh-utils
+%else
+Requires: sh-utils
+%endif
+
+%description
+Hawq is an open-source, distributed, MPP database engine
+
+%package doc
+Summary: Hawq Documentation
+Group: Documentation
+BuildArch: noarch
+
+%description doc
+Documentation for Hawq platform
+
+%prep
+%setup -n %{vcs_tag}
+
+%build
+bash %{SOURCE1}
+
+%install
+%__rm -rf $RPM_BUILD_ROOT
+env HAWQ_VERSION=%{version} bash %{SOURCE2} \
+  --build-dir=target/bin \
+  --doc-dir=%{doc_hawq} \
+  --conf-dir=%{etc_hawq_conf_dist} \
+	--prefix=$RPM_BUILD_ROOT
+
+%__install -d -m 0755 $RPM_BUILD_ROOT/%{initd_dir}/
+
+%__install -d -m 0755 $RPM_BUILD_ROOT/etc/default/
+%__install -m 0644 %{SOURCE5} $RPM_BUILD_ROOT/etc/default/%{name}
+
+%__install -d  -m 0755  %{buildroot}/%{_localstatedir}/log/%{name}
+
+ln -s %{_localstatedir}/log/%{name} %{buildroot}/%{logs_hawq}
+
+%__install -d  -m 0755  %{buildroot}/%{_localstatedir}/run/%{name}
+ln -s %{_localstatedir}/run/%{name} %{buildroot}/%{pids_hawq}
+
+master_init_file=$RPM_BUILD_ROOT/%{initd_dir}/%{name}
+segment_init_file=$RPM_BUILD_ROOT/%{initd_dir}/%{name}-segment
+bash %{SOURCE4} ${RPM_SOURCE_DIR}/%{SOURCE3} rpm $master_init_file
+bash %{SOURCE4} ${RPM_SOURCE_DIR}/%{SOURCE6} rpm $segment_init_file
+chmod 755 $master_init_file $segment_init_file
+
+%__install -d -m 0755 $RPM_BUILD_ROOT/usr/bin
+
+%pre
+getent group hawq 2>/dev/null >/dev/null || /usr/sbin/groupadd -r hawq
+getent passwd hawq 2>&1 > /dev/null || /usr/sbin/useradd -c "hawq" -s /sbin/nologin -g hawq -r -d /var/run/hawq hawq 2> /dev/null || :
+
+%post
+%{alternatives_cmd} --install %{etc_hawq_conf} %{name}-conf %{etc_hawq_conf_dist} 30
+chkconfig --add %{name}
+
+%preun
+if [ "$1" = 0 ]; then
+        %{alternatives_cmd} --remove %{name}-conf %{etc_hawq_conf_dist} || :
+fi
+
+if [ $1 = 0 ] ; then
+        service %{name} stop > /dev/null 2>&1
+        chkconfig --del %{name}
+fi
+
+%postun
+if [ $1 -ge 1 ]; then
+        service %{name} condrestart >/dev/null 2>&1
+fi
+
+
+#######################
+#### FILES SECTION ####
+#######################
+%files
+%defattr(-,hawq,hawq)
+%attr(0755,root,root)/%{initd_dir}/%{name}
+%attr(0755,root,root)/%{initd_dir}/%{name}-segment
+%dir %{_localstatedir}/log/%{name}
+%dir %{_localstatedir}/run/%{name}
+
+%defattr(-,root,root)
+%config(noreplace) %{_sysconfdir}/default/%{name}
+%{hawq_home}
+%config(noreplace) %{etc_hawq_conf_dist}
+
+%files doc
+%defattr(-,root,root)
+%doc %{doc_hawq}
+
+%clean

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop-packages/src/rpm/hawq/SRPMS/.gitignore
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/hawq/SRPMS/.gitignore b/bigtop-packages/src/rpm/hawq/SRPMS/.gitignore
new file mode 100644
index 0000000..e69de29

http://git-wip-us.apache.org/repos/asf/bigtop/blob/8a946b70/bigtop.bom
----------------------------------------------------------------------
diff --git a/bigtop.bom b/bigtop.bom
index 2a709bf..f8a6878 100644
--- a/bigtop.bom
+++ b/bigtop.bom
@@ -431,5 +431,15 @@ bigtop {
       url     { site = "https://github.com/quantcast/qfs/archive"
                 archive = site }
     }
+    'hawq' {
+      name     = 'hawq'
+      relNotes = 'Apache HAWQ (incubating)'
+      version { base = '2.0.0'; pkg = base; release = 1 }
+      tarball { source      = "HAWQ-307.tar.gz"
+                destination = "incubator-${name}-HAWQ-307.tar.gz" }
+      url     { download_path = "/apache/incubator-$name/archive"
+                site = "https://github.com/${download_path}"
+                archive = site }
+    }
   }
 }