You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bigtop.apache.org by ma...@apache.org on 2015/02/26 22:54:18 UTC

bigtop git commit: BIGTOP-989: Add Apache Kafka to Apache Bigtop

Repository: bigtop
Updated Branches:
  refs/heads/master abba9b4da -> 1a9efe60f


BIGTOP-989: Add Apache Kafka to Apache Bigtop

Signed-off-by: Mark Grover <ma...@apache.org>


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

Branch: refs/heads/master
Commit: 1a9efe60f82f9061dc42cadba5550292f77aea0f
Parents: abba9b4
Author: Mani Narayan <nm...@gmail.com>
Authored: Tue Feb 24 20:58:07 2015 -0800
Committer: Mark Grover <ma...@apache.org>
Committed: Thu Feb 26 13:54:08 2015 -0800

----------------------------------------------------------------------
 .../src/common/kafka/do-component-build         |  41 +++++
 .../src/common/kafka/install_kafka.sh           | 162 ++++++++++++++++++
 .../src/common/kafka/kafka-server.svc           |  95 +++++++++++
 bigtop-packages/src/common/kafka/kafka.default  |   0
 bigtop-packages/src/deb/kafka/changelog         |   1 +
 bigtop-packages/src/deb/kafka/compat            |   1 +
 bigtop-packages/src/deb/kafka/control           |  34 ++++
 bigtop-packages/src/deb/kafka/copyright         |  11 ++
 .../src/deb/kafka/kafka-server.postinst         |  32 ++++
 .../src/deb/kafka/kafka-server.postrm           |  32 ++++
 bigtop-packages/src/deb/kafka/kafka.postinst    |  35 ++++
 bigtop-packages/src/deb/kafka/kafka.preinst     |  33 ++++
 bigtop-packages/src/deb/kafka/kafka.prerm       |  32 ++++
 bigtop-packages/src/deb/kafka/rules             |  38 +++++
 bigtop-packages/src/deb/kafka/source/format     |   1 +
 bigtop-packages/src/rpm/kafka/SPECS/kafka.spec  | 163 +++++++++++++++++++
 bigtop.mk                                       |  14 ++
 pom.xml                                         |   6 +
 18 files changed, 731 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/common/kafka/do-component-build
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/kafka/do-component-build b/bigtop-packages/src/common/kafka/do-component-build
new file mode 100644
index 0000000..ba5a577
--- /dev/null
+++ b/bigtop-packages/src/common/kafka/do-component-build
@@ -0,0 +1,41 @@
+#!/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
+
+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 -DrecompileMode=all"
+
+## this might be an issue at times
+#        http://maven.40175.n5.nabble.com/Not-finding-artifact-in-local-repo-td3727753.html
+export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=512m -XX:PermSize=1024m -XX:MaxPermSize=1024m"
+
+SCALA_VERSION=2.10.0
+
+./gradlew -PscalaVersion=${SCALA_VERSION} clean releaseTarGz -x signArchives
+
+rm -rf build
+mkdir build
+  
+cp core/build/distributions/kafka_*.tgz build/`basename core/build/distributions/kafka* .tgz`.tar.gz
+

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/common/kafka/install_kafka.sh
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/kafka/install_kafka.sh b/bigtop-packages/src/common/kafka/install_kafka.sh
new file mode 100644
index 0000000..1b044b5
--- /dev/null
+++ b/bigtop-packages/src/common/kafka/install_kafka.sh
@@ -0,0 +1,162 @@
+#!/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/kafka]
+     --lib-dir=DIR               path to install Kafka home [/usr/lib/kafka]
+     --installed-lib-dir=DIR     path where lib-dir will end up on target system
+     --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 '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
+        ;;
+        --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
+
+MAN_DIR=${MAN_DIR:-/usr/share/man/man1}
+DOC_DIR=${DOC_DIR:-/usr/share/doc/kafka}
+LIB_DIR=${LIB_DIR:-/usr/lib/kafka}
+EXAMPLES_DIR=${EXAMPLES_DIR:-$DOC_DIR/examples}
+CONF_DIR=${CONF_DIR:-/etc/kafka/conf.dist}
+KAFKA_HOME=${KAFKA_HOME:-/usr/lib/kafka}
+ETC_KAFKA_DIR=${ETC_KAFKA_DIR:-/etc/kafka}
+BIN_DIR=/usr/bin
+INSTALLED_KAFKA_DIR=${LIB_DIR}
+
+install -d -m 0755 $PREFIX/$LIB_DIR
+install -d -m 0755 $PREFIX/$DOC_DIR
+install -d -m 0755 $PREFIX/$EXAMPLES_DIR
+
+install -d -m 0755 $PREFIX/var/lib/kafka/
+install -d -m 0755 $PREFIX/var/log/kafka/
+install -d -m 0755 $PREFIX/var/run/kafka/
+
+# Create the data directory.
+install -d -m 0755 $PREFIX/var/lib/kafka/data
+
+# Unzip binary tarball contents to lib directory.
+tar zxf ${BUILD_DIR}/build/kafka*.tar.gz -C ${PREFIX}/${LIB_DIR} --strip 1
+rm -f ${PREFIX}/${LIB_DIR}/{LICENSE,NOTICE}
+
+#Remove config directory. Creating symlink below.
+rm -rf ${PREFIX}/${LIB_DIR}/config
+
+#Removing:
+# - javadoc scaladoc source jars from under libs.
+# - bat files from under bin
+rm -rf ${PREFIX}/${LIB_DIR}/libs/kafka_*source*
+rm -rf ${PREFIX}/${LIB_DIR}/libs/kafka_*javadoc*
+rm -rf ${PREFIX}/${LIB_DIR}/libs/kafka_*scaladoc*
+rm -rf ${PREFIX}/${LIB_DIR}/bin/windows
+
+chmod 755 $PREFIX/$LIB_DIR/bin/*
+
+# Exposing a few scripts by placing them under /usr/bin
+install -d -m 0755 $PREFIX/$BIN_DIR
+for file in kafka-console-consumer.sh kafka-console-producer.sh kafka-run-class.sh kafka-topics.sh
+do
+  wrapper=$PREFIX/$BIN_DIR/$file
+  cat >>$wrapper <<EOF
+#!/bin/bash
+
+# Autodetect JAVA_HOME if not defined
+. /usr/lib/bigtop-utils/bigtop-detect-javahome
+
+exec $INSTALLED_KAFKA_DIR/bin/$file "\$@"
+EOF
+  chmod 755 $wrapper
+done
+
+# Copy in the configuration files
+install -d -m 0755 $PREFIX/$CONF_DIR
+cp -a ${BUILD_DIR}/config/{tools-log4j.properties,server.properties,log4j.properties} $PREFIX/$CONF_DIR/
+ln -s ${ETC_KAFKA_DIR}/conf $PREFIX/$LIB_DIR/config
+
+# Creating symlink to /var/log/kafka
+ln -s /var/log/kafka ${PREFIX}/$LIB_DIR/logs
+
+# Removing zookeeper*.jar from under libs and dropping a symlink in place
+rm -f ${PREFIX}/${LIB_DIR}/libs/zookeeper-*.jar
+ln -sf /usr/lib/zookeeper/zookeeper.jar ${PREFIX}/${LIB_DIR}/libs/
+
+# Copy in the defaults file
+install -d -m 0755 ${PREFIX}/etc/default
+cp ${SOURCE_DIR}/kafka.default ${PREFIX}/etc/default/kafka
+
+cp ${BUILD_DIR}/{LICENSE,NOTICE} ${PREFIX}/${LIB_DIR}/

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/common/kafka/kafka-server.svc
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/kafka/kafka-server.svc b/bigtop-packages/src/common/kafka/kafka-server.svc
new file mode 100644
index 0000000..879dd43
--- /dev/null
+++ b/bigtop-packages/src/common/kafka/kafka-server.svc
@@ -0,0 +1,95 @@
+# 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="kafka-server"
+DESC="Kafka Server"
+EXEC_PATH="/usr/lib/kafka/bin/kafka-server-start.sh"
+SVC_USER="kafka"
+WORKING_DIR="/var/lib/kafka"
+DAEMON_FLAGS=""
+CONF_DIR="/etc/kafka/conf"
+PIDFILE="/var/run/kafka/${DAEMON}.pid"
+PRIORITY=86
+DEFAULTS_DIR="/etc/default/kafka"
+
+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/kafka/${DAEMON}.out
+    export LOG_DIR=`dirname $LOG_FILE`
+
+    echo "Starting $desc (kafka-server): "
+
+    if [ -z "$KAFKA_JAVA_OPTS" ]; then
+      JAVA_OPTS="-Xmx512M -server  -Dlog4j.configuration=file:$CONF_DIR/log4j.properties"
+    fi
+
+    su -s /bin/bash $SVC_USER -c "nohup $EXEC_PATH $CONF_DIR/server.properties > $LOG_FILE 2>&1 < /dev/null & "'echo $! '"> $PIDFILE"
+    sleep 3
+
+    checkstatusofproc
+    RETVAL=$?
+    if [ $RETVAL -eq $STATUS_RUNNING ]; then
+        touch $LOCKFILE
+        log_success_msg "Starting $DESC (${DAEMON}): "
+    else
+        log_failure_msg "Failure to start $DESC (${DAEMON}). Return value: $RETVAL"
+    fi
+    return $RETVAL
+}
+__EOT__
+
+}
+
+generate_stop() {
+
+cat <<'__EOT__'
+stop() {
+checkstatusofproc
+  if [ "$?" = "$STATUS_RUNNING" ] ; then
+    if [ -f $PIDFILE ]; then
+      PID=`cat $PIDFILE`
+      if [ -n $PID ]; then
+        kill -TERM $PID &>/dev/null
+      fi
+    fi
+  RETVAL=$?
+  else
+    RETVAL=$RETVAL_SUCCESS
+  fi
+
+  if [ $RETVAL -eq $RETVAL_SUCCESS ]; then
+     rm -f $LOCKFILE $PIDFILE
+     log_success_msg "Stopped ${DESC}: "
+  else
+     log_failure_msg "Failed to stop ${DESC}. Return value: $RETVAL"
+  fi
+  return $RETVAL
+}
+
+__EOT__
+
+}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/common/kafka/kafka.default
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/common/kafka/kafka.default b/bigtop-packages/src/common/kafka/kafka.default
new file mode 100644
index 0000000..e69de29

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

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

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/deb/kafka/control
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/control b/bigtop-packages/src/deb/kafka/control
new file mode 100644
index 0000000..2c67ea0
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/control
@@ -0,0 +1,34 @@
+# 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: kafka
+Section: misc
+Priority: extra
+Maintainer: Bigtop <de...@bigtop.apache.org>
+Build-Depends: debhelper (>= 7.0.50~)
+Standards-Version: 3.8.0
+Homepage: http://kafka.apache.org/
+
+Package: kafka
+Architecture: all
+Depends: zookeeper, bigtop-utils (>= 0.7)
+Description: Apache Kafka is publish-subscribe messaging rethought as a distributed commit log.
+ A single Kafka broker can handle hundreds of megabytes of reads and writes per second from thousands of clients. It can be elastically and transparently expanded without downtime. Data streams are partitioned and spread over a cluster of machines to allow data streams larger than the capability of any single machine and to allow clusters of co-ordinated consumers
+
+Package: kafka-server
+Architecture: all
+Depends: kafka
+Description: Bundles the init script for kafka server.
+ This package starts the kafka server on startup

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/deb/kafka/copyright
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/copyright b/bigtop-packages/src/deb/kafka/copyright
new file mode 100644
index 0000000..35c5893
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/copyright
@@ -0,0 +1,11 @@
+Format: http://dep.debian.net/deps/dep7
+Source: http://kafka.apache.org/
+Upstream-Name: Apache Kafka
+
+Files: *
+Copyright: 2008-2015, 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/1a9efe60/bigtop-packages/src/deb/kafka/kafka-server.postinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/kafka-server.postinst b/bigtop-packages/src/deb/kafka/kafka-server.postinst
new file mode 100644
index 0000000..acaf30c
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/kafka-server.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)
+        update-rc.d kafka-server defaults >/dev/null || exit 1
+    ;;
+    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/1a9efe60/bigtop-packages/src/deb/kafka/kafka-server.postrm
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/kafka-server.postrm b/bigtop-packages/src/deb/kafka/kafka-server.postrm
new file mode 100644
index 0000000..ca6f39d
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/kafka-server.postrm
@@ -0,0 +1,32 @@
+#!/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
+    purge)
+        update-rc.d -f kafka-server remove > /dev/null || exit 1
+    ;;
+    remove|upgrade|failed-upgrade|abort-install|abort-upgrade|disappear)
+    ;;
+    *)
+        echo "postrm called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+#DEBHELPER#

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/deb/kafka/kafka.postinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/kafka.postinst b/bigtop-packages/src/deb/kafka/kafka.postinst
new file mode 100644
index 0000000..71dc564
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/kafka.postinst
@@ -0,0 +1,35 @@
+#!/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)
+        update-alternatives --install /etc/kafka/conf kafka-conf /etc/kafka/conf.dist 30
+        chown kafka:kafka -R /var/log/kafka
+        chown kafka:kafka -R /var/run/kafka
+        chown kafka:kafka -R /var/lib/kafka
+    ;;
+    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/1a9efe60/bigtop-packages/src/deb/kafka/kafka.preinst
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/kafka.preinst b/bigtop-packages/src/deb/kafka/kafka.preinst
new file mode 100644
index 0000000..e20ad51
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/kafka.preinst
@@ -0,0 +1,33 @@
+#!/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
+    install|upgrade)
+        getent group kafka >/dev/null || groupadd -r kafka
+        getent passwd kafka > /dev/null || useradd -c "kafka" -s /bin/bash -g kafka -d /var/lib/kafka kafka 2> /dev/null || :
+    ;;
+    abort-upgrade)
+    ;;
+    *)
+        echo "preinst called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+#DEBHELPER#

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/deb/kafka/kafka.prerm
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/kafka.prerm b/bigtop-packages/src/deb/kafka/kafka.prerm
new file mode 100644
index 0000000..6d64f02
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/kafka.prerm
@@ -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
+    remove|upgrade|deconfigure)
+        update-alternatives --remove kafka-conf /etc/kafka/conf.dist || :
+    ;;
+    failed-upgrade)
+    ;;
+    *)
+        echo "prerm called with unknown argument \`$1'" >&2
+        exit 1
+    ;;
+esac
+
+#DEBHELPER#

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/deb/kafka/rules
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/deb/kafka/rules b/bigtop-packages/src/deb/kafka/rules
new file mode 100644
index 0000000..b5e590b
--- /dev/null
+++ b/bigtop-packages/src/deb/kafka/rules
@@ -0,0 +1,38 @@
+#!/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:
+	env FULL_VERSION=${KAFKA_BASE_VERSION} bash debian/do-component-build
+
+override_dh_auto_install:
+	env FULL_VERSION=${KAFKA_BASE_VERSION} bash -x debian/install_kafka.sh \
+	  --build-dir=./ \
+	  --source-dir=debian \
+	  --prefix=debian/kafka
+	mkdir -p debian/kafka-server/etc/init.d/
+	bash debian/init.d.tmpl debian/kafka-server.svc deb debian/kafka-server/etc/init.d/kafka-server

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

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop-packages/src/rpm/kafka/SPECS/kafka.spec
----------------------------------------------------------------------
diff --git a/bigtop-packages/src/rpm/kafka/SPECS/kafka.spec b/bigtop-packages/src/rpm/kafka/SPECS/kafka.spec
new file mode 100644
index 0000000..58aa132
--- /dev/null
+++ b/bigtop-packages/src/rpm/kafka/SPECS/kafka.spec
@@ -0,0 +1,163 @@
+# 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 kafka_name kafka
+%define lib_kafka /usr/lib/%{kafka_name}
+%define var_lib_kafka /var/lib/%{kafka_name}
+%define var_run_kafka /var/run/%{kafka_name}
+%define var_log_kafka /var/log/%{kafka_name}
+%define bin_kafka /usr/lib/%{kafka_name}/bin
+%define etc_kafka /etc/%{kafka_name}
+%define config_kafka %{etc_kafka}/conf
+%define bin /usr/bin
+%define man_dir /usr/share/man
+
+%if  %{!?suse_version:1}0
+%define doc_kafka %{_docdir}/%{kafka_name}-%{kafka_version}
+%define alternatives_cmd alternatives
+%global initd_dir %{_sysconfdir}/rc.d/init.d
+
+%else
+
+# 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
+
+%define alternatives_cmd update-alternatives
+%define doc_kafka %{_docdir}/%{kafka_name}-%{kafka_version}
+%global initd_dir %{_sysconfdir}/rc.d
+
+%define __os_install_post \
+    %{suse_check} ; \
+    /usr/lib/rpm/brp-compress ; \
+    %{nil}
+
+%endif
+
+
+# disable repacking jars
+%define __os_install_post %{nil}
+
+Name: kafka
+Version: %{kafka_version}
+Release: %{kafka_release}
+Summary: Apache Kafka is publish-subscribe messaging rethought as a distributed commit log.
+URL: http://kafka.apache.org/
+Group: Development/Libraries
+BuildArch: noarch
+Buildroot: %(mktemp -ud %{_tmppath}/%{kafka_name}-%{version}-%{release}-XXXXXX)
+License: ASL 2.0
+Source0: %{kafka_name}-%{kafka_base_version}.tar.gz
+Source1: do-component-build
+Source2: install_%{kafka_name}.sh
+Source3: kafka-server.svc
+Source4: init.d.tmpl
+Source6: kafka.default
+Requires: zookeeper
+Requires: bigtop-utils >= 0.7
+Requires(preun): /sbin/service
+
+%description
+Apache Kafka is publish-subscribe messaging rethought as a distributed commit log.
+A single Kafka broker can handle hundreds of megabytes of reads and writes per second
+from thousands of clients. It can be elastically and transparently expanded without downtime.
+Data streams are partitioned and spread over a cluster of machines to allow data streams
+larger than the capability of any single machine and to allow clusters of co-ordinated consumers
+
+%package server
+Summary: Server for kafka
+Group: System/Daemons
+Requires: kafka = %{version}-%{release}
+
+%description server
+Bundles the init script for kafka server.
+
+%prep
+%setup -n %{kafka_name}-%{kafka_base_version}-src
+
+%build
+bash $RPM_SOURCE_DIR/do-component-build
+
+%install
+%__rm -rf $RPM_BUILD_ROOT
+%__install -d -m 0755 $RPM_BUILD_ROOT/etc/default/
+%__install -m 0644 %{SOURCE6} $RPM_BUILD_ROOT/etc/default/%{kafka_name}
+
+bash $RPM_SOURCE_DIR/install_kafka.sh \
+          --build-dir=`pwd`         \
+          --source-dir=$RPM_SOURCE_DIR \
+          --prefix=$RPM_BUILD_ROOT  \
+		  --doc-dir=%{doc_kafka} \
+
+# Generate the init script
+init_file=$RPM_BUILD_ROOT/%{initd_dir}/%{kafka_name}-server
+bash $RPM_SOURCE_DIR/init.d.tmpl $RPM_SOURCE_DIR/%{kafka_name}-server.svc rpm $init_file
+
+
+#######################
+#### Kafka core section ####
+#######################
+%pre
+getent group kafka >/dev/null || groupadd -r kafka
+getent passwd kafka >/dev/null || useradd -c "Kafka" -s /sbin/nologin -g kafka -r -d %{var_lib_kafka} kafka 2> /dev/null || :
+
+%post
+%{alternatives_cmd} --install %{config_kafka} %{kafka_name}-conf %{config_kafka}.dist 30
+
+%preun
+if [ "$1" = 0 ]; then
+  %{alternatives_cmd} --remove %{kafka_name}-conf %{config_kafka}.dist || :
+fi
+
+/sbin/service %{kafka_name}-server status > /dev/null 2>&1
+if [ $? -eq 0 ]; then
+  /sbin/service %{kafka_name}-server stop > /dev/null 2>&1
+fi
+
+#######################
+#### Kafka-server section ####
+#######################
+%post server
+chkconfig --add %{kafka_name}-server
+
+%preun server
+/sbin/service %{kafka_name}-server status > /dev/null 2>&1
+if [ $? -eq 0  ] ; then
+  service kafka-server stop > /dev/null 2>&1
+  chkconfig --del %{kafka_name}-server
+fi
+
+%postun server
+if [ $1 -ge 1 ]; then
+  service %{kafka_name}-server condrestart >/dev/null 2>&1
+fi
+
+#######################
+#### FILES SECTION ####
+#######################
+%files server
+%attr(0755,root,root) %{initd_dir}/%{kafka_name}-server
+
+%files
+%defattr(-,root,root,755)
+%{bin}
+%config(noreplace) %{config_kafka}.dist
+%config(noreplace) /etc/default/kafka
+%attr(0755,kafka,kafka) %{lib_kafka}
+%attr(0755,kafka,kafka) %docdir %{doc_kafka}
+%attr(0755,kafka,kafka) %{var_lib_kafka}
+%attr(0755,kafka,kafka) %{var_run_kafka}
+%attr(0755,kafka,kafka) %{var_log_kafka}

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/bigtop.mk
----------------------------------------------------------------------
diff --git a/bigtop.mk b/bigtop.mk
index 5789e4c..bec0f82 100644
--- a/bigtop.mk
+++ b/bigtop.mk
@@ -332,3 +332,17 @@ 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))
+
+# Kafka
+KAFKA_NAME=kafka
+KAFKA_RELNOTES_NAME=Apache Kafka
+KAFKA_PKG_NAME=kafka
+KAFKA_BASE_VERSION=0.8.1.1
+KAFKA_PKG_VERSION=$(KAFKA_BASE_VERSION)
+KAFKA_RELEASE_VERSION=1
+KAFKA_TARBALL_DST=kafka-$(KAFKA_BASE_VERSION).tar.gz
+KAFKA_TARBALL_SRC=kafka-$(KAFKA_BASE_VERSION)-src.tgz
+KAFKA_DOWNLOAD_PATH=/kafka/$(KAFKA_BASE_VERSION)
+KAFKA_SITE=$(APACHE_MIRROR)$(KAFKA_DOWNLOAD_PATH)
+KAFKA_ARCHIVE=$(APACHE_ARCHIVE)$(KAFKA_DOWNLOAD_PATH)
+$(eval $(call PACKAGE,kafka,KAFKA))

http://git-wip-us.apache.org/repos/asf/bigtop/blob/1a9efe60/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bc0d1d5..1e8df77 100644
--- a/pom.xml
+++ b/pom.xml
@@ -49,6 +49,7 @@
     <giraph.version>1.0.0</giraph.version>
     <solr.version>4.6.0</solr.version>
     <spark.version>0.9.1</spark.version>
+    <kafka.version>0.8.1.1</kafka.version>
     <phoenix.version>4.2.2</phoenix.version>
     <spark-smoke.version>${project.version}</spark-smoke.version>
 
@@ -244,6 +245,11 @@
         <artifactId>spark-core_2.10</artifactId>
         <version>${spark.version}</version>
       </dependency>
+       <dependency>
+        <groupId>org.apache.kafka</groupId>
+        <artifactId>kafka</artifactId>
+        <version>${kafka.version}</version>
+      </dependency>
       <dependency>
         <groupId>org.apache.phoenix</groupId>
         <artifactId>phoenix-core</artifactId>