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 2013/08/02 23:26:39 UTC

git commit: BIGTOP-715. Add Spark packaging

Updated Branches:
  refs/heads/master de6703ca5 -> e993a3edb


BIGTOP-715. Add Spark packaging

Adding Spark packaging, system services, etc. The bits are taken from the current HEAD of
the project's git master


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

Branch: refs/heads/master
Commit: e993a3edb779ed4766679ec7e748f1cad212a491
Parents: de6703c
Author: Konstantin Boudnik <co...@apache.org>
Authored: Thu Jul 18 18:36:50 2013 -0700
Committer: Konstantin Boudnik <co...@apache.org>
Committed: Fri Aug 2 14:24:21 2013 -0700

----------------------------------------------------------------------
 DEVNOTES                                        |   5 +
 .../src/common/spark/compute-classpath.sh       |  69 +++++++
 .../src/common/spark/do-component-build         |  38 ++++
 .../src/common/spark/install_spark.sh           | 188 +++++++++++++++++++
 bigtop-packages/src/common/spark/run            | 133 +++++++++++++
 bigtop-packages/src/common/spark/spark-executor |   7 +
 .../src/common/spark/spark-master.svc           |  71 +++++++
 .../src/common/spark/spark-worker.svc           |  75 ++++++++
 bigtop-packages/src/deb/spark/changelog         |   1 +
 bigtop-packages/src/deb/spark/compat            |   1 +
 bigtop-packages/src/deb/spark/control           |  33 ++++
 bigtop-packages/src/deb/spark/copyright         |  15 ++
 bigtop-packages/src/deb/spark/rules             |  91 +++++++++
 bigtop-packages/src/deb/spark/spark.postinst    |  34 ++++
 bigtop-packages/src/deb/spark/spark.prerm       |  38 ++++
 bigtop-packages/src/rpm/spark/SPECS/spark.spec  | 134 +++++++++++++
 bigtop.mk                                       |  13 ++
 package.mk                                      |   1 +
 18 files changed, 947 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/DEVNOTES
----------------------------------------------------------------------
diff --git a/DEVNOTES b/DEVNOTES
index 8c13886..f94f2a9 100644
--- a/DEVNOTES
+++ b/DEVNOTES
@@ -16,5 +16,10 @@
 1. Debian
   1.0. apt-get install -y git subversion build-essential dh-make debhelper devscripts ant ant-optional autoconf automake subversion liblzo2-dev libzip-dev sharutils libfuse-dev reprepro libssl-dev
 
