You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@celeborn.apache.org by et...@apache.org on 2023/01/03 04:58:46 UTC

[incubator-celeborn] branch main updated: [CELEBORN-180][BUILD] Script for creating binary release artifact (#1129)

This is an automated email from the ASF dual-hosted git repository.

ethanfeng pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-celeborn.git


The following commit(s) were added to refs/heads/main by this push:
     new a16ba0e8 [CELEBORN-180][BUILD] Script for creating binary release artifact (#1129)
a16ba0e8 is described below

commit a16ba0e8070d0304c2a0811d66b981f880e1c7d5
Author: Cheng Pan <ch...@apache.org>
AuthorDate: Tue Jan 3 12:58:42 2023 +0800

    [CELEBORN-180][BUILD] Script for creating binary release artifact (#1129)
---
 build/make-distribution.sh             | 224 +++++++++++++++++++++++++++++++++
 build/release/create-package.sh        |  18 ++-
 dev/make-distribution.sh               | 176 --------------------------
 dev/reformat                           |   8 +-
 licenses-binary/LICENSE-leveldbjni.txt |  27 ++++
 licenses-binary/LICENSE-protobuf.txt   |  42 +++++++
 licenses-binary/LICENSE-slf4j.txt      |  21 ++++
 7 files changed, 334 insertions(+), 182 deletions(-)

diff --git a/build/make-distribution.sh b/build/make-distribution.sh
new file mode 100755
index 00000000..c96ff3d0
--- /dev/null
+++ b/build/make-distribution.sh
@@ -0,0 +1,224 @@
+#!/usr/bin/env 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 -o pipefail
+set -e
+set -x
+
+PROJECT_DIR="$(cd "`dirname "$0"`/.."; pwd)"
+DIST_DIR="$PROJECT_DIR/dist"
+NAME="bin"
+RELEASE="false"
+
+function exit_with_usage {
+  echo "make-distribution.sh - tool for making binary distributions of Celeborn"
+  echo ""
+  echo "usage:"
+  cl_options="[--name <custom_name>] [--release]"
+  echo "make-distribution.sh $cl_options <maven build options>"
+  echo ""
+  exit 1
+}
+
+# Parse arguments
+while (( "$#" )); do
+  case $1 in
+    --name)
+      NAME="bin-$2"
+      shift
+      ;;
+    --release)
+      RELEASE="true"
+      ;;
+    --help)
+      exit_with_usage
+      ;;
+    --*)
+      echo "Error: $1 is not supported"
+      exit_with_usage
+      ;;
+    -*)
+      break
+      ;;
+    *)
+      echo "Error: $1 is not supported"
+      exit_with_usage
+      ;;
+  esac
+  shift
+done
+
+if [ -z "$JAVA_HOME" ]; then
+  # Fall back on JAVA_HOME from rpm, if found
+  if [ $(command -v rpm) ]; then
+    RPM_JAVA_HOME="$(rpm -E %java_home 2>/dev/null)"
+    if [ "$RPM_JAVA_HOME" != "%java_home" ]; then
+      JAVA_HOME="$RPM_JAVA_HOME"
+      echo "No JAVA_HOME set, proceeding with '$JAVA_HOME' learned from rpm"
+    fi
+  fi
+
+  if [ -z "$JAVA_HOME" ]; then
+    if [ `command -v java` ]; then
+      # If java is in /usr/bin/java, we want /usr
+      JAVA_HOME="$(dirname $(dirname $(which java)))"
+    fi
+  fi
+fi
+
+if [ -z "$JAVA_HOME" ]; then
+  echo "Error: JAVA_HOME is not set, cannot proceed."
+  exit -1
+fi
+
+MVN="$PROJECT_DIR/build/mvn"
+export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=1g}"
+
+if [ ! "$(command -v "$MVN")" ] ; then
+    echo -e "Could not locate Maven command: '$MVN'."
+    exit -1;
+fi
+
+if [ $(command -v git) ]; then
+    GITREV=$(git rev-parse --short HEAD 2>/dev/null || :)
+    if [ ! -z "$GITREV" ]; then
+        GITREVSTRING=" (git revision $GITREV)"
+    fi
+    unset GITREV
+fi
+
+cd "$PROJECT_DIR"
+
+# Make directories
+rm -rf "$DIST_DIR"
+mkdir -p "$DIST_DIR"
+
+function build_service {
+  VERSION=$("$MVN" help:evaluate -Dexpression=project.version $@ 2>/dev/null \
+      | grep -v "INFO" \
+      | grep -v "WARNING" \
+      | tail -n 1)
+  SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null \
+      | grep -v "INFO" \
+      | grep -v "WARNING" \
+      | tail -n 1)
+
+  echo "Celeborn version is $VERSION"
+  echo "Making apache-celeborn-$VERSION-$NAME.tgz"
+
+  echo "Celeborn $VERSION$GITREVSTRING" > "$DIST_DIR/RELEASE"
+  echo "Build flags: $@" >> "$DIST_DIR/RELEASE"
+
+  # Store the command as an array because $MVN variable might have spaces in it.
+  # Normal quoting tricks don't work.
+  # See: http://mywiki.wooledge.org/BashFAQ/050
+  BUILD_COMMAND=("$MVN" clean package -DskipTests $@)
+
+  # Actually build the jar
+  echo -e "\nBuilding with..."
+  echo -e "\$ ${BUILD_COMMAND[@]}\n"
+
+  "${BUILD_COMMAND[@]}"
+
+  mkdir -p "$DIST_DIR/jars"
+  mkdir -p "$DIST_DIR/master-jars"
+  mkdir -p "$DIST_DIR/worker-jars"
+
+  ## Copy master jars
+  cp "$PROJECT_DIR"/master/target/celeborn-master_$SCALA_VERSION-$VERSION.jar "$DIST_DIR/master-jars/"
+  cp "$PROJECT_DIR"/master/target/scala-$SCALA_VERSION/jars/*.jar "$DIST_DIR/jars/"
+  for jar in $(ls "$PROJECT_DIR/master/target/scala-$SCALA_VERSION/jars"); do
+    (cd $DIST_DIR/master-jars; ln -snf "../jars/$jar" .)
+  done
+  ## Copy worker jars
+  cp "$PROJECT_DIR"/worker/target/celeborn-worker_$SCALA_VERSION-$VERSION.jar "$DIST_DIR/worker-jars/"
+  cp "$PROJECT_DIR"/worker/target/scala-$SCALA_VERSION/jars/*.jar "$DIST_DIR/jars/"
+  for jar in $(ls "$PROJECT_DIR/worker/target/scala-$SCALA_VERSION/jars"); do
+    (cd $DIST_DIR/worker-jars; ln -snf "../jars/$jar" .)
+  done
+}
+
+function build_spark_client {
+  VERSION=$("$MVN" help:evaluate -Dexpression=project.version $@ 2>/dev/null \
+      | grep -v "INFO" \
+      | grep -v "WARNING" \
+      | tail -n 1)
+  SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null \
+      | grep -v "INFO" \
+      | grep -v "WARNING" \
+      | tail -n 1)
+  SPARK_VERSION=$("$MVN" help:evaluate -Dexpression=spark.version $@ 2>/dev/null \
+      | grep -v "INFO" \
+      | grep -v "WARNING" \
+      | tail -n 1)
+  SPARK_MAJOR_VERSION=${SPARK_VERSION%%.*}
+
+  # Store the command as an array because $MVN variable might have spaces in it.
+  # Normal quoting tricks don't work.
+  # See: http://mywiki.wooledge.org/BashFAQ/050
+  BUILD_COMMAND=("$MVN" clean package -DskipTests -pl :celeborn-client-spark-${SPARK_MAJOR_VERSION}-shaded_$SCALA_VERSION -am $@)
+
+  # Actually build the jar
+  echo -e "\nBuilding with..."
+  echo -e "\$ ${BUILD_COMMAND[@]}\n"
+
+  "${BUILD_COMMAND[@]}"
+
+  mkdir -p "$DIST_DIR/spark"
+
+  ## Copy spark client jars
+  cp "$PROJECT_DIR"/client-spark/spark-$SPARK_MAJOR_VERSION-shaded/target/celeborn-client-spark-${SPARK_MAJOR_VERSION}-shaded_$SCALA_VERSION-$VERSION.jar "$DIST_DIR/spark/"
+}
+
+if [ "$RELEASE" == "true" ]; then
+  build_service
+  build_spark_client -Pspark-2.4
+  build_spark_client -Pspark-3.3
+else
+  build_service $@
+  build_spark_client $@
+fi
+
+# Copy configuration templates
+mkdir "$DIST_DIR/conf"
+cp "$PROJECT_DIR"/conf/*.template "$DIST_DIR/conf"
+
+# Copy shell scripts
+cp -r "$PROJECT_DIR/bin" "$DIST_DIR"
+cp -r "$PROJECT_DIR/sbin" "$DIST_DIR"
+
+# Copy container related resources
+mkdir "$DIST_DIR/docker"
+cp "$PROJECT_DIR/docker/Dockerfile" "$DIST_DIR/docker"
+cp -r "$PROJECT_DIR/docker/helm" "$DIST_DIR/docker"
+
+# Copy license files
+cp "$PROJECT_DIR/DISCLAIMER" "$DIST_DIR/DISCLAIMER"
+if [[ -f $"$PROJECT_DIR/LICENSE-binary" ]]; then
+  cp "$PROJECT_DIR/LICENSE-binary" "$DIST_DIR/LICENSE"
+  cp -r "$PROJECT_DIR/licenses-binary" "$DIST_DIR/licenses"
+  cp "$PROJECT_DIR/NOTICE-binary" "$DIST_DIR/NOTICE"
+fi
+
+# Create tarball
+TARDIR_NAME="apache-celeborn-$VERSION-$NAME"
+TARDIR="$PROJECT_DIR/$TARDIR_NAME"
+rm -rf "$TARDIR"
+cp -R "$DIST_DIR" "$TARDIR"
+tar czf "apache-celeborn-$VERSION-$NAME.tgz" -C "$PROJECT_DIR" "$TARDIR_NAME"
+rm -rf "$TARDIR"
diff --git a/build/release/create-package.sh b/build/release/create-package.sh
index c148fef9..9b407155 100755
--- a/build/release/create-package.sh
+++ b/build/release/create-package.sh
@@ -69,8 +69,22 @@ package_source() {
 }
 
 package_binary() {
-  echo "TODO"
-  exit 1
+  BIN_TGZ_FILE="apache-celeborn-${RELEASE_VERSION}-bin.tgz"
+  BIN_TGZ="${RELEASE_DIR}/${BIN_TGZ_FILE}"
+
+  mkdir -p "${RELEASE_DIR}"
+  rm -f "${BIN_TGZ}*"
+
+  echo "Creating binary release tarball ${BIN_TGZ_FILE}"
+
+  ${PROJECT_DIR}/build/make-distribution.sh --release
+
+  cp "${BIN_TGZ_FILE}" "${RELEASE_DIR}"
+
+  if [ "$SKIP_GPG" == "false" ] ; then
+    gpg --armor --detach-sig "${BIN_TGZ}"
+  fi
+  (cd "${RELEASE_DIR}" && $SHASUM "${BIN_TGZ_FILE}" > "${BIN_TGZ_FILE}.sha512")
 }
 
 if [[ "$1" == "source" ]]; then
diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh
deleted file mode 100755
index ef62d0c0..00000000
--- a/dev/make-distribution.sh
+++ /dev/null
@@ -1,176 +0,0 @@
-#!/usr/bin/env 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 -o pipefail
-set -e
-set -x
-
-# Figure out where the RSS framework is installed
-PROJECT_DIR="$(cd "`dirname "$0"`/.."; pwd)"
-DIST_DIR="$PROJECT_DIR/dist"
-NAME="bin"
-
-function exit_with_usage {
-  echo "make-distribution.sh - tool for making binary distributions of Celeborn"
-  echo ""
-  echo "usage:"
-  cl_options="[--name <custom_name>]"
-  echo "make-distribution.sh $cl_options <maven build options>"
-  echo ""
-  exit 1
-}
-
-# Parse arguments
-while (( "$#" )); do
-  case $1 in
-    --name)
-      NAME="bin-$2"
-      shift
-      ;;
-    --help)
-      exit_with_usage
-      ;;
-    --*)
-      echo "Error: $1 is not supported"
-      exit_with_usage
-      ;;
-    -*)
-      break
-      ;;
-    *)
-      echo "Error: $1 is not supported"
-      exit_with_usage
-      ;;
-  esac
-  shift
-done
-
-MVN="$PROJECT_DIR/build/mvn"
-
-if [ -z "$JAVA_HOME" ]; then
-  # Fall back on JAVA_HOME from rpm, if found
-  if [ $(command -v rpm) ]; then
-    RPM_JAVA_HOME="$(rpm -E %java_home 2>/dev/null)"
-    if [ "$RPM_JAVA_HOME" != "%java_home" ]; then
-      JAVA_HOME="$RPM_JAVA_HOME"
-      echo "No JAVA_HOME set, proceeding with '$JAVA_HOME' learned from rpm"
-    fi
-  fi
-
-  if [ -z "$JAVA_HOME" ]; then
-    if [ `command -v java` ]; then
-      # If java is in /usr/bin/java, we want /usr
-      JAVA_HOME="$(dirname $(dirname $(which java)))"
-    fi
-  fi
-fi
-
-if [ -z "$JAVA_HOME" ]; then
-  echo "Error: JAVA_HOME is not set, cannot proceed."
-  exit -1
-fi
-
-if [ ! "$(command -v "$MVN")" ] ; then
-    echo -e "Could not locate Maven command: '$MVN'."
-    exit -1;
-fi
-
-if [ $(command -v git) ]; then
-    GITREV=$(git rev-parse --short HEAD 2>/dev/null || :)
-    if [ ! -z "$GITREV" ]; then
-        GITREVSTRING=" (git revision $GITREV)"
-    fi
-    unset GITREV
-fi
-
-VERSION=$("$MVN" help:evaluate -Dexpression=project.version $@ 2>/dev/null\
-    | grep -v "INFO"\
-    | grep -v "WARNING"\
-    | tail -n 1)
-SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version $@ 2>/dev/null\
-    | grep -v "INFO"\
-    | grep -v "WARNING"\
-    | tail -n 1)
-SPARK_VERSION=$("$MVN" help:evaluate -Dexpression=spark.version $@ 2>/dev/null\
-    | grep -v "INFO"\
-    | grep -v "WARNING"\
-    | tail -n 1)
-
-SPARK_MAJOR_VERSION=${SPARK_VERSION%%.*}
-
-echo "Celeborn version is $VERSION"
-
-echo "Making apache-celeborn-$VERSION-$NAME.tgz"
-
-# Build uber fat JAR
-cd "$PROJECT_DIR"
-
-export MAVEN_OPTS="${MAVEN_OPTS:--Xmx2g -XX:ReservedCodeCacheSize=1g}"
-
-# Store the command as an array because $MVN variable might have spaces in it.
-# Normal quoting tricks don't work.
-# See: http://mywiki.wooledge.org/BashFAQ/050
-BUILD_COMMAND=("$MVN" clean package -DskipTests $@)
-
-# Actually build the jar
-echo -e "\nBuilding with..."
-echo -e "\$ ${BUILD_COMMAND[@]}\n"
-
-"${BUILD_COMMAND[@]}"
-
-# Make directories
-rm -rf "$DIST_DIR"
-mkdir -p "$DIST_DIR/jars"
-mkdir -p "$DIST_DIR/master-jars"
-mkdir -p "$DIST_DIR/worker-jars"
-mkdir -p "$DIST_DIR/spark"
-
-echo "Celeborn $VERSION$GITREVSTRING" > "$DIST_DIR/RELEASE"
-echo "Build flags: $@" >> "$DIST_DIR/RELEASE"
-
-# Copy jars
-## Copy master jars
-cp "$PROJECT_DIR"/master/target/celeborn-master_$SCALA_VERSION-$VERSION.jar "$DIST_DIR/master-jars/"
-cp "$PROJECT_DIR"/master/target/scala-$SCALA_VERSION/jars/*.jar "$DIST_DIR/jars/"
-for jar in $(ls "$PROJECT_DIR/master/target/scala-$SCALA_VERSION/jars"); do
-  (cd $DIST_DIR/master-jars; ln -snf "../jars/$jar" .)
-done
-## Copy worker jars
-cp "$PROJECT_DIR"/worker/target/celeborn-worker_$SCALA_VERSION-$VERSION.jar "$DIST_DIR/worker-jars/"
-cp "$PROJECT_DIR"/worker/target/scala-$SCALA_VERSION/jars/*.jar "$DIST_DIR/jars/"
-for jar in $(ls "$PROJECT_DIR/worker/target/scala-$SCALA_VERSION/jars"); do
-  (cd $DIST_DIR/worker-jars; ln -snf "../jars/$jar" .)
-done
-## Copy spark client jars
-cp "$PROJECT_DIR"/client-spark/spark-$SPARK_MAJOR_VERSION-shaded/target/celeborn-client-spark-${SPARK_MAJOR_VERSION}-shaded_$SCALA_VERSION-$VERSION.jar "$DIST_DIR/spark/"
-
-# Copy other things
-mkdir "$DIST_DIR/conf"
-cp "$PROJECT_DIR"/conf/*.template "$DIST_DIR/conf"
-cp -r "$PROJECT_DIR/bin" "$DIST_DIR"
-cp -r "$PROJECT_DIR/sbin" "$DIST_DIR"
-mkdir "$DIST_DIR/docker"
-cp "$PROJECT_DIR/docker/Dockerfile" "$DIST_DIR/docker"
-cp -r "$PROJECT_DIR/docker/helm" "$DIST_DIR/docker"
-
-TARDIR_NAME="apache-celeborn-$VERSION-$NAME"
-TARDIR="$PROJECT_DIR/$TARDIR_NAME"
-rm -rf "$TARDIR"
-cp -R "$DIST_DIR" "$TARDIR"
-tar czf "apache-celeborn-$VERSION-$NAME.tgz" -C "$PROJECT_DIR" "$TARDIR_NAME"
-rm -rf "$TARDIR"
diff --git a/dev/reformat b/dev/reformat
index 901dc9ca..420d1361 100755
--- a/dev/reformat
+++ b/dev/reformat
@@ -18,7 +18,7 @@
 
 set -x
 
-PROJECT_HOME="$(cd "`dirname "$0"`/.."; pwd)"
-${PROJECT_HOME}/build/mvn spotless:apply -Pflink-1.14
-${PROJECT_HOME}/build/mvn spotless:apply -Pspark-2.4
-${PROJECT_HOME}/build/mvn spotless:apply -Pspark-3.3
+PROJECT_DIR="$(cd "`dirname "$0"`/.."; pwd)"
+${PROJECT_DIR}/build/mvn spotless:apply -Pflink-1.14
+${PROJECT_DIR}/build/mvn spotless:apply -Pspark-2.4
+${PROJECT_DIR}/build/mvn spotless:apply -Pspark-3.3
diff --git a/licenses-binary/LICENSE-leveldbjni.txt b/licenses-binary/LICENSE-leveldbjni.txt
new file mode 100644
index 00000000..b4dabb91
--- /dev/null
+++ b/licenses-binary/LICENSE-leveldbjni.txt
@@ -0,0 +1,27 @@
+Copyright (c) 2011 FuseSource Corp. All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+   * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+   * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+   * Neither the name of FuseSource Corp. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
\ No newline at end of file
diff --git a/licenses-binary/LICENSE-protobuf.txt b/licenses-binary/LICENSE-protobuf.txt
new file mode 100644
index 00000000..b4350ec8
--- /dev/null
+++ b/licenses-binary/LICENSE-protobuf.txt
@@ -0,0 +1,42 @@
+This license applies to all parts of Protocol Buffers except the following:
+
+  - Atomicops support for generic gcc, located in
+    src/google/protobuf/stubs/atomicops_internals_generic_gcc.h.
+    This file is copyrighted by Red Hat Inc.
+
+  - Atomicops support for AIX/POWER, located in
+    src/google/protobuf/stubs/atomicops_internals_aix.h.
+    This file is copyrighted by Bloomberg Finance LP.
+
+Copyright 2014, Google Inc.  All rights reserved.
+
+Redistribution and use in source and binary forms, with or without
+modification, are permitted provided that the following conditions are
+met:
+
+    * Redistributions of source code must retain the above copyright
+notice, this list of conditions and the following disclaimer.
+    * Redistributions in binary form must reproduce the above
+copyright notice, this list of conditions and the following disclaimer
+in the documentation and/or other materials provided with the
+distribution.
+    * Neither the name of Google Inc. nor the names of its
+contributors may be used to endorse or promote products derived from
+this software without specific prior written permission.
+
+THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
+"AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
+LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
+A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
+OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
+SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
+LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
+DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
+THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
+(INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
+OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
+
+Code generated by the Protocol Buffer compiler is owned by the owner
+of the input file used when generating it.  This code is not
+standalone and requires a support library to be linked with it.  This
+support library is itself covered by the above license.
\ No newline at end of file
diff --git a/licenses-binary/LICENSE-slf4j.txt b/licenses-binary/LICENSE-slf4j.txt
new file mode 100644
index 00000000..28ba154f
--- /dev/null
+++ b/licenses-binary/LICENSE-slf4j.txt
@@ -0,0 +1,21 @@
+Copyright (c) 2004-2013 QOS.ch
+ All rights reserved.
+
+ Permission is hereby granted, free  of charge, to any person obtaining
+ a  copy  of this  software  and  associated  documentation files  (the
+ "Software"), to  deal in  the Software without  restriction, including
+ without limitation  the rights to  use, copy, modify,  merge, publish,
+ distribute,  sublicense, and/or sell  copies of  the Software,  and to
+ permit persons to whom the Software  is furnished to do so, subject to
+ the following conditions:
+
+ The  above  copyright  notice  and  this permission  notice  shall  be
+ included in all copies or substantial portions of the Software.
+
+ THE  SOFTWARE IS  PROVIDED  "AS  IS", WITHOUT  WARRANTY  OF ANY  KIND,
+ EXPRESS OR  IMPLIED, INCLUDING  BUT NOT LIMITED  TO THE  WARRANTIES OF
+ MERCHANTABILITY,    FITNESS    FOR    A   PARTICULAR    PURPOSE    AND
+ NONINFRINGEMENT. IN NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE
+ LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, WHETHER IN AN ACTION
+ OF CONTRACT, TORT OR OTHERWISE,  ARISING FROM, OUT OF OR IN CONNECTION
+ WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE.