You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by ja...@apache.org on 2014/11/25 16:34:24 UTC

bigtop git commit: BIGTOP-1497. Add tachyon into bigtop.

Repository: bigtop
Updated Branches:
  refs/heads/master d3bc6b24c -> 57f590052


BIGTOP-1497. Add tachyon into bigtop.

Signed-off-by: jayunit100 <ja...@apache.org>


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

Branch: refs/heads/master
Commit: 57f5900520d6208fd548cc08856514587294a4cd
Parents: d3bc6b2
Author: rootfs <hc...@redhat.com>
Authored: Mon Nov 3 14:43:29 2014 -0500
Committer: jayunit100 <ja...@apache.org>
Committed: Tue Nov 25 10:27:15 2014 -0500

----------------------------------------------------------------------
 bigtop-deploy/puppet/config/site.csv            |   5 +
 bigtop-deploy/puppet/manifests/cluster.pp       |   7 +
 .../puppet/modules/tachyon/manifests/init.pp    |  45 +++
 .../src/common/tachyon/do-component-build       |  21 ++
 bigtop-packages/src/common/tachyon/init.d.tmpl  | 313 +++++++++++++++++++
 .../src/common/tachyon/install_tachyon.sh       | 154 +++++++++
 .../src/common/tachyon/tachyon-master.svc       |  70 +++++
 .../src/common/tachyon/tachyon-worker.svc       |  73 +++++
 bigtop-packages/src/deb/tachyon/changelog       |   1 +
 bigtop-packages/src/deb/tachyon/compat          |   1 +
 bigtop-packages/src/deb/tachyon/control         |  37 +++
 bigtop-packages/src/deb/tachyon/copyright       |  15 +
 bigtop-packages/src/deb/tachyon/rules           |  40 +++
 bigtop-packages/src/deb/tachyon/source/format   |   1 +
 bigtop-packages/src/deb/tachyon/tachyon.dirs    |   3 +
 bigtop-packages/src/deb/tachyon/tachyon.install |   3 +
 .../src/deb/tachyon/tachyon.postinst            |  32 ++
 bigtop-packages/src/deb/tachyon/tachyon.prerm   |  37 +++
 .../src/rpm/tachyon/SPECS/tachyon.spec          | 124 ++++++++
 bigtop.mk                                       |  13 +
 20 files changed, 995 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-deploy/puppet/config/site.csv
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/config/site.csv b/bigtop-deploy/puppet/config/site.csv
new file mode 100644
index 0000000..2b01b13
--- /dev/null
+++ b/bigtop-deploy/puppet/config/site.csv
@@ -0,0 +1,5 @@
+hadoop_head_node,bigtop1.vagrant
+hadoop_storage_dirs,/data/1,/data/2
+bigtop_yumrepo_uri,http://bigtop01.cloudera.org:8080/view/Releases/job/Bigtop-0.8.0/label=centos6/6/artifact/output/
+jdk_package_name,java-1.7.0-openjdk-devel.x86_64
+components,tachyon,pig

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-deploy/puppet/manifests/cluster.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/manifests/cluster.pp b/bigtop-deploy/puppet/manifests/cluster.pp
index 0f9baef..41c7089 100644
--- a/bigtop-deploy/puppet/manifests/cluster.pp
+++ b/bigtop-deploy/puppet/manifests/cluster.pp
@@ -290,6 +290,13 @@ if ($hadoop_security_authentication == "kerberos") {
   }
   }
 
+  if ($components[0] == undef or "tachyon" in $components) {
+   tachyon::master { "tachyon master":
+       master_host    => $tachyon_master_host,
+       master_port    => $tachyon_master_port
+   }
+  }
+
   if ($components[0] == undef or "hbase" in $components) {
     hadoop-zookeeper::server { "zookeeper":
           myid => "0",

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-deploy/puppet/modules/tachyon/manifests/init.pp
----------------------------------------------------------------------
diff --git a/bigtop-deploy/puppet/modules/tachyon/manifests/init.pp b/bigtop-deploy/puppet/modules/tachyon/manifests/init.pp
new file mode 100644
index 0000000..8af9b68
--- /dev/null
+++ b/bigtop-deploy/puppet/modules/tachyon/manifests/init.pp
@@ -0,0 +1,45 @@
+# 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 tachyon {
+  class common {
+    package { "tachyon":
+      ensure => latest,
+    }
+  }
+
+  define master($master_host, $master_port) {
+    include common
+
+    if ( $fqdn == $master_host ) {
+      service { "tachyon-master":
+        ensure => running,
+        require => [ Package["tachyon"] ],
+        hasrestart => true,
+        hasstatus => true,
+      }
+    }
+  }
+
+  define worker($master_host, $master_port) {
+    include common
+
+   if ( $fqdn == $master_host ) {
+      # We want master to run first in all cases
+      Service["tachyon-master"] ~> Service["tachyon-worker"]
+    }
+    service { "tachyon-worker":
+      ensure => running,
+      hasrestart => true,
+      hasstatus => true,
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/common/tachyon/do-component-build
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/tachyon/do-component-build b/bigtop-packages/src/common/tachyon/do-component-build
new file mode 100644
index 0000000..9c50834
--- /dev/null
+++ b/bigtop-packages/src/common/tachyon/do-component-build
@@ -0,0 +1,21 @@
+#!/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
+
+mvn -q clean install -DskipTests -Dhadoop.version=$HADOOP_VERSION "$@"

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/common/tachyon/init.d.tmpl
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/tachyon/init.d.tmpl b/bigtop-packages/src/common/tachyon/init.d.tmpl
new file mode 100755
index 0000000..b8de352
--- /dev/null
+++ b/bigtop-packages/src/common/tachyon/init.d.tmpl
@@ -0,0 +1,313 @@
+#!/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.
+
+# This is a poor man's templating engine for generating init.d scripts to
+# support all the Apache services that Bigtop distro has. An actual init.d
+# script gets generate via running this script under bash and giving it
+# a mandatory argument of a file containing the configuration for the service.
+# The argument file should be a valid piece of bash code since it gets directly
+# source into this template. E.g.
+#    $ bash ./init.d.tmpl hadoop-hdfs-namenode.svc rpm hadoop-hdfs-namenode
+#
+# You must declare the following in your .svc configuration file:
+#     DAEMON="name of the resulting init.d script"
+#     DESC="Free form human readable description of the service"
+#     EXEC_PATH="path to the upstream daemon management script"
+#     SVC_USER="user to run this service as"
+#     DAEMON_FLAGS="flags to be passed to the $EXEC_PATH"
+#     WORKING_DIR="working directory to cd into before starting the daemon"
+#     CONF_DIR="path to the configuration directory"
+#     PIDFILE="file holding a PID of the running daemon"
+#     LOCKFILE="file signifying the service lock"
+#
+#     CHKCONFIG="chkconfig(8) registration signature"
+#     INIT_DEFAULT_START="run levels to use"
+#     INIT_DEFAULT_STOP="run levels not to use ;-)"
+#
+# You can, also, override parts of the generated init.d script by providing
+# function definitions for: generate_start, generate_stop, generate_functions, and generate_extra_commands.
+# See the default implemenations below and feel free to customize. Also look
+# for exising .svc files in common to see how different services are tweaking
+# the defaults.
+#
+# Of course, if this whole templating thing grows too big we might need to
+# consider a real templating engine (I have played with m4, but it seems
+# qutie brittle when used for manipulating pieces of the shell code -- think
+# $0 the like).
+
+if [ $# -lt 3 ] ; then
+  echo "Usage: ${BASH_SOURCE-0} service.definition.svc rpm|deb init.d.script"
+  exit 1
+fi
+
+# Override anything necessary for supported systems
+case "$2" in
+  "deb" )
+    # The defaults will work well on Debian-based systems
+    ;;
+  "rpm" )
+    # On RedHat, SuSE and Mageia run-level 2 is networkless, hence excluding it
+    CHKCONFIG="345 85 15"
+    INIT_DEFAULT_START="3 4 5"
+    INIT_DEFAULT_STOP="0 1 2 6"
+    ;;
+  "*" )
+    ;;
+esac
+
+# Create the init script, make it executable, and send stdout to it
+mkdir -p `dirname $3`
+touch $3
+chmod 0755 $3
+exec > $3
+
+generate_start() {
+
+cat <<'__EOT__'
+
+start() {
+  [ -x $EXEC_PATH ] || exit $ERROR_PROGRAM_NOT_INSTALLED
+  [ -d $CONF_DIR ] || exit $ERROR_PROGRAM_NOT_CONFIGURED
+
+  su -s /bin/bash $SVC_USER -c "cd $WORKING_DIR && $EXEC_PATH --config '$CONF_DIR' start $DAEMON_FLAGS"
+
+  # Some processes are slow to start
+  sleep $SLEEP_TIME
+  checkstatusofproc
+  RETVAL=$?
+
+  if [ $RETVAL -eq $STATUS_RUNNING ]; then
+    touch $LOCKFILE
+    log_success_msg "Started ${DESC}: "
+  else
+	log_failure_msg "Failed to start ${DESC}. Return value: $RETVAL"
+  fi
+  return $RETVAL
+}
+
+__EOT__
+
+}
+
+generate_stop() {
+
+cat <<'__EOT__'
+
+stop() {
+  start_daemon $EXEC_PATH --config "$CONF_DIR" stop $DAEMON_FLAGS
+  RETVAL=$?
+
+  if [ $RETVAL -eq $RETVAL_SUCCESS ]; then
+	 log_success_msg "Stopped ${DESC}: "
+	 rm -f $LOCKFILE $PIDFILE
+ else
+	 log_failure_msg "Failed to stop ${DESC}. Return value: $RETVAL."
+ fi
+}
+
+__EOT__
+
+}
+
+generate_functions() {
+    :
+}
+
+generate_extra_commands() {
+
+cat <<'__EOT__'
+    *)
+      echo $"Usage: $0 {start|stop|status|restart|try-restart|condrestart}"
+      exit 1
+__EOT__
+
+}
+
+###################################################################
+# Some reasonable defaults for the run-level settings, these are
+# tweaked on a per-OS basis from the SPEC/rules files that call us
+# and can be further tweaked inside of individual .svc templates
+CHKCONFIG=${CHKCONFIG:-"2345 85 15"}
+INIT_DEFAULT_START=${INIT_DEFAULT_START:-"2 3 4 5"}
+INIT_DEFAULT_STOP=${INIT_DEFAULT_STOP:-"0 1 6"}
+
+###################################################################
+# NOTE how we are sourcing the argument here so that a user-defined
+# settings have a chance to override the default values for
+# generate_start, generate_stop, generate_functions and generate_extra_commands. If you
+# ever want to make this template even more flexible -- define the
+# default values above this line
+. $1
+
+cat <<__EOT__
+#!/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.
+#
+# Starts a $DESC
+#
+# chkconfig: $CHKCONFIG
+# description: $DESC
+#
+### BEGIN INIT INFO
+# Provides:          $DAEMON
+# Short-Description: $DESC
+# Default-Start:     $INIT_DEFAULT_START
+# Default-Stop:      $INIT_DEFAULT_STOP
+# Required-Start:    \$syslog \$remote_fs
+# Required-Stop:     \$syslog \$remote_fs
+# Should-Start:
+# Should-Stop:
+### END INIT INFO
+
+. /lib/lsb/init-functions
+BIGTOP_DEFAULTS_DIR=\${BIGTOP_DEFAULTS_DIR-/etc/default}
+[ -n "\${BIGTOP_DEFAULTS_DIR}" -a -r \${BIGTOP_DEFAULTS_DIR}/hadoop ] && . \${BIGTOP_DEFAULTS_DIR}/hadoop
+[ -n "\${BIGTOP_DEFAULTS_DIR}" -a -r \${BIGTOP_DEFAULTS_DIR}/$DAEMON ] && . \${BIGTOP_DEFAULTS_DIR}/$DAEMON
+
+# Autodetect JAVA_HOME if not defined
+. /usr/lib/bigtop-utils/bigtop-detect-javahome
+
+RETVAL_SUCCESS=0
+
+STATUS_RUNNING=0
+STATUS_DEAD=1
+STATUS_DEAD_AND_LOCK=2
+STATUS_NOT_RUNNING=3
+STATUS_OTHER_ERROR=102
+
+
+ERROR_PROGRAM_NOT_INSTALLED=5
+ERROR_PROGRAM_NOT_CONFIGURED=6
+
+
+RETVAL=0
+SLEEP_TIME=5
+PROC_NAME="java"
+
+DAEMON="$DAEMON"
+DESC="$DESC"
+EXEC_PATH="$EXEC_PATH"
+SVC_USER="$SVC_USER"
+DAEMON_FLAGS="$DAEMON_FLAGS"
+CONF_DIR="$CONF_DIR"
+PIDFILE="$PIDFILE"
+LOCKDIR="/var/lock/subsys"
+LOCKFILE="\$LOCKDIR/$DAEMON"
+WORKING_DIR="${WORKING_DIR:-~/}"
+
+install -d -m 0755 -o $SVC_USER -g $SVC_USER $(dirname $PIDFILE) 1>/dev/null 2>&1 || :
+[ -d "\$LOCKDIR" ] || install -d -m 0755 \$LOCKDIR 1>/dev/null 2>&1 || :
+__EOT__
+
+generate_functions
+generate_start
+generate_stop
+
+cat <<'__EOT__'
+restart() {
+  stop
+  start
+}
+
+checkstatusofproc(){
+  pidofproc -p $PIDFILE $PROC_NAME > /dev/null
+}
+
+checkstatus(){
+  checkstatusofproc
+  status=$?
+
+  case "$status" in
+    $STATUS_RUNNING)
+      log_success_msg "${DESC} is running"
+      ;;
+    $STATUS_DEAD)
+      log_failure_msg "${DESC} is dead and pid file exists"
+      ;;
+    $STATUS_DEAD_AND_LOCK)
+      log_failure_msg "${DESC} is dead and lock file exists"
+      ;;
+    $STATUS_NOT_RUNNING)
+      log_failure_msg "${DESC} is not running"
+      ;;
+    *)
+      log_failure_msg "${DESC} status is unknown"
+      ;;
+  esac
+  return $status
+}
+
+condrestart(){
+  [ -e $LOCKFILE ] && restart || :
+}
+
+check_for_root() {
+  if [ $(id -ur) -ne 0 ]; then
+    echo 'Error: root user required'
+    echo
+    exit 1
+  fi
+}
+
+service() {
+  case "$1" in
+    start)
+      check_for_root
+      start
+      ;;
+    stop)
+      check_for_root
+      stop
+      ;;
+    status)
+      checkstatus
+      RETVAL=$?
+      ;;
+    restart)
+      check_for_root
+      restart
+      ;;
+    condrestart|try-restart)
+      check_for_root
+      condrestart
+      ;;
+__EOT__
+
+generate_extra_commands
+
+cat <<'__EOT__'
+  esac
+}
+
+service "$1"
+
+exit $RETVAL
+__EOT__

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/common/tachyon/install_tachyon.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/tachyon/install_tachyon.sh b/bigtop-packages/src/common/tachyon/install_tachyon.sh
new file mode 100644
index 0000000..ee4faf8
--- /dev/null
+++ b/bigtop-packages/src/common/tachyon/install_tachyon.sh
@@ -0,0 +1,154 @@
+#!/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 Tachyon dist.dir
+     --prefix=PREFIX             path to install into
+
+  Optional options:
+     --bin-dir=DIR               path to install bin
+     --data-dir=DIR              path to install Tachyon webapp
+     ... [ see source for more similar options ]
+  "
+  exit 1
+}
+
+OPTS=$(getopt \
+  -n $0 \
+  -o '' \
+  -l 'prefix:' \
+  -l 'bin-dir:' \
+  -l 'libexec-dir:' \
+  -l 'var-dir:' \
+  -l 'lib-dir:' \
+  -l 'data-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
+        ;;
+        --libexec-dir)
+        LIBEXEC_DIR=$2 ; shift 2
+        ;;
+        --lib-dir)
+        LIB_DIR=$2 ; shift 2
+        ;;
+        --bin-dir)
+        BIN_DIR=$2 ; shift 2
+        ;;
+        --var-dir)
+        VAR_DIR=$2 ; shift 2
+        ;;
+        --data-dir)
+        DATA_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
+
+LIB_DIR=${LIB_DIR:-/usr/lib/tachyon}
+LIBEXEC_DIR=${INSTALLED_LIB_DIR:-/usr/libexec}
+BIN_DIR=${BIN_DIR:-/usr/bin}
+
+install -d -m 0755 $PREFIX/$LIB_DIR
+install -d -m 0755 $PREFIX/$LIB_DIR/bin
+install -d -m 0755 $PREFIX/$LIB_DIR/libexec
+install -d -m 0755 $PREFIX/$LIB_DIR/lib
+install -d -m 0755 $PREFIX/$DATA_DIR
+install -d -m 0755 $PREFIX/$DATA_DIR/tachyon
+install -d -m 0755 $PREFIX/$DATA_DIR/tachyon/web
+install -d -m 0755 $PREFIX/etc
+install -d -m 0755 $PREFIX/etc/tachyon
+install -d -m 0755 $PREFIX/$VAR_DIR/log/tachyon
+install -d -m 0755 $PREFIX/$VAR_DIR/lib/tachyon/journal
+install -d -m 0755 $PREFIX/$VAR_DIR/run/tachyon
+
+#cp -ra ${BUILD_DIR}/lib/* $PREFIX/${LIB_DIR}/lib/
+cp client/target/tachyon-client*.jar core/target/tachyon*.jar $PREFIX/$LIB_DIR
+cp -a bin/* $PREFIX/${LIB_DIR}/bin
+cp -a libexec/* $PREFIX/${LIB_DIR}/libexec
+cp -rf core/src/main/webapp $PREFIX/$DATA_DIR/tachyon/web
+
+# Copy in the configuration files
+install -m 0644 conf/log4j.properties conf/slaves  $PREFIX/etc/tachyon
+cp conf/tachyon-env.sh.template  $PREFIX/etc/tachyon/tachyon-env.sh
+
+# Copy in the /usr/bin/tachyon wrapper
+install -d -m 0755 $PREFIX/$BIN_DIR
+
+# Prefix is correct at time of install,
+# but we dont want to escape it before that point.
+cat > $PREFIX/$BIN_DIR/tachyon <<EOF
+#!/bin/bash
+
+# Autodetect JAVA_HOME if not defined
+. /usr/lib/bigtop-utils/bigtop-detect-javahome
+# Lib dir => ${LIB_DIR}
+#!/usr/bin/env bash
+exec ${LIB_DIR}/bin/tachyon "\$@"
+EOF
+chmod 755 $PREFIX/$BIN_DIR/tachyon
+
+cat >$PREFIX/$LIB_DIR/libexec/tachyon-layout.sh <<EOF
+#!/usr/bin/env bash
+
+export TACHYON_SYSTEM_INSTALLATION="TRUE"
+export TACHYON_PREFIX="$LIB_DIR"
+export TACHYON_HOME="/var/lib/tachyon"
+export TACHYON_CONF_DIR="/etc/tachyon"
+export TACHYON_LOGS_DIR="/var/log/tachyon"
+export TACHYON_DATA_DIR="/var/run/tachyon"
+export TACHYON_JAR="\`find $LIB_DIR/ -name tachyon*dependencies.jar|grep -v client\`"
+
+# find JAVA_HOME
+. /usr/lib/bigtop-utils/bigtop-detect-javahome
+
+if [ -z "JAVA_HOME" ]; then
+  export JAVA="/usr/bin/java"
+else
+  export JAVA="\$JAVA_HOME/bin/java"
+fi
+EOF

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/common/tachyon/tachyon-master.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/tachyon/tachyon-master.svc b/bigtop-packages/src/common/tachyon/tachyon-master.svc
new file mode 100644
index 0000000..e5b8fdd
--- /dev/null
+++ b/bigtop-packages/src/common/tachyon/tachyon-master.svc
@@ -0,0 +1,70 @@
+# 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.
+
+TYPE="master"
+DAEMON="tachyon-${TYPE}"
+DESC="Tachyon ${TYPE}"
+EXEC_PATH="/usr/lib/tachyon/bin/tachyon-start.sh"
+WORKING_DIR="/var/lib/tachyon"
+DAEMON_FLAGS="master Mount"
+CONF_DIR="/etc/tachyon/conf"
+PIDFILE="/var/run/tachyon/${DAEMON}.pid"
+
+generate_start() {
+
+cat <<'__EOT__'
+start() {
+    [ -x $EXE_FILE ] || 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
+
+    LOG_FILE=/var/log/tachyon/${DAEMON}.out
+
+    /bin/bash -c "nohup nice -n 0 \
+        ${EXEC_PATH} ${DAEMON_FLAGS} \
+        > $LOG_FILE 2>&1 & "'echo $!' > "$PIDFILE"
+
+    sleep 3
+
+    checkstatusofproc
+    RETVAL=$?
+    [ $RETVAL -eq $STATUS_RUNNING ] && touch $LOCKFILE
+    return $RETVAL
+}
+__EOT__
+
+}
+
+generate_stop() {
+
+cat <<'__EOT__'
+stop() {
+    log_success_msg "Stopping $DESC (${DAEMON}): "
+    killproc -p $PIDFILE java
+    RETVAL=$?
+    /usr/lib/tachyon/bin/tachyon-stop.sh 2>&1 > /dev/null
+
+    [ $RETVAL -eq $RETVAL_SUCCESS ] && rm -f $LOCKFILE $PIDFILE
+    return $RETVAL
+}
+__EOT__
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/common/tachyon/tachyon-worker.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/tachyon/tachyon-worker.svc b/bigtop-packages/src/common/tachyon/tachyon-worker.svc
new file mode 100644
index 0000000..2183926
--- /dev/null
+++ b/bigtop-packages/src/common/tachyon/tachyon-worker.svc
@@ -0,0 +1,73 @@
+# 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.
+
+TYPE="worker"
+DAEMON="tachyon-${TYPE}"
+DESC="Tachyon ${TYPE}"
+EXEC_PATH="/usr/lib/tachyon/bin/tachyon-start.sh"
+WORKING_DIR="/var/lib/tachyon"
+DAEMON_FLAGS="worker Mount"
+CONF_DIR="/etc/tachyon/conf"
+PIDFILE="/var/run/tachyon/${DAEMON}.pid"
+
+generate_start() {
+
+cat <<'__EOT__'
+start() {
+    [ -x $EXE_FILE ] || 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
+
+    LOG_FILE=/var/log/tachyon/${DAEMON}.out
+
+    if [ -f $CONF_DIR/tachyon-env.sh ]; then
+        . $CONF_DIR/tachyon-env.sh
+    fi
+
+    /bin/bash -c "nohup nice -n 0 \
+        ${EXEC_PATH} ${DAEMON_FLAGS} \
+        > $LOG_FILE 2>&1 & "'echo $!' > "$PIDFILE"
+
+    sleep 3
+
+    checkstatusofproc
+    RETVAL=$?
+    [ $RETVAL -eq $STATUS_RUNNING ] && touch $LOCKFILE
+    return $RETVAL
+}
+__EOT__
+
+}
+
+generate_stop() {
+
+cat <<'__EOT__'
+stop() {
+    log_success_msg "Stopping $DESC (${DAEMON}): "
+    killproc -p $PIDFILE java
+    RETVAL=$?
+
+    [ $RETVAL -eq $RETVAL_SUCCESS ] && rm -f $LOCKFILE $PIDFILE
+    return $RETVAL
+}
+__EOT__
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/changelog
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/changelog b/bigtop-packages/src/deb/tachyon/changelog
new file mode 100644
index 0000000..d4858cd
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/changelog
@@ -0,0 +1 @@
+--- This is auto-generated

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/compat
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/compat b/bigtop-packages/src/deb/tachyon/compat
new file mode 100644
index 0000000..7f8f011
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/compat
@@ -0,0 +1 @@
+7

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/control
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/control b/bigtop-packages/src/deb/tachyon/control
new file mode 100644
index 0000000..96e4677
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/control
@@ -0,0 +1,37 @@
+# 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: tachyon
+Section: misc
+Priority: extra
+Maintainer: Huamin Chen <hc...@redhat.com>
+Build-Depends: debhelper (>= 7.0.50~)
+Standards-Version: 0.5.0
+Homepage: http://tachyon-project.org
+
+Package:  tachyon
+Architecture: all
+Depends: bigtop-utils (>= 0.7)
+Description: Reliable file sharing at memory speed across cluster frameworks
+ Tachyon is a fault tolerant distributed file system
+ enabling reliable file sharing at memory-speed
+ across cluster frameworks, such as Spark and MapReduce.
+ It achieves high performance by leveraging lineage
+ information and using memory aggressively.
+ Tachyon caches working set files in memory, and
+ enables different jobs/queries and frameworks to
+ access cached files at memory speed. Thus, Tachyon
+ avoids going to disk to load data-sets that
+ are frequently read.

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/copyright
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/copyright b/bigtop-packages/src/deb/tachyon/copyright
new file mode 100644
index 0000000..1cbe16f
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/copyright
@@ -0,0 +1,15 @@
+Format: http://dep.debian.net/deps/dep5
+Source: http://tachyon-project.org/
+Upstream-Name: Tachyon Project
+
+Files: *
+Copyright: 2010-2011, 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/57f59005/bigtop-packages/src/deb/tachyon/rules
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/rules b/bigtop-packages/src/deb/tachyon/rules
new file mode 100755
index 0000000..2b0cb04
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/rules
@@ -0,0 +1,40 @@
+#!/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 -*-
+
+# 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
+
+%:
+	dh $@
+
+override_dh_auto_build:
+	# we'll just use the build from the tarball.
+	bash debian/do-component-build -Dmaven.repo.local=${HOME}/.m2/repository
+
+override_dh_auto_install:
+	bash -x debian/install_tachyon.sh \
+    --build-dir=build \
+    --bin-dir=/usr/bin \
+    --data-dir=/usr/share \
+    --libexec-dir=/usr/lib/tachyon/libexec \
+    --var-dir= /var/ \
+    --prefix=debian/tmp

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

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/tachyon.dirs
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/tachyon.dirs b/bigtop-packages/src/deb/tachyon/tachyon.dirs
new file mode 100644
index 0000000..1216d18
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/tachyon.dirs
@@ -0,0 +1,3 @@
+/etc/tachyon/
+/usr/lib/tachyon
+/usr/bin

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/tachyon.install
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/tachyon.install b/bigtop-packages/src/deb/tachyon/tachyon.install
new file mode 100644
index 0000000..9ee1e55
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/tachyon.install
@@ -0,0 +1,3 @@
+/etc/tachyon/
+/usr/lib/tachyon
+/usr/bin/tachyon

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop-packages/src/deb/tachyon/tachyon.postinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/tachyon.postinst b/bigtop-packages/src/deb/tachyon/tachyon.postinst
new file mode 100644
index 0000000..a2078db
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/tachyon.postinst
@@ -0,0 +1,32 @@
+#!/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 -e
+
+case "$1" in
+    configure)
+    ;;
+
+    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/57f59005/bigtop-packages/src/deb/tachyon/tachyon.prerm
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/tachyon/tachyon.prerm b/bigtop-packages/src/deb/tachyon/tachyon.prerm
new file mode 100644
index 0000000..4ba1a94
--- /dev/null
+++ b/bigtop-packages/src/deb/tachyon/tachyon.prerm
@@ -0,0 +1,37 @@
+#!/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 -e
+
+case "$1" in
+    remove|upgrade|deconfigure)
+    ;;
+
+    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/57f59005/bigtop-packages/src/rpm/tachyon/SPECS/tachyon.spec
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/tachyon/SPECS/tachyon.spec b/bigtop-packages/src/rpm/tachyon/SPECS/tachyon.spec
new file mode 100644
index 0000000..210dd2e
--- /dev/null
+++ b/bigtop-packages/src/rpm/tachyon/SPECS/tachyon.spec
@@ -0,0 +1,124 @@
+# 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.
+
+Name:           tachyon
+Version:        %{tachyon_version}
+Release:        %{tachyon_release}
+Summary:       Reliable file sharing at memory speed across cluster frameworks
+License:       ASL 2.0
+URL:           http://tachyon-project.org/
+Group:         Development/Libraries
+BuildArch:     noarch
+
+Source0:       %{name}-%{tachyon_version}.tar.gz
+Source1:       do-component-build
+Source2:       install_tachyon.sh
+Source3:       init.d.tmpl
+Source4:       tachyon-master.svc
+Source5:       tachyon-worker.svc
+%define        tachyon_home /usr/lib/%{name}
+%define        tachyon_services master worker
+%define        var_lib /var/lib/%{name}
+%define        var_run /var/run/%{name}
+%define        var_log /var/log/%{name}
+
+%global        initd_dir %{_sysconfdir}/init.d
+
+%if  %{?suse_version:1}0
+# Required for init scripts
+Requires: insserv
+%global        initd_dir %{_sysconfdir}/rc.d
+
+%else
+# Required for init scripts
+Requires: /lib/lsb/init-functions
+
+%global        initd_dir %{_sysconfdir}/rc.d/init.d
+
+%endif
+
+# disable repacking jars
+%define __arch_install_post %{nil}
+
+%description
+Tachyon is a fault tolerant distributed file system
+enabling reliable file sharing at memory-speed
+across cluster frameworks, such as Spark and MapReduce.
+It achieves high performance by leveraging lineage
+information and using memory aggressively.
+Tachyon caches working set files in memory, and
+enables different jobs/queries and frameworks to
+access cached files at memory speed. Thus, Tachyon
+avoids going to disk to load data-sets that
+are frequently read.
+
+%prep
+%setup -n %{name}-%{tachyon_base_version}
+
+%build
+bash $RPM_SOURCE_DIR/do-component-build
+
+%install
+rm -rf $RPM_BUILD_ROOT
+
+# See /usr/lib/rpm/macros for info on how vars are defined.
+# Here we run the tachyon installation script.
+bash %{SOURCE2} \
+    --build-dir=%{buildroot} \
+    --bin-dir=%{_bindir} \
+    --data-dir=%{_datadir} \
+    --libexec-dir=%{_libexecdir} \
+    --var-dir=%{_var}  \
+    --prefix="${RPM_BUILD_ROOT}"
+
+for service in %{tachyon_services}
+do
+    # Install init script
+    init_file=$RPM_BUILD_ROOT/%{initd_dir}/%{name}-${service}
+    bash $RPM_SOURCE_DIR/init.d.tmpl $RPM_SOURCE_DIR/tachyon-${service}.svc rpm $init_file
+done
+
+%preun
+for service in %{tachyon_services}; do
+  /sbin/service %{name}-${service} status > /dev/null 2>&1
+  if [ $? -eq 0 ]; then
+    /sbin/service %{tachyon_name}-${service} stop > /dev/null 2>&1
+  fi
+done
+
+
+%files
+%defattr(-,root,root,-)
+%doc LICENSE README.md
+%dir %{_sysconfdir}/%{name}
+%config(noreplace) %{_sysconfdir}/%{name}/log4j.properties
+%config(noreplace) %{_sysconfdir}/%{name}/slaves
+%config(noreplace) %{initd_dir}/%{name}-master
+%config(noreplace) %{initd_dir}/%{name}-worker
+%config(noreplace) %{_sysconfdir}/%{name}/tachyon-env.sh
+%config(noreplace) %{tachyon_home}/libexec/tachyon-layout.sh
+%attr(0755,root,root) %{var_lib}
+%attr(0755,root,root) %{var_run}
+%attr(0755,root,root) %{var_log}
+%{tachyon_home}/tachyon*
+%{tachyon_home}/bin/tachyon*
+%{tachyon_home}/libexec/tachyon*
+%{_datadir}/%{name}
+/usr/bin/tachyon
+
+%clean
+rm -rf $RPM_BUILD_ROOT
+
+%changelog

http://git-wip-us.apache.org/repos/asf/bigtop/blob/57f59005/bigtop.mk
----------------------------------------------------------------------
diff --git a/bigtop.mk b/bigtop.mk
index e2e8e26..b49d0e3 100644
--- a/bigtop.mk
+++ b/bigtop.mk
@@ -321,3 +321,16 @@ BIGTOP_TOMCAT_TARBALL_DST=apache-tomcat-$(BIGTOP_TOMCAT_BASE_VERSION).tar.gz
 BIGTOP_TOMCAT_SITE=$(APACHE_MIRROR)/tomcat/tomcat-6/v$(BIGTOP_TOMCAT_BASE_VERSION)/src/
 BIGTOP_TOMCAT_ARCHIVE=$(APACHE_ARCHIVE)/tomcat/tomcat-6/v$(BIGTOP_TOMCAT_BASE_VERSION)/src/
 $(eval $(call PACKAGE,bigtop-tomcat,BIGTOP_TOMCAT))
+
+# Tachyon
+TACHYON_NAME=tachyon
+TACHYON_RELNOTES_NAME=Tachyon: a memory-centric distributed file system
+TACHYON_PKG_NAME=tachyon
+TACHYON_BASE_VERSION=0.5.0
+TACHYON_PKG_VERSION=0.5.0
+TACHYON_RELEASE_VERSION=1
+TACHYON_TARBALL_DST=tachyon-$(TACHYON_BASE_VERSION).tar.gz
+TACHYON_TARBALL_SRC=v$(TACHYON_BASE_VERSION).tar.gz
+TACHYON_SITE=https://github.com/amplab/tachyon/archive
+TACHYON_ARCHIVE=$(TACHYON_SITE)
+$(eval $(call PACKAGE,tachyon,TACHYON))