+  FIXME?: You also need to have Scala 2.9.3  installed and SCALA_HOME
+  env. var pointed properly. Otherwise, Spark project won't be build as it
+  requires scala. Only older version of the language environment is available
+  on Ubuntu at the moment.
+
 2. RedHat
   2.0 yum install -y git subversion fuse-devel fuse fuse-libs

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/common/spark/compute-classpath.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/compute-classpath.sh b/bigtop-packages/src/common/spark/compute-classpath.sh
new file mode 100644
index 0000000..6bf4790
--- /dev/null
+++ b/bigtop-packages/src/common/spark/compute-classpath.sh
@@ -0,0 +1,69 @@
+#!/bin/bash
+
+# This script computes Spark's classpath and prints it to stdout; it's used by both the "run"
+# script and the ExecutorRunner in standalone cluster mode.
+
+SCALA_VERSION=2.9.3
+
+# Figure out where Spark is installed
+FWDIR="$(cd `dirname $0`/..; pwd)"
+
+# Load environment variables from conf/spark-env.sh, if it exists
+if [ -e $FWDIR/conf/spark-env.sh ] ; then
+  . $FWDIR/conf/spark-env.sh
+fi
+
+CORE_DIR="$FWDIR/core"
+REPL_DIR="$FWDIR/repl"
+REPL_BIN_DIR="$FWDIR/repl-bin"
+EXAMPLES_DIR="$FWDIR/examples"
+BAGEL_DIR="$FWDIR/bagel"
+MLLIB_DIR="$FWDIR/mllib"
+STREAMING_DIR="$FWDIR/streaming"
+PYSPARK_DIR="$FWDIR/python"
+
+# Build up classpath
+CLASSPATH="$SPARK_CLASSPATH"
+CLASSPATH="$CLASSPATH:$FWDIR/conf"
+CLASSPATH="$CLASSPATH:$CORE_DIR/lib/*"
+CLASSPATH="$CLASSPATH:$REPL_DIR/lib/*"
+CLASSPATH="$CLASSPATH:$EXAMPLES_DIR/lib/*"
+CLASSPATH="$CLASSPATH:$BAGEL_DIR/lib/*"
+CLASSPATH="$CLASSPATH:$MLLIB_DIR/lib/*"
+CLASSPATH="$CLASSPATH:$STREAMING_DIR/lib/*"
+CLASSPATH="$CLASSPATH:$FWDIR/lib/*"
+#CLASSPATH="$CLASSPATH:$CORE_DIR/src/main/resources"
+if [ -e "$PYSPARK_DIR" ]; then
+  for jar in `find $PYSPARK_DIR/lib -name '*jar'`; do
+    CLASSPATH="$CLASSPATH:$jar"
+  done
+fi
+
+# Add hadoop conf dir - else FileSystem.*, etc fail !
+# Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts
+# the configuration files.
+
+export DEFAULT_HADOOP=/usr/lib/hadoop
+export DEFAULT_HADOOP_CONF=/etc/hadoop/conf
+export HADOOP_HOME=${HADOOP_HOME:-$DEFAULT_HADOOP}
+export HADOOP_CONF_DIR=${HADOOP_CONF_DIR:-$DEFAULT_HADOOP_CONF}
+
+CLASSPATH="$CLASSPATH:$HADOOP_CONF_DIR"
+if [ "x" != "x$YARN_CONF_DIR" ]; then
+  CLASSPATH="$CLASSPATH:$YARN_CONF_DIR"
+fi
+# Let's make sure that all needed hadoop libs are added properly
+CLASSPATH="$CLASSPATH:$HADOOP_HOME/lib/*:$HADOOP_HOME/*:${HADOOP_HOME}-hdfs/lib/*:${HADOOP_HOME}-hdfs/*:${HADOOP_HOME}-yarn/*:/usr/lib/hadoop-mapreduce/*"
+# Add Scala standard library
+if [ -z "$SCALA_LIBRARY_PATH" ]; then
+  if [ -z "$SCALA_HOME" ]; then
+    echo "SCALA_HOME is not set" >&2
+    exit 1
+  fi
+  SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
+fi
+CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-library.jar"
+CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/scala-compiler.jar"
+CLASSPATH="$CLASSPATH:$SCALA_LIBRARY_PATH/jline.jar"
+
+echo "$CLASSPATH"

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/common/spark/do-component-build
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/do-component-build b/bigtop-packages/src/common/spark/do-component-build
new file mode 100644
index 0000000..f36ab68
--- /dev/null
+++ b/bigtop-packages/src/common/spark/do-component-build
@@ -0,0 +1,38 @@
+#!/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
+
+if [ "x$SCALA_HOME" = "x" ]; then
+    echo "SCALA_HOME has to be set before the build can run"
+    exit 2
+fi
+
+BUILD_OPTS="-Divy.home=${HOME}/.ivy2 -Dsbt.ivy.home=${HOME}/.ivy2 -Duser.home=${HOME} \
+            -Drepo.maven.org=$IVY_MIRROR_PROP \
+            -Dreactor.repo=file://${HOME}/.m2/repository \
+            -DskipTests"
+## this might be an issue at times
+#        http://maven.40175.n5.nabble.com/Not-finding-artifact-in-local-repo-td3727753.html
+export MAVEN_OPTS="-XX:PermSize=1024m -XX:MaxPermSize=1024m"
+
+# FIXME: this is a really unfortunate hack: make sure we are building with
+# correct version
+sed -ie "s#<yarn.version>2.0.2-alpha#<yarn.version>$HADOOP_VERSION#" pom.xml
+
+mvn -Phadoop2-yarn,assembly -DnoExpensive $BUILD_OPTS package

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/common/spark/install_spark.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/install_spark.sh b/bigtop-packages/src/common/spark/install_spark.sh
new file mode 100644
index 0000000..0f4db99
--- /dev/null
+++ b/bigtop-packages/src/common/spark/install_spark.sh
@@ -0,0 +1,188 @@
+#!/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
+
+usage() {
+  echo "
+usage: $0 <options>
+  Required not-so-options:
+     --build-dir=DIR             path to dist.dir
+     --source-dir=DIR            path to package shared files dir
+     --prefix=PREFIX             path to install into
+
+  Optional options:
+     --doc-dir=DIR               path to install docs into [/usr/share/doc/spark]
+     --lib-dir=DIR               path to install Spark home [/usr/lib/spark]
+     --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 'source-dir:' \
+  -l 'examples-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
+        ;;
+        --source-dir)
+        SOURCE_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
+        ;;
+        --)
+        shift ; break
+        ;;
+        *)
+        echo "Unknown option: $1"
+        usage
+        exit 1
+        ;;
+    esac
+done
+
+for var in PREFIX BUILD_DIR SOURCE_DIR; do
+  if [ -z "$(eval "echo \$$var")" ]; then
+    echo Missing param: $var
+    usage
+  fi
+done
+
+if [ -z "${SCALA_HOME}" ]; then
+    echo Missing env. var SCALA_HOME
+    usage
+fi
+
+MAN_DIR=${MAN_DIR:-/usr/share/man/man1}
+DOC_DIR=${DOC_DIR:-/usr/share/doc/spark}
+LIB_DIR=${LIB_DIR:-/usr/lib/spark}
+SPARK_BIN_DIR=${LIB_DIR:-/usr/lib/spark/bin}
+INSTALLED_LIB_DIR=${INSTALLED_LIB_DIR:-/usr/lib/spark}
+EXAMPLES_DIR=${EXAMPLES_DIR:-$DOC_DIR/examples}
+BIN_DIR=${BIN_DIR:-/usr/bin}
+CONF_DIR=${CONF_DIR:-/etc/spark/conf.dist}
+SCALA_HOME=${SCALA_HOME:-/usr/share/scala}
+
+install -d -m 0755 $PREFIX/$LIB_DIR
+install -d -m 0755 $PREFIX/$LIB_DIR/lib
+install -d -m 0755 $PREFIX/$SPARK_BIN_DIR
+install -d -m 0755 $PREFIX/$DOC_DIR
+
+tar --wildcards -C $PREFIX/$LIB_DIR -zxf ${BUILD_DIR}/assembly/target/spark-assembly-*-dist.tar.gz 'lib/*'
+
+for comp in core repl bagel mllib streaming; do
+  install -d -m 0755 $PREFIX/$LIB_DIR/$comp/lib
+  tar --wildcards -C $PREFIX/$LIB_DIR/$comp/lib -zxf ${BUILD_DIR}/assembly/target/spark-assembly-*-dist.tar.gz spark-$comp\*
+done
+
+# FIXME: executor scripts need to reside in bin
+cp -a ${SOURCE_DIR}/run $PREFIX/$LIB_DIR
+cp -a ${SOURCE_DIR}/spark-executor $PREFIX/$LIB_DIR
+cp -a ${SOURCE_DIR}/compute-classpath.sh $PREFIX/$SPARK_BIN_DIR
+cp -a ${BUILD_DIR}/spark-shell $PREFIX/$LIB_DIR
+
+# Copy in the configuration files
+install -d -m 0755 $PREFIX/$CONF_DIR
+cp -a ${BUILD_DIR}/conf/* $PREFIX/$CONF_DIR
+cp  $PREFIX/$CONF_DIR/spark-env.sh.template $PREFIX/$CONF_DIR/spark-env.sh
+ln -s /etc/spark/conf $PREFIX/$LIB_DIR/conf
+
+# Unpack static UI resources into install_dir/spark where it is expected to be
+tar --wildcards --transform 's,ui-resources/spark,spark,' -C $PREFIX/$LIB_DIR -zxf ${BUILD_DIR}/assembly/target/spark-assembly-*-dist.tar.gz ui-resources/\*
+
+# set correct permissions for exec. files
+for execfile in run spark-shell spark-executor ; do
+  chmod 755 $PREFIX/$LIB_DIR/$execfile
+done
+chmod 755 $PREFIX/$SPARK_BIN_DIR/compute-classpath.sh
+
+# Copy in the wrappers
+install -d -m 0755 $PREFIX/$BIN_DIR
+for wrap in spark-executor spark-shell ; do
+  cat > $PREFIX/$BIN_DIR/$wrap <<EOF
+#!/bin/sh 
+
+# Autodetect JAVA_HOME if not defined
+if [ -e /usr/libexec/bigtop-detect-javahome ]; then
+  . /usr/libexec/bigtop-detect-javahome
+elif [ -e /usr/lib/bigtop-utils/bigtop-detect-javahome ]; then
+  . /usr/lib/bigtop-utils/bigtop-detect-javahome
+fi
+
+exec $INSTALLED_LIB_DIR/$wrap "\$@"
+EOF
+  chmod 755 $PREFIX/$BIN_DIR/$wrap
+done
+
+cat >> $PREFIX/$CONF_DIR/spark-env.sh <<EOF
+
+### Let's run everything with JVM runtime, instead of Scala
+export SPARK_LAUNCH_WITH_SCALA=0
+export SPARK_LIBRARY_PATH=\${SPARK_HOME}/lib
+export SCALA_LIBRARY_PATH=\${SPARK_HOME}/lib
+export SPARK_MASTER_WEBUI_PORT=18080
+export SPARK_MASTER_PORT=7077
+
+### Comment above 2 lines and uncomment the following if
+### you want to run with scala version, that is included with the package
+#export SCALA_HOME=\${SCALA_HOME:-$LIB_DIR/scala}
+#export PATH=\$PATH:\$SCALA_HOME/bin
+
+### change the following to specify a real cluster's Master host
+export STANDALONE_SPARK_MASTER_HOST=\`hostname\`
+
+EOF
+
+install -d -m 1755 $PREFIX/log/spark
+install -d -m 1755 $PREFIX/run/spark
+install -d -m 1755 $PREFIX/run/spark/work
+ln -s /var/run/spark/work $PREFIX/$LIB_DIR/work

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/common/spark/run
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/run b/bigtop-packages/src/common/spark/run
new file mode 100644
index 0000000..a02b2db
--- /dev/null
+++ b/bigtop-packages/src/common/spark/run
@@ -0,0 +1,133 @@
+#!/bin/bash
+
+SCALA_VERSION=2.9.3
+
+# Figure out where the Scala framework is installed
+FWDIR="$(cd `dirname $0`; pwd)"
+
+# Export this as SPARK_HOME
+export SPARK_HOME="$FWDIR"
+
+# Load environment variables from conf/spark-env.sh, if it exists
+if [ -e $FWDIR/conf/spark-env.sh ] ; then
+  . $FWDIR/conf/spark-env.sh
+fi
+
+if [ -z "$1" ]; then
+  echo "Usage: run <spark-class> [<args>]" >&2
+  exit 1
+fi
+
+# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable
+# values for that; it doesn't need a lot
+if [ "$1" = "spark.deploy.master.Master" -o "$1" = "spark.deploy.worker.Worker" ]; then
+  SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m}
+  SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true"
+  # Do not overwrite SPARK_JAVA_OPTS environment variable in this script
+  OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS"   # Empty by default
+else
+  OUR_JAVA_OPTS="$SPARK_JAVA_OPTS"
+fi
+
+
+# Add java opts for master, worker, executor. The opts maybe null
+case "$1" in
+  'spark.deploy.master.Master')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS"
+    ;;
+  'spark.deploy.worker.Worker')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS"
+    ;;
+  'spark.executor.StandaloneExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'spark.executor.MesosExecutorBackend')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS"
+    ;;
+  'spark.repl.Main')
+    OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS"
+    ;;
+esac
+
+# Figure out whether to run our class with java or with the scala launcher.
+# In most cases, we'd prefer to execute our process with java because scala
+# creates a shell script as the parent of its Java process, which makes it
+# hard to kill the child with stuff like Process.destroy(). However, for
+# the Spark shell, the wrapper is necessary to properly reset the terminal
+# when we exit, so we allow it to set a variable to launch with scala.
+if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
+  if [ "$SCALA_HOME" ]; then
+    RUNNER="${SCALA_HOME}/bin/scala"
+  else
+    if [ `command -v scala` ]; then
+      RUNNER="scala"
+    else
+      echo "SCALA_HOME is not set and scala is not in PATH" >&2
+      exit 1
+    fi
+  fi
+else
+  if [ -n "${JAVA_HOME}" ]; then
+    RUNNER="${JAVA_HOME}/bin/java"
+  else
+    if [ `command -v java` ]; then
+      RUNNER="java"
+    else
+      echo "JAVA_HOME is not set" >&2
+      exit 1
+    fi
+  fi
+  if [ -z "$SCALA_LIBRARY_PATH" ]; then
+    if [ -z "$SCALA_HOME" ]; then
+      echo "SCALA_HOME is not set" >&2
+      exit 1
+    fi
+    SCALA_LIBRARY_PATH="$SCALA_HOME/lib"
+  fi
+fi
+
+# Figure out how much memory to use per executor and set it as an environment
+# variable so that our process sees it and can report it to Mesos
+if [ -z "$SPARK_MEM" ] ; then
+  SPARK_MEM="512m"
+fi
+export SPARK_MEM
+
+# Set JAVA_OPTS to be able to load native libraries and to set heap size
+JAVA_OPTS="$OUR_JAVA_OPTS"
+JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH"
+JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM"
+# Load extra JAVA_OPTS from conf/java-opts, if it exists
+if [ -e $FWDIR/conf/java-opts ] ; then
+  JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`"
+fi
+export JAVA_OPTS
+# Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala!
+
+CORE_DIR="$FWDIR/core"
+EXAMPLES_DIR="$FWDIR/examples"
+REPL_DIR="$FWDIR/repl"
+
+# Compute classpath using external script
+CLASSPATH=`$FWDIR/bin/compute-classpath.sh`
+export CLASSPATH
+
+# Figure out the JAR file that our examples were packaged into. This includes a bit of a hack
+# to avoid the -sources and -doc packages that are built by publish-local.
+if [ -e "$EXAMPLES_DIR/lib/spark-examples"*[0-9T].jar ]; then
+  # Use the JAR from the SBT build
+  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/lib/spark-examples"*[0-9T].jar`
+fi
+if [ -e "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar ]; then
+  # Use the JAR from the Maven build
+  export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR/target/spark-examples"*[0-9T].jar`
+fi
+
+if [ "$SPARK_LAUNCH_WITH_SCALA" == "1" ]; then
+  EXTRA_ARGS=""     # Java options will be passed to scala as JAVA_OPTS
+else
+  # The JVM doesn't read JAVA_OPTS by default so we need to pass it in
+  EXTRA_ARGS="$JAVA_OPTS"
+fi
+
+exec "$RUNNER" -cp "$CLASSPATH" $EXTRA_ARGS "$@"

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/common/spark/spark-executor
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/spark-executor b/bigtop-packages/src/common/spark/spark-executor
new file mode 100644
index 0000000..55a7113
--- /dev/null
+++ b/bigtop-packages/src/common/spark/spark-executor
@@ -0,0 +1,7 @@
+#!/bin/bash
+# This is a modification of the standard Spark executor script to run
+# with Standalone executor backend
+# TODO: we are likely need to add YARN executor later
+FWDIR="`dirname $0`"
+echo "Running spark-executor with framework dir = $FWDIR"
+exec $FWDIR/run spark.executor.StandaloneExecutorBackend

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/common/spark/spark-master.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/spark-master.svc b/bigtop-packages/src/common/spark/spark-master.svc
new file mode 100644
index 0000000..856bf31
--- /dev/null
+++ b/bigtop-packages/src/common/spark/spark-master.svc
@@ -0,0 +1,71 @@
+# 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="spark-${TYPE}"
+DESC="Spark ${TYPE}"
+EXEC_PATH="/usr/lib/spark/run"
+SVC_USER="spark"
+WORKING_DIR="/var/lib/spark"
+DAEMON_FLAGS=""
+CONF_DIR="/etc/spark/conf"
+PIDFILE="/var/run/spark/${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/spark/${DAEMON}.out
+
+    su -s /bin/sh $SVC_USER -c "nohup nice -n 0 \
+        ${EXEC_PATH} spark.deploy.master.Master $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/e993a3ed/bigtop-packages/src/common/spark/spark-worker.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/spark/spark-worker.svc b/bigtop-packages/src/common/spark/spark-worker.svc
new file mode 100644
index 0000000..7f9e680
--- /dev/null
+++ b/bigtop-packages/src/common/spark/spark-worker.svc
@@ -0,0 +1,75 @@
+# 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="spark-${TYPE}"
+DESC="Spark ${TYPE}"
+EXEC_PATH="/usr/lib/spark/run"
+SVC_USER="spark"
+WORKING_DIR="/var/lib/spark"
+DAEMON_FLAGS=""
+CONF_DIR="/etc/spark/conf"
+PIDFILE="/var/run/spark/${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/spark/${DAEMON}.out
+
+    if [ -f $CONF_DIR/spark-env.sh ]; then
+        . $CONF_DIR/spark-env.sh
+    fi
+
+    su -s /bin/sh $SVC_USER -c "nohup nice -n 0 \
+        ${EXEC_PATH} spark.deploy.worker.Worker spark://$STANDALONE_SPARK_MASTER_HOST:$SPARK_MASTER_PORT $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/e993a3ed/bigtop-packages/src/deb/spark/changelog
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/changelog b/bigtop-packages/src/deb/spark/changelog
new file mode 100644
index 0000000..547ed02
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/changelog
@@ -0,0 +1 @@
+--- This is auto-generated 

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/deb/spark/compat
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/compat b/bigtop-packages/src/deb/spark/compat
new file mode 100644
index 0000000..1e8b314
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/compat
@@ -0,0 +1 @@
+6

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/deb/spark/control
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/control b/bigtop-packages/src/deb/spark/control
new file mode 100644
index 0000000..b1edc5e
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/control
@@ -0,0 +1,33 @@
+# 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: spark
+Section: misc
+Priority: extra
+Maintainer: Bigtop <bi...@incubator.apache.org>
+Build-Depends: debhelper (>= 6)
+Standards-Version: 3.8.0
+Homepage: http://www.spark-project.org/
+
+Package: spark 
+Architecture: all
+Depends: bigtop-utils
+Description: Lightning-Fast Cluster Computing
+ Spark is a MapReduce-like cluster computing framework designed to support
+ low-latency iterative jobs and interactive use from an interpreter. It is
+ written in Scala, a high-level language for the JVM, and exposes a clean 
+ language-integrated syntax that makes it easy to write parallel jobs. 
+ Spark runs on top of the Apache Mesos cluster manager.
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/deb/spark/copyright
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/copyright b/bigtop-packages/src/deb/spark/copyright
new file mode 100644
index 0000000..13dbc02
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/copyright
@@ -0,0 +1,15 @@
+Format: http://dep.debian.net/deps/dep5
+Source: http://www.spark-project.org/
+Upstream-Name: Spark Project
+
+Files: *
+Copyright: 2010-2011, The Spark Project
+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/e993a3ed/bigtop-packages/src/deb/spark/rules
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/rules b/bigtop-packages/src/deb/spark/rules
new file mode 100644
index 0000000..8957eb1
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/rules
@@ -0,0 +1,91 @@
+#!/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
+
+patch: patch-stamp
+patch-stamp:	
+	touch $@
+
+clean:	
+	dh_testdir
+	dh_testroot
+	rm -f *-stamp
+	dh_clean
+	rm -Rf debian/tmp debian/spark
+	find debian -name .\*swp -exec rm -f {} \;
+
+build-indep: build-indep-stamp
+build-indep-stamp: patch-stamp
+	# we'll just use the build from the tarball.
+	bash debian/do-component-build
+	mkdir -p debian/tmp
+	touch $@
+
+install: install-indep
+install-indep:	
+	dh_testdir
+	dh_testroot
+	sh -x debian/install_spark.sh \
+	  --build-dir=`pwd` \
+          --doc-dir=/usr/share/doc/spark \
+	  --prefix=debian/spark
+	dh_install -i
+	(dh_lintian) || /bin/true
+
+binary-common:	
+	dh_testdir
+	dh_testroot
+	dh_installchangelogs
+	dh_installdocs
+#	dh_installexamples
+#	dh_installmenu
+#	dh_installdebconf
+#	dh_installlogrotate
+#	dh_installemacsen
+#	dh_installpam
+#	dh_installmime
+#	dh_python
+#	dh_installinit
+#	dh_installcron
+#	dh_installinfo
+	dh_installman
+	dh_link
+	dh_strip
+	dh_compress
+	dh_fixperms
+#	dh_perl
+	dh_makeshlibs
+	dh_installdeb
+	dh_shlibdeps
+	dh_gencontrol
+	dh_md5sums
+	dh_builddeb
+
+binary-indep: build-indep install-indep
+	$(MAKE) -f debian/rules DH_OPTIONS=-i binary-common
+
+binary-arch:	
+
+
+binary: binary-indep
+.PHONY: build clean binary-indep binary install-indep binary-arch

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop-packages/src/deb/spark/spark.postinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/spark.postinst b/bigtop-packages/src/deb/spark/spark.postinst
new file mode 100644
index 0000000..f3dc198
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/spark.postinst
@@ -0,0 +1,34 @@
+#!/bin/sh
+# 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)
+        # Install config alternatives
+        update-alternatives  --install /etc/spark/conf spark-conf /etc/spark/conf.dist 30
+    ;;
+
+    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/e993a3ed/bigtop-packages/src/deb/spark/spark.prerm
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/spark/spark.prerm b/bigtop-packages/src/deb/spark/spark.prerm
new file mode 100644
index 0000000..744c349
--- /dev/null
+++ b/bigtop-packages/src/deb/spark/spark.prerm
@@ -0,0 +1,38 @@
+#!/bin/sh
+# 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)
+      update-alternatives --remove spark-conf /etc/spark/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/e993a3ed/bigtop-packages/src/rpm/spark/SPECS/spark.spec
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/spark/SPECS/spark.spec b/bigtop-packages/src/rpm/spark/SPECS/spark.spec
new file mode 100644
index 0000000..b5949f4
--- /dev/null
+++ b/bigtop-packages/src/rpm/spark/SPECS/spark.spec
@@ -0,0 +1,134 @@
+# 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 spark_name spark
+%define lib_spark /usr/lib/%{spark_name}
+%define var_lib_spark /var/lib/%{spark_name}
+%define bin_spark /usr/lib/%{spark_name}/bin
+%define etc_spark /etc/%{spark_name}
+%define config_spark %{etc_spark}/conf
+%define bin /usr/bin
+%define man_dir /usr/share/man
+%define spark_services master worker
+
+%if  %{?suse_version:1}0
+%define doc_spark %{_docdir}/spark
+%define alternatives_cmd update-alternatives
+%else
+%define doc_spark %{_docdir}/spark-%{spark_version}
+%define alternatives_cmd alternatives
+%endif
+
+# disable repacking jars
+%define __os_install_post %{nil}
+
+Name: spark
+Version: %{spark_version}
+Release: %{spark_release}
+Summary: Lightning-Fast Cluster Computing
+URL: http://www.spark-project.org/
+Group: Development/Libraries
+BuildArch: noarch
+Buildroot: %(mktemp -ud %{_tmppath}/%{name}-%{version}-%{release}-XXXXXX)
+License: ASL 2.0 
+Source0: %{name}-%{spark_base_version}.tar.gz
+Source1: do-component-build 
+Source2: install_%{name}.sh
+Source3: spark-master.svc
+Source4: spark-worker.svc
+Requires: bigtop-utils
+Requires(preun): /sbin/service
+
+%global initd_dir %{_sysconfdir}/init.d
+
+%if  %{?suse_version:1}0
+# Required for init scripts
+Requires: insserv
+%define alternatives_cmd alternatives
+%global initd_dir %{_sysconfdir}/rc.d
+
+%else
+# Required for init scripts
+Requires: redhat-lsb
+
+%global initd_dir %{_sysconfdir}/rc.d/init.d
+
+%endif
+
+%description 
+Spark is a MapReduce-like cluster computing framework designed to support
+low-latency iterative jobs and interactive use from an interpreter. It is
+written in Scala, a high-level language for the JVM, and exposes a clean
+language-integrated syntax that makes it easy to write parallel jobs.
+Spark runs on top of the Apache Mesos cluster manager.
+    
+%prep
+#%setup -n %{name}-%{spark_base_version}
+%setup -n spark-master
+
+%build
+bash $RPM_SOURCE_DIR/do-component-build
+
+%install
+%__rm -rf $RPM_BUILD_ROOT
+sh $RPM_SOURCE_DIR/install_spark.sh \
+          --build-dir=`pwd`         \
+          --source-dir=$RPM_SOURCE_DIR \
+          --prefix=$RPM_BUILD_ROOT  \
+          --doc-dir=%{doc_spark} 
+
+%__install -d -m 0755 $RPM_BUILD_ROOT/%{_localstatedir}/log/%{name}
+%__install -d -m 0755 $RPM_BUILD_ROOT/%{_localstatedir}/run/%{name}
+%__install -d -m 0755 $RPM_BUILD_ROOT/%{initd_dir}/
+
+for service in %{spark_services}
+do
+    # Install init script
+    init_file=$RPM_BUILD_ROOT/%{initd_dir}/%{name}-${service}
+    bash $RPM_SOURCE_DIR/init.d.tmpl $RPM_SOURCE_DIR/spark-${service}.svc rpm $init_file
+done
+
+%pre
+getent group spark >/dev/null || groupadd -r spark
+getent passwd spark >/dev/null || useradd -c "Spark" -s /sbin/nologin -g spark -r -d %{var_lib_spark} spark 2> /dev/null || :
+
+%post
+%{alternatives_cmd} --install %{config_spark} %{spark_name}-conf %{config_spark}.dist 30
+
+%preun
+if [ "$1" = 0 ]; then
+        %{alternatives_cmd} --remove %{spark_name}-conf %{config_spark}.dist || :
+fi
+
+for service in %{spark_services}; do
+  /sbin/service %{name}-${service} status > /dev/null 2>&1
+  if [ $? -eq 0 ]; then
+    /sbin/service %{name}-${service} stop > /dev/null 2>&1
+  fi
+done
+
+#######################
+#### FILES SECTION ####
+#######################
+%files 
+%defattr(-,root,root,755)
+%config(noreplace) %{config_spark}.dist
+%doc %{doc_spark}
+%{lib_spark}
+%attr(0755,root,root) %{initd_dir}/spark-master
+%attr(0755,root,root) %{initd_dir}/spark-worker
+#%attr(0755,root,root) %{bin_spark}
+%{bin}/spark-shell
+%{bin}/spark-executor

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/bigtop.mk
----------------------------------------------------------------------
diff --git a/bigtop.mk b/bigtop.mk
index d3a6a67..f4c851d 100644
--- a/bigtop.mk
+++ b/bigtop.mk
@@ -237,6 +237,19 @@ CRUNCH_SITE=$(APACHE_MIRROR)$(CRUNCH_DOWNLOAD_PATH)
 CRUNCH_ARCHIVE=$(APACHE_ARCHIVE)$(CRUNCH_DOWNLOAD_PATH)
 $(eval $(call PACKAGE,crunch,CRUNCH))
 
+# Spark 
+SPARK_NAME=spark
+SPARK_RELNOTES_NAME=Spark
+SPARK_PKG_NAME=spark
+SPARK_BASE_VERSION=0.8.0-SNAPSHOT
+SPARK_PKG_VERSION=0.8.0
+SPARK_RELEASE_VERSION=1
+SPARK_TARBALL_DST=spark-${SPARK_BASE_VERSION}.tar.gz
+SPARK_TARBALL_SRC=master.zip
+SPARK_SITE=https://github.com/mesos/spark/archive
+SPARK_ARCHIVE=${SPARK_SITE}
+$(eval $(call PACKAGE,spark,SPARK))
+
 # Bigtop-utils
 BIGTOP_UTILS_NAME=bigtop-utils
 BIGTOP_UTILS__RELNOTES_NAME=Bigtop-utils

http://git-wip-us.apache.org/repos/asf/bigtop/blob/e993a3ed/package.mk
----------------------------------------------------------------------
diff --git a/package.mk b/package.mk
index 02aa8fb..bec000d 100644
--- a/package.mk
+++ b/package.mk
@@ -132,6 +132,7 @@ $(BUILD_DIR)/%/.deb:
 				--preserve-envvar MAVEN3_HOME \
 				--preserve-envvar MAVEN_OPTS \
 				--preserve-envvar JAVA_HOME \
+				--preserve-envvar SCALA_HOME \
 				--set-envvar=$(PKG)_BASE_VERSION=$($(PKG)_BASE_VERSION) \
 				--set-envvar=$(PKG)_VERSION=$($(PKG)_PKG_VERSION)$(BIGTOP_BUILD_STAMP) \
 				--set-envvar=$(PKG)_RELEASE=$($(PKG)_RELEASE_VERSION) \