You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by aa...@apache.org on 2016/06/16 05:04:54 UTC

hadoop git commit: HADOOP-12892. fix/rewrite create-release. Contributed by Allen Wittenauer.

Repository: hadoop
Updated Branches:
  refs/heads/branch-2 32b115da1 -> d1c475ddd


HADOOP-12892. fix/rewrite create-release. Contributed by Allen Wittenauer.


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

Branch: refs/heads/branch-2
Commit: d1c475ddd4c6c5e90f75eefb94bd05374a13d845
Parents: 32b115d
Author: Akira Ajisaka <aa...@apache.org>
Authored: Thu Jun 16 14:02:30 2016 +0900
Committer: Akira Ajisaka <aa...@apache.org>
Committed: Thu Jun 16 14:03:57 2016 +0900

----------------------------------------------------------------------
 dev-support/bin/create-release                  | 624 +++++++++++++++++++
 dev-support/bin/dist-copynativelibs             | 156 +++++
 dev-support/create-release.sh                   | 144 -----
 dev-support/docker/Dockerfile                   |  43 +-
 .../main/resources/assemblies/hadoop-dist.xml   |   2 +-
 hadoop-dist/pom.xml                             |   2 +
 hadoop-hdfs-project/hadoop-hdfs-client/pom.xml  |   4 +
 .../hadoop-hdfs-native-client/pom.xml           |   3 +-
 .../src/CMakeLists.txt                          |  12 +-
 hadoop-project-dist/pom.xml                     | 125 +---
 10 files changed, 857 insertions(+), 258 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/dev-support/bin/create-release
----------------------------------------------------------------------
diff --git a/dev-support/bin/create-release b/dev-support/bin/create-release
new file mode 100755
index 0000000..8d7f0a8
--- /dev/null
+++ b/dev-support/bin/create-release
@@ -0,0 +1,624 @@
+#!/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.
+
+if [[ -z "${BASH_VERSINFO[0]}" ]] \
+   || [[ "${BASH_VERSINFO[0]}" -lt 3 ]] \
+   || [[ "${BASH_VERSINFO[0]}" -eq 3 && "${BASH_VERSINFO[1]}" -lt 2 ]]; then
+  echo "bash v3.2+ is required. Sorry."
+  exit 1
+fi
+
+function centered_text
+{
+  local text="$*"
+  local spacing=$(( (75+${#text}) /2 ))
+  printf "%*s\n"  ${spacing} "${text}"
+}
+
+function big_console_header
+{
+  printf "\n\n"
+  echo "****************************************************************************"
+  centered_text "${@}"
+  echo "****************************************************************************"
+  printf "\n\n"
+}
+
+## @description  Given a filename or dir, return the absolute version of it
+## @audience     public
+## @stability    stable
+## @param        directory
+## @replaceable  no
+## @return       0 success
+## @return       1 failure
+## @return       stdout abspath
+function hadoop_abs
+{
+  declare obj=$1
+  declare dir
+  declare fn
+
+  if [[ ! -e ${obj} ]]; then
+    return 1
+  elif [[ -d ${obj} ]]; then
+    dir=${obj}
+  else
+    dir=$(dirname -- "${obj}")
+    fn=$(basename -- "${obj}")
+    fn="/${fn}"
+  fi
+
+  dir=$(cd -P -- "${dir}" >/dev/null 2>/dev/null && pwd -P)
+  if [[ $? = 0 ]]; then
+    echo "${dir}${fn}"
+    return 0
+  fi
+  return 1
+}
+
+## @description  Print a message to stderr
+## @audience     public
+## @stability    stable
+## @replaceable  no
+## @param        string
+function hadoop_error
+{
+  echo "$*" 1>&2
+}
+
+
+function run_and_redirect
+{
+  declare logfile=$1
+  shift
+  declare res
+
+  echo "\$ ${*} > ${logfile} 2>&1"
+  # to the log
+  {
+    date
+    echo "cd $(pwd)"
+    echo "${*}"
+  } > "${logfile}"
+  # run the actual command
+  "${@}" >> "${logfile}" 2>&1
+  res=$?
+  if [[ ${res} != 0 ]]; then
+    echo
+    echo "Failed!"
+    echo
+    exit "${res}"
+  fi
+}
+
+function hadoop_native_flags
+{
+
+  # modified version of the Yetus personality
+
+  if [[ ${NATIVE} != true ]]; then
+    return
+  fi
+
+  # Based upon HADOOP-11937
+  #
+  # Some notes:
+  #
+  # - getting fuse to compile on anything but Linux
+  #   is always tricky.
+  # - Darwin assumes homebrew is in use.
+  # - HADOOP-12027 required for bzip2 on OS X.
+  # - bzip2 is broken in lots of places.
+  #   e.g, HADOOP-12027 for OS X. so no -Drequire.bzip2
+  #
+
+  case "${OSNAME}" in
+    Linux)
+      # shellcheck disable=SC2086
+      echo -Pnative -Drequire.snappy -Drequire.openssl -Drequire.fuse
+    ;;
+    Darwin)
+      echo \
+      -Pnative -Drequire.snappy  \
+      -Drequire.openssl \
+        -Dopenssl.prefix=/usr/local/opt/openssl/ \
+        -Dopenssl.include=/usr/local/opt/openssl/include \
+        -Dopenssl.lib=/usr/local/opt/openssl/lib
+    ;;
+    *)
+      # shellcheck disable=SC2086
+      echo \
+        -Pnative \
+        -Drequire.snappy -Drequire.openssl \
+        -Drequire.test.libhadoop
+    ;;
+  esac
+}
+
+# Function to probe the exit code of the script commands,
+# and stop in the case of failure with an contextual error
+# message.
+function run()
+{
+  declare res
+  declare logfile
+
+  echo "\$ ${*}"
+  "${@}"
+  res=$?
+  if [[ ${res} != 0 ]]; then
+    echo
+    echo "Failed!"
+    echo
+    exit "${res}"
+  fi
+}
+
+function domd5()
+{
+  run "${MD5SUM}" "${1}" > "${1}.md5"
+}
+
+function header()
+{
+  echo
+  printf "\n\n"
+  echo "============================================================================"
+  echo "============================================================================"
+  centered_text "Hadoop Release Creator"
+  echo "============================================================================"
+  echo "============================================================================"
+  printf "\n\n"
+  echo "Version to create      : ${HADOOP_VERSION}"
+  echo "Release Candidate Label: ${RC_LABEL##-}"
+  echo "Source Version         : ${DEFAULT_HADOOP_VERSION}"
+  printf "\n\n"
+}
+
+function set_defaults
+{
+  BINDIR=$(dirname "${BIN}")
+  BASEDIR=$(hadoop_abs "${BINDIR}/../..")
+
+  ARTIFACTS_DIR="${BASEDIR}/target/artifacts"
+
+  # Extract Hadoop version from ${BASEDIR}/pom.xml
+  DEFAULT_HADOOP_VERSION=$(grep "<version>" "${BASEDIR}/pom.xml" \
+    | head -1 \
+    | sed  -e 's|^ *<version>||' -e 's|</version>.*$||')
+
+  DOCKER=false
+  DOCKERCACHE=false
+  DOCKERFILE="${BASEDIR}/dev-support/docker/Dockerfile"
+  DOCKERRAN=false
+
+  # Extract Java version from ${BASEDIR}/pom.xml
+  # doing this outside of maven means we can do this before
+  # the docker container comes up...
+  JVM_VERSION=$(grep "<javac.version>" "${BASEDIR}/hadoop-project/pom.xml" \
+    | head -1 \
+    | sed  -e 's|^ *<javac.version>||' -e 's|</javac.version>.*$||' -e 's|..||')
+
+  GIT=$(command -v git)
+
+  GPG=$(command -v gpg)
+  GPGAGENT=$(command -v gpg-agent)
+
+  HADOOP_VERSION="${DEFAULT_HADOOP_VERSION}"
+
+  INDOCKER=false
+
+  LOGDIR="${BASEDIR}/patchprocess"
+
+  if [[ -z "${MVN}" ]]; then
+    if [[ -n "${MAVEN_HOME}" ]]; then
+      MVN=${MAVEN_HOME}/bin/mvn
+    else
+      MVN=$(command -v mvn)
+    fi
+  fi
+
+  MD5SUM=$(command -v md5sum)
+  if [[ -z "${MD5SUM}" ]]; then
+    MD5SUM=$(command -v md5)
+  fi
+
+  NATIVE=false
+  OSNAME=$(uname -s)
+
+  PUBKEYFILE="https://dist.apache.org/repos/dist/release/hadoop/common/KEYS"
+}
+
+function startgpgagent
+{
+  if [[ "${SIGN}" = true ]]; then
+    if [[ -n "${GPGAGENT}" && -z "${GPG_AGENT_INFO}" ]]; then
+      echo "starting gpg agent"
+      touch "${LOGDIR}/gpgagent.conf"
+      eval $("${GPGAGENT}" --daemon \
+        --options "${LOGDIR}/gpgagent.conf" \
+        --log-file=${LOGDIR}/create-release-gpgagent.log)
+      GPGAGENTPID=$(echo ${GPG_AGENT_INFO} | cut -f 2 -d:)
+    fi
+  fi
+}
+
+function stopgpgagent
+{
+  if [[ -n "${GPGAGENTPID}" ]]; then
+    kill ${GPGAGENTPID}
+  fi
+}
+
+function usage
+{
+  echo "--artifactsdir=[path]   Path to use to store release bits"
+  echo "--asfrelease             Make an ASF release"
+  echo "--docker                Use Hadoop's Dockerfile for guaranteed environment"
+  echo "--dockercache           Use a Docker-private maven cache"
+  echo "--logdir=[path]         Path to store logs"
+  echo "--mvncache=[path]       Path to the maven cache to use"
+  echo "--native                Also build the native components"
+  echo "--rc-label=[label]      Add this label to the builds"
+  echo "--sign                  Use .gnupg dir to sign the jars"
+  echo "--version=[version]     Use an alternative version string"
+}
+
+function option_parse
+{
+  declare i
+
+  for i in "$@"; do
+    case ${i} in
+      --asfrelease)
+        ASFRELEASE=true
+        NATIVE=true
+        SIGN=true
+      ;;
+      --artifactsdir=*)
+        ARTIFACTS_DIR=${i#*=}
+      ;;
+      --docker)
+        DOCKER=true
+      ;;
+      --dockercache)
+        DOCKERCACHE=true
+      ;;
+      --help)
+        usage
+        exit
+      ;;
+      --indocker)
+        INDOCKER=true
+      ;;
+      --logdir=*)
+        LOGDIR=${i#*=}
+      ;;
+      --mvncache=*)
+        MVNCACHE=${i#*=}
+      ;;
+      --native)
+        NATIVE=true
+      ;;
+      --rc-label=*)
+        RC_LABEL=${i#*=}
+      ;;
+      --sign)
+        SIGN=true
+      ;;
+      --version=*)
+        HADOOP_VERSION=${i#*=}
+      ;;
+    esac
+  done
+
+  if [[ ! -d "${HOME}/.gnupg" ]]; then
+    hadoop_error "ERROR: No .gnupg dir. Disabling signing capability."
+    SIGN=false
+  fi
+
+  DOCKERCMD=$(command -v docker)
+  if [[ "${DOCKER}" = true && -z "${DOCKERCMD}" ]]; then
+      hadoop_error "ERROR: docker binary not found. Disabling docker mode."
+      DOCKER=false
+  fi
+
+  if [[ "${DOCKERCACHE}" = true && "${DOCKER}" = false ]]; then
+    if [[ "${INDOCKER}" = false ]]; then
+      hadoop_error "ERROR: docker mode not enabled. Disabling dockercache."
+    fi
+    DOCKERCACHE=false
+  fi
+
+  if [[ "${DOCKERCACHE}" = true && -n "${MVNCACHE}" ]]; then
+    hadoop_error "ERROR: Cannot set --mvncache and --dockercache simultaneously."
+    exit 1
+  else
+    MVNCACHE=${MVNCACHE:-"${HOME}/.m2"}
+  fi
+
+  if [[ "${ASFRELEASE}" = true ]]; then
+    if [[ "${SIGN}" = false ]]; then
+      hadoop_error "ERROR: --asfrelease requires --sign. Exiting."
+      exit 1
+    fi
+
+    if [[ "${OSNAME}" = Linux ]]; then
+      if [[ "${DOCKER}" = false && "${INDOCKER}" = false ]]; then
+        hadoop_error "ERROR: --asfrelease requires --docker on Linux. Exiting."
+        exit 1
+      elif [[ "${DOCKERCACHE}" = false && "${INDOCKER}" = false ]]; then
+        hadoop_error "ERROR: --asfrelease on Linux requires --dockercache. Exiting."
+        exit 1
+      fi
+    fi
+  fi
+
+  if [[ -n "${MVNCACHE}" ]]; then
+    mkdir -p "${MVNCACHE}"
+    if [[ -d "${MVNCACHE}" ]]; then
+      MVN_ARGS=("-Dmaven.repo.local=${MVNCACHE}")
+    fi
+  fi
+}
+
+function dockermode
+{
+  declare lines
+  declare -a modp
+  declare imgname
+  declare -a extrad
+  declare user_name
+  declare group_id
+
+  if [[ "${DOCKER}" != true ]]; then
+    return
+  fi
+
+  user_name=${SUDO_USER:=$USER}
+  user_id=$(id -u "${user_name}")
+  group_id=$(id -g "${user_name}")
+
+  imgname="hadoop/createrelease:${HADOOP_VERSION}_${RANDOM}"
+
+  if [[ -d "${HOME}/.gnupg" ]]; then
+    extrad+=("-v" "${HOME}/.gnupg:/home/${user_name}/.gnupg")
+  fi
+
+  if [[ -n "${LOGDIR}" ]]; then
+    if [[ ! -d "${LOGDIR}" ]]; then
+      mkdir -p "${LOGDIR}"
+    fi
+    lines=$(hadoop_abs "${LOGDIR}")
+    extrad+=("-v" "${lines}:${lines}")
+  fi
+
+  if [[ -n "${ARTIFACTS_DIR}" ]]; then
+    if [[ ! -d "${ARTIFACTS_DIR}" ]]; then
+      mkdir -p "${ARTIFACTS_DIR}"
+    fi
+    lines=$(hadoop_abs "${ARTIFACTS_DIR}")
+    extrad+=("-v" "${lines}:${lines}")
+  fi
+
+  if [[ "${DOCKERCACHE}" = true ]]; then
+    modp+=("--mvncache=/maven")
+  else
+    lines=$(hadoop_abs "${MVNCACHE}")
+    extrad+=("-v" "${lines}:${lines}")
+  fi
+
+  for lines in "${PARAMS[@]}"; do
+    if [[ "${lines}" != "--docker" ]]; then
+      modp+=("$lines")
+    fi
+  done
+
+  modp+=("--indocker")
+
+  (
+    lines=$(grep -n 'YETUS CUT HERE' "${DOCKERFILE}" | cut -f1 -d:)
+    if [[ -z "${lines}" ]]; then
+      cat "${DOCKERFILE}"
+    else
+      head -n "${lines}" "${DOCKERFILE}"
+    fi
+    # make sure we put some space between, just in case last
+    # line isn't an empty line or whatever
+    printf "\n\n"
+    echo "RUN groupadd --non-unique -g ${group_id} ${user_name}"
+    echo "RUN useradd -g ${group_id} -u ${user_id} -m ${user_name}"
+    echo "RUN chown -R ${user_name} /home/${user_name}"
+    echo "ENV HOME /home/${user_name}"
+    echo "RUN mkdir -p /maven"
+    echo "RUN chown -R ${user_name} /maven"
+
+    # we always force build with the Oracle JDK
+    # but with the correct version
+    echo "ENV JAVA_HOME /usr/lib/jvm/java-${JVM_VERSION}-oracle"
+    echo "USER ${user_name}"
+    printf "\n\n"
+  ) | docker build -t "${imgname}" -
+
+  run docker run -i -t \
+    --privileged \
+    "${extrad[@]}"  \
+    -v "${BASEDIR}:/build/source" \
+    -u "${user_name}" \
+    -w "/build/source" \
+    "${imgname}" \
+    "/build/source/dev-support/bin/create-release" "${modp[@]}"
+
+  DOCKERRAN=true
+}
+
+function makearelease
+{
+  # let's start at the root
+  run cd "${BASEDIR}"
+
+  big_console_header "Cleaning the Source Tree"
+
+  # git clean to clear any remnants from previous build
+  run "${GIT}" clean -xdf
+
+  mkdir -p "${LOGDIR}"
+
+  # mvn clean for sanity
+  run_and_redirect "${LOGDIR}/mvn_clean.log" "${MVN}" "${MVN_ARGS[@]}" clean
+
+  # Create staging dir for release artifacts
+  run mkdir -p "${ARTIFACTS_DIR}"
+
+  big_console_header "Apache RAT Check"
+
+  # Create RAT report
+  run_and_redirect "${LOGDIR}/mvn_apache_rat.log" "${MVN}" "${MVN_ARGS[@]}" apache-rat:check
+
+  big_console_header "Maven Build and Install"
+
+  # Create SRC and BIN tarballs for release,
+  # Using 'install\u2019 goal instead of 'package' so artifacts are available
+  # in the Maven local cache for the site generation
+  #
+  # shellcheck disable=SC2046
+  run_and_redirect "${LOGDIR}/mvn_install.log" \
+    "${MVN}" "${MVN_ARGS[@]}" install -Pdist,src \
+      -DskipTests -Dtar $(hadoop_native_flags)
+
+  big_console_header "Maven Site"
+
+  # Create site for release
+  run_and_redirect "${LOGDIR}/mvn_site.log" "${MVN}" "${MVN_ARGS[@]}" site site:stage -Pdist,src,releasedocs
+
+  big_console_header "Staging the release"
+
+  run mv "${BASEDIR}/target/staging/hadoop-project" "${BASEDIR}/target/r${HADOOP_VERSION}/"
+  run cd "${BASEDIR}/target/"
+  run tar czpf "hadoop-site-${HADOOP_VERSION}.tar.gz" "r${HADOOP_VERSION}"/*
+  run cd "${BASEDIR}"
+
+  # Stage RAT report
+  #shellcheck disable=SC2038
+  find . -name rat.txt | xargs -I% cat % > "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-rat.txt"
+
+  # Stage CHANGES and RELEASENOTES files
+  for i in CHANGES RELEASENOTES; do
+    run cp -p \
+        "${BASEDIR}/hadoop-common-project/hadoop-common/src/site/markdown/release/${HADOOP_VERSION}"/${i}*.md \
+        "${ARTIFACTS_DIR}/${i}.md"
+  done
+
+  # Prepare and stage BIN tarball
+  run cd "${BASEDIR}/hadoop-dist/target/"
+  run tar -xzpf "hadoop-${HADOOP_VERSION}.tar.gz"
+  run mkdir -p "hadoop-${HADOOP_VERSION}/share/doc/hadoop/"
+  run cp -r "${BASEDIR}/target/r${HADOOP_VERSION}"/* "hadoop-${HADOOP_VERSION}/share/doc/hadoop/"
+  run tar -czpf "hadoop-${HADOOP_VERSION}.tar.gz" "hadoop-${HADOOP_VERSION}"
+  run cd "${BASEDIR}"
+  run mv \
+    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
+
+  # Stage SRC tarball
+  run mv \
+    "${BASEDIR}/hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz"
+
+  # Stage SITE tarball
+  run mv \
+     "${BASEDIR}/target/hadoop-site-${HADOOP_VERSION}.tar.gz" \
+    "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-site.tar.gz"
+}
+
+function signartifacts
+{
+  declare i
+
+  if [[ "${SIGN}" = false ]]; then
+    for i in ${ARTIFACTS_DIR}/*; do
+      domd5 "${i}"
+    done
+    echo ""
+    echo "Remember to sign the artifacts before staging them on the open"
+    echo ""
+    return
+  fi
+
+  big_console_header "Signing the release"
+
+  for i in ${ARTIFACTS_DIR}/*; do
+    gpg --use-agent --armor --output "${i}.asc" --detach-sig "${i}"
+    gpg --print-mds "${i}" > "${i}.mds"
+    domd5 "${i}"
+  done
+
+  if [[ "${ASFRELEASE}" = true ]]; then
+    echo "Fetching the Apache Hadoop KEYS file..."
+    curl -L "${PUBKEYFILE}" -o "${BASEDIR}/target/KEYS"
+    gpg --import --trustdb "${BASEDIR}/target/testkeysdb" "${BASEDIR}/target/KEYS"
+    gpg --verify --trustdb "${BASEDIR}/target/testkeysdb" \
+      "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz.asc" \
+        "${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz"
+    if [[ $? != 0 ]]; then
+      hadoop_error "ERROR: GPG key is not present in ${PUBKEYFILE}."
+      hadoop_error "ERROR: This MUST be fixed. Exiting."
+      exit 1
+    fi
+  fi
+}
+
+# find root of the source tree
+BIN=$(hadoop_abs "${BASH_SOURCE:-$0}")
+PARAMS=("$@")
+
+set_defaults
+
+option_parse "${PARAMS[@]}"
+
+dockermode
+
+header
+
+if [[ -n ${RC_LABEL} ]]; then
+  RC_LABEL="-${RC_LABEL}"
+fi
+
+if [[ "${INDOCKER}" = true || "${DOCKERRAN}" = false ]]; then
+
+  startgpgagent
+
+  makearelease
+
+  signartifacts
+
+  stopgpgagent
+fi
+
+if [[ "${INDOCKER}" = true ]]; then
+  exit $?
+fi
+
+if [[ $? == 0 ]]; then
+  echo
+  echo "Congratulations, you have successfully built the release"
+  echo "artifacts for Apache Hadoop ${HADOOP_VERSION}${RC_LABEL}"
+  echo
+  echo "The artifacts for this run are available at ${ARTIFACTS_DIR}:"
+  run ls -1 "${ARTIFACTS_DIR}"
+
+  echo
+fi
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/dev-support/bin/dist-copynativelibs
----------------------------------------------------------------------
diff --git a/dev-support/bin/dist-copynativelibs b/dev-support/bin/dist-copynativelibs
new file mode 100755
index 0000000..61817ed
--- /dev/null
+++ b/dev-support/bin/dist-copynativelibs
@@ -0,0 +1,156 @@
+#!/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
+
+# Bundle a native library if requested. Exit 1 in case error happens.
+# Usage: bundle_native_lib bundleoption liboption libpattern libdir
+function bundle_native_lib()
+{
+  declare bundleoption="$1"
+  declare liboption="$2"
+  declare libpattern="$3"
+  declare libdir="$4"
+
+
+  echo "Checking to bundle with:"
+  echo "bundleoption=${bundleoption}, liboption=${liboption}, pattern=${libpattern} libdir=${libdir}"
+
+  if [[ "${bundleoption}" != "true" ]]; then
+    return
+  fi
+
+  if [[ -z "${libdir}" ]] || [[ ! -d "${libdir}" ]]; then
+    echo "The required option ${liboption} isn't given or invalid. Bundling the lib failed"
+    exit 1
+  fi
+
+  cd "${libdir}"  || exit 1
+  ${TAR} ./*"${libpattern}"* | (cd "${TARGET_DIR}"/ || exit 1; ${UNTAR})
+  if [[ $? -ne 0 ]]; then
+    echo "Bundling library with ${liboption} failed "
+    exit 1
+  fi
+}
+
+function bundle_native_bin
+{
+  declare bundleoption="$1"
+  declare libbundle="$2"
+  declare binoption="$3"
+  declare binpattern="$4"
+  declare libdir="$5"
+
+  echo "Checking to bundle with:"
+  echo "bundleoption=${bundleoption}, libbundle=${libbundle}, binoption=${binoption}, libdir=${libdir}, binpattern=${binpattern}"
+
+
+  if [[ "${bundleoption}" != "true" ]]; then
+    return
+  fi
+
+  if [[ "${libbundle}" != "true" ]]; then
+    return
+  fi
+
+  if [[ -z "${libdir}" ]] || [[ ! -d "${libdir}" ]]; then
+    echo "The required option ${liboption} isn't given or invalid. Bundling the lib failed"
+    exit 1
+  fi
+
+  cd "${libdir}" || exit 1
+  ${TAR} ./*"${libpattern}"* | (cd "${TARGET_BIN_DIR}"/  || exit 1 ; ${UNTAR})
+  if [[ $? -ne 0 ]]; then
+    echo "Bundling bin files for ${binoption} failed"
+    exit 1
+  fi
+}
+
+for i in "$@"; do
+  case "${i}" in
+    --version=*)
+      VERSION=${i#*=}
+    ;;
+    --artifactid=*)
+      ARTIFACTID=${i#*=}
+    ;;
+    --builddir=*)
+      BUILD_DIR=${i#*=}
+    ;;
+    --opensslbinbundle=*)
+      OPENSSLBINBUNDLE=${i#*=}
+    ;;
+    --openssllib=*)
+      OPENSSLLIB=${i#*=}
+    ;;
+    --openssllibbundle=*)
+      OPENSSLLIBBUNDLE=${i#*=}
+    ;;
+    --snappybinbundle=*)
+      SNAPPYBINBUNDLE=${i#*=}
+    ;;
+    --snappylib=*)
+      SNAPPYLIB=${i#*=}
+    ;;
+    --snappylibbundle=*)
+      SNAPPYLIBBUNDLE=${i#*=}
+    ;;
+
+  esac
+done
+
+TAR='tar cf -'
+UNTAR='tar xfBp -'
+LIB_DIR="${BUILD_DIR}/native/target/usr/local/lib"
+BIN_DIR="${BUILD_DIR}/bin"
+TARGET_DIR="${BUILD_DIR}/${ARTIFACTID}-${VERSION}/lib/native"
+TARGET_BIN_DIR="${BUILD_DIR}/${ARTIFACTID}-${VERSION}/bin"
+
+
+# Most systems
+
+if [[ -d "${LIB_DIR}" ]]; then
+  mkdir -p "${TARGET_DIR}"
+  cd "${LIB_DIR}" || exit 1
+  ${TAR} lib* | (cd "${TARGET_DIR}"/ || exit 1; ${UNTAR})
+  if [[ $? -ne 0 ]]; then
+    echo "Bundling lib files failed"
+    exit 1
+  fi
+
+  bundle_native_lib "${SNAPPYLIBBUNDLE}" "snappy.lib" "snappy" "${SNAPPYLIB}"
+
+  bundle_native_lib "${OPENSSLLIBBUNDLE}" "openssl.lib" "crypto" "${OPENSSLLIB}"
+fi
+
+# Windows
+
+# Windows doesn't have a LIB_DIR, everything goes into bin
+
+if [[ -d "${BIN_DIR}" ]] ; then
+  mkdir -p "${TARGET_BIN_DIR}"
+  cd "${BIN_DIR}"  || exit 1
+  ${TAR} ./* | (cd "${TARGET_BIN_DIR}"/ || exit 1; ${UNTAR})
+  if [[ $? -ne 0 ]]; then
+    echo "Bundling bin files failed"
+    exit 1
+  fi
+
+  bundle_native_bin "${SNAPPYBINBUNDLE}" "${SNAPPYLIBBUNDLE}" "snappy.lib" "snappy" "${SNAPPYLIB}"
+
+  bundle_native_bin "${OPENSSLBINBUNDLE}" "${OPENSSLLIBBUNDLE}" "openssl.lib" "crypto" "${OPENSSLLIB}"
+
+fi

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/dev-support/create-release.sh
----------------------------------------------------------------------
diff --git a/dev-support/create-release.sh b/dev-support/create-release.sh
deleted file mode 100755
index 7a3dd70..0000000
--- a/dev-support/create-release.sh
+++ /dev/null
@@ -1,144 +0,0 @@
-#!/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.
-
-
-# Function to probe the exit code of the script commands, 
-# and stop in the case of failure with an contextual error 
-# message.
-run() {
-  echo "\$ ${@}"
-  "${@}"
-  exitCode=$?
-  if [[ $exitCode != 0 ]]; then
-    echo
-    echo "Failed! running ${@} in `pwd`"
-    echo
-    exit $exitCode
-  fi
-}
-
-doMD5() {
-  MD5CMD="md5sum"
-  which $MD5CMD
-  if [[ $? != 0 ]]; then
-    MD5CMD="md5"
-  fi
-  run $MD5CMD ${1} > ${1}.md5
-}
-
-# If provided, the created release artifacts will be tagged with it 
-# (use RC#, i.e: RC0). Do not use a label to create the final release 
-# artifact.
-RC_LABEL=$1
-
-# Extract Hadoop version from POM
-HADOOP_VERSION=`cat pom.xml | grep "<version>" | head -1 | sed 's|^ *<version>||' | sed 's|</version>.*$||'`
-
-# Setup git
-GIT=${GIT:-git}
-
-echo
-echo "*****************************************************************"
-echo
-echo "Hadoop version to create release artifacts: ${HADOOP_VERSION}"
-echo 
-echo "Release Candidate Label: ${RC_LABEL}"
-echo
-echo "*****************************************************************"
-echo
-
-if [[ ! -z ${RC_LABEL} ]]; then
-  RC_LABEL="-${RC_LABEL}"
-fi
-
-# Get Maven command
-if [ -z "$MAVEN_HOME" ]; then
-  MVN=mvn
-else
-  MVN=$MAVEN_HOME/bin/mvn
-fi
-
-ARTIFACTS_DIR="target/artifacts"
-
-# git clean to clear any remnants from previous build
-run ${GIT} clean -xdf
-
-# mvn clean for sanity
-run ${MVN} clean
-
-# Create staging dir for release artifacts
-run mkdir -p ${ARTIFACTS_DIR}
-
-# Create RAT report
-run ${MVN} apache-rat:check
-
-# Create SRC and BIN tarballs for release,
-# Using 'install\u2019 goal instead of 'package' so artifacts are available 
-# in the Maven local cache for the site generation
-run ${MVN} install -Pdist,src,native -DskipTests -Dtar
-
-# Create site for release
-run ${MVN} site site:stage -Pdist -Psrc
-run mkdir -p target/staging/hadoop-project/hadoop-project-dist/hadoop-yarn
-run mkdir -p target/staging/hadoop-project/hadoop-project-dist/hadoop-mapreduce
-run cp ./hadoop-common-project/hadoop-common/src/main/docs/releasenotes.html target/staging/hadoop-project/hadoop-project-dist/hadoop-common/
-run cp ./hadoop-common-project/hadoop-common/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-common/
-run cp ./hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-hdfs/
-run cp ./hadoop-yarn-project/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-yarn/
-run cp ./hadoop-mapreduce-project/CHANGES.txt target/staging/hadoop-project/hadoop-project-dist/hadoop-mapreduce/
-run mv target/staging/hadoop-project target/r${HADOOP_VERSION}/
-run cd target/
-run tar czf hadoop-site-${HADOOP_VERSION}.tar.gz r${HADOOP_VERSION}/*
-run cd ..
-
-# Stage RAT report
-find . -name rat.txt | xargs -I% cat % > ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-rat.txt
-
-# Stage CHANGES.txt files
-run cp ./hadoop-common-project/hadoop-common/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-COMMON-${HADOOP_VERSION}${RC_LABEL}.txt
-run cp ./hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-HDFS-${HADOOP_VERSION}${RC_LABEL}.txt
-run cp ./hadoop-mapreduce-project/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-MAPREDUCE-${HADOOP_VERSION}${RC_LABEL}.txt
-run cp ./hadoop-yarn-project/CHANGES.txt ${ARTIFACTS_DIR}/CHANGES-YARN-${HADOOP_VERSION}${RC_LABEL}.txt
-
-# Prepare and stage BIN tarball
-run cd hadoop-dist/target/
-run tar -xzf hadoop-${HADOOP_VERSION}.tar.gz
-run cp -r ../../target/r${HADOOP_VERSION}/* hadoop-${HADOOP_VERSION}/share/doc/hadoop/
-run tar -czf hadoop-${HADOOP_VERSION}.tar.gz hadoop-${HADOOP_VERSION}
-run cd ../..
-run mv hadoop-dist/target/hadoop-${HADOOP_VERSION}.tar.gz ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz
-
-# Stage SRC tarball
-run mv hadoop-dist/target/hadoop-${HADOOP_VERSION}-src.tar.gz ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz
-
-# Stage SITE tarball
-run mv target/hadoop-site-${HADOOP_VERSION}.tar.gz ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-site.tar.gz
-
-# MD5 SRC and BIN tarballs
-doMD5 ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}.tar.gz
-doMD5 ${ARTIFACTS_DIR}/hadoop-${HADOOP_VERSION}${RC_LABEL}-src.tar.gz
-
-run cd ${ARTIFACTS_DIR}
-ARTIFACTS_DIR=`pwd`
-echo
-echo "Congratulations, you have successfully built the release"
-echo "artifacts for Apache Hadoop ${HADOOP_VERSION}${RC_LABEL}"
-echo
-echo "The artifacts for this run are available at ${ARTIFACTS_DIR}:"
-run ls -1 ${ARTIFACTS_DIR}
-echo 
-echo "Remember to sign them before staging them on the open"
-echo

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/dev-support/docker/Dockerfile
----------------------------------------------------------------------
diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile
index 1de092f..4be0a08 100644
--- a/dev-support/docker/Dockerfile
+++ b/dev-support/docker/Dockerfile
@@ -27,19 +27,38 @@ WORKDIR /root
 # Install common dependencies from packages
 ######
 RUN apt-get update && apt-get install --no-install-recommends -y \
-    git curl ant make maven \
-    cmake gcc g++ \
-    protobuf-compiler libprotoc-dev \
-    protobuf-c-compiler libprotobuf-dev \
-    build-essential libtool \
-    zlib1g-dev pkg-config libssl-dev \
-    snappy libsnappy-dev \
-    bzip2 libbz2-dev \
-    libjansson-dev \
-    fuse libfuse-dev \
+    ant \
+    build-essential \
+    bzip2 \
+    cmake \
+    curl \
+    doxygen \
+    fuse \
+    g++ \
+    gcc \
+    git \
+    gnupg-agent \
+    make \
+    maven \
+    libbz2-dev \
     libcurl4-openssl-dev \
-    python python2.7 pylint \
-    openjdk-7-jdk doxygen
+    libfuse-dev \
+    libjansson-dev \
+    libprotobuf-dev \
+    libprotoc-dev \
+    libsnappy-dev \
+    libssl-dev \
+    libtool \
+    openjdk-7-jdk \
+    pinentry-curses \
+    pkg-config \
+    protobuf-compiler \
+    protobuf-c-compiler \
+    python \
+    python2.7 \
+    pylint \
+    snappy \
+    zlib1g-dev
 
 # Fixing the Apache commons / Maven dependency problem under Ubuntu:
 # See http://wiki.apache.org/commons/VfsProblems

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
----------------------------------------------------------------------
diff --git a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
index 7128c75..64b15a6 100644
--- a/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
+++ b/hadoop-assemblies/src/main/resources/assemblies/hadoop-dist.xml
@@ -125,7 +125,7 @@
       <outputDirectory>/share/doc/hadoop/${hadoop.component}</outputDirectory>
     </fileSet>
     <fileSet>
-      <directory>${basedir}/src/main/native/libhdfs</directory>
+      <directory>${basedir}/src/main/native/libhdfs/include/hdfs</directory>
       <includes>
         <include>hdfs.h</include>
       </includes>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/hadoop-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-dist/pom.xml b/hadoop-dist/pom.xml
index 0e81e86..b7d56d8 100644
--- a/hadoop-dist/pom.xml
+++ b/hadoop-dist/pom.xml
@@ -123,6 +123,8 @@
                       run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-httpfs/target/hadoop-hdfs-httpfs-${project.version}/* .
                       run cp -r $ROOT/hadoop-common-project/hadoop-kms/target/hadoop-kms-${project.version}/* .
                       run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-nfs/target/hadoop-hdfs-nfs-${project.version}/* .
+                      run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-client/target/hadoop-hdfs-client-${project.version}/* .
+                      run cp -r $ROOT/hadoop-hdfs-project/hadoop-hdfs-native-client/target/hadoop-hdfs-native-client-${project.version}/* .
                       run cp -r $ROOT/hadoop-yarn-project/target/hadoop-yarn-project-${project.version}/* .
                       run cp -r $ROOT/hadoop-mapreduce-project/target/hadoop-mapreduce-${project.version}/* .
                       run cp -r $ROOT/hadoop-tools/hadoop-tools-dist/target/hadoop-tools-dist-${project.version}/* .

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
index 66a500a..a240d09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml
@@ -30,6 +30,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
   <name>Apache Hadoop HDFS Client</name>
   <packaging>jar</packaging>
 
+  <properties>
+    <hadoop.component>hdfs</hadoop.component>
+  </properties>
+
   <dependencies>
     <dependency>
       <groupId>com.squareup.okhttp</groupId>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
index fbc5efe..7d05539 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/pom.xml
@@ -32,6 +32,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
 
   <properties>
     <require.fuse>false</require.fuse>
+    <hadoop.component>hdfs</hadoop.component>
   </properties>
 
   <dependencies>
@@ -222,7 +223,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd">
                     <exec executable="ctest" failonerror="true" dir="${project.build.directory}/">
                       <env key="CLASSPATH" value="${test_classpath}:${compile_classpath}"/>
                       <!-- Make sure libhadoop.so is on LD_LIBRARY_PATH. -->
-                      <env key="LD_LIBRARY_PATH" value="${env.LD_LIBRARY_PATH}:${project.build.directory}/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
+                      <env key="LD_LIBRARY_PATH" value="${env.LD_LIBRARY_PATH}:${project.build.directory}/native/target/usr/local/lib:${hadoop.common.build.dir}/native/target/usr/local/lib"/>
                     </exec>
                   </target>
                 </configuration>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
index d7bfb76..14b62e7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs-native-client/src/CMakeLists.txt
@@ -58,11 +58,19 @@ if(WIN32)
     # Omit unneeded headers.
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -DWIN32_LEAN_AND_MEAN")
     set(OS_DIR ${CMAKE_SOURCE_DIR}/main/native/libhdfs/os/windows)
-    set(OUT_DIR target/bin)
+
+    # IMPORTANT: OUT_DIR MUST be relative to maven's
+    # project.build.directory (=target) and match dist-copynativelibs
+    # in order to be in a release
+    set(OUT_DIR bin)
 else()
     set(CMAKE_C_FLAGS "${CMAKE_C_FLAGS} -fvisibility=hidden")
     set(OS_DIR ${CMAKE_SOURCE_DIR}/main/native/libhdfs/os/posix)
-    set(OUT_DIR target/usr/local/lib)
+
+    # IMPORTANT: OUT_DIR MUST be relative to maven's
+    # project.build.directory (=target) and match dist-copynativelibs
+    # in order to be in a release
+    set(OUT_DIR native/target/usr/local/lib)
 endif()
 
 # Configure JNI.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d1c475dd/hadoop-project-dist/pom.xml
----------------------------------------------------------------------
diff --git a/hadoop-project-dist/pom.xml b/hadoop-project-dist/pom.xml
index 372bd96..2951db1 100644
--- a/hadoop-project-dist/pom.xml
+++ b/hadoop-project-dist/pom.xml
@@ -39,12 +39,14 @@
     <test.build.classes>${project.build.directory}/test-classes</test.build.classes>
 
     <hadoop.component>UNDEF</hadoop.component>
+    <snappy.lib></snappy.lib>
     <bundle.snappy>false</bundle.snappy>
     <bundle.snappy.in.bin>false</bundle.snappy.in.bin>
+    <openssl.lib></openssl.lib>
     <bundle.openssl>false</bundle.openssl>
     <bundle.openssl.in.bin>false</bundle.openssl.in.bin>
   </properties>
-  
+
   <dependencies>
     <dependency>
       <groupId>org.apache.hadoop</groupId>
@@ -312,112 +314,39 @@
       <build>
         <plugins>
           <plugin>
-            <groupId>org.apache.maven.plugins</groupId>
-            <artifactId>maven-antrun-plugin</artifactId>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>exec-maven-plugin</artifactId>
+            <version>${exec-maven-plugin.version}</version>
             <executions>
               <execution>
                 <id>pre-dist</id>
                 <phase>prepare-package</phase>
                 <goals>
-                  <goal>run</goal>
+                  <goal>exec</goal>
                 </goals>
                 <configuration>
-                  <target>
-                    <!-- Using Unix script to preserve symlinks -->
-                    <echo file="${project.build.directory}/dist-copynativelibs.sh">
-
-                      # Bundle a native library if requested. Exit 1 in case error happens.
-                      # Usage: bundle_native_lib bundleOption libOption libPattern libDir
-                      function bundle_native_lib() {
-                        bundleOption="$1"
-                        libOption="$2"
-                        libPattern="$3"
-                        libDir="$4"
-
-                        echo "Checking to bundle with:"
-                        echo "bundleOption=${bundleOption}, libOption=${libOption}, libDir=${libDir}, pattern=${libPattern}"
-
-                        if [[ "${bundleOption}" != "true" ]]; then
-                          return
-                        fi
-
-                        if [[ -z "${libDir}" ]] || [[ ! -d ${libDir} ]]; then
-                          echo "The required option $libOption isn't given or invalid. Bundling the lib failed"
-                          exit 1
-                        fi
-
-                        cd "$libDir"
-                        $$TAR *$libPattern* | (cd $${TARGET_DIR}/; $$UNTAR)
-                        if [[ $? -ne 0 ]]; then
-                          echo "Bundling library with ${libOption} failed "
-                          exit 1
-                        fi
-                      }
-
-                      BUILD_DIR="${project.build.directory}"
-                      TAR='tar cf -'
-                      UNTAR='tar xfBp -'
-                      LIB_DIR="${BUILD_DIR}/native/target/usr/local/lib"
-
-                      set -o pipefail
-
-                      if [[ -d $${LIB_DIR} ]]; then
-                        TARGET_DIR="${BUILD_DIR}/${project.artifactId}-${project.version}/lib/native"
-                        mkdir -p $${TARGET_DIR}
-                        cd $${LIB_DIR}
-                        $$TAR lib* | (cd $${TARGET_DIR}/; $$UNTAR)
-                        if [[ $? -ne 0 ]]; then
-                          echo "Bundling lib files failed"
-                          exit 1
-                        fi
-
-                        echo bundle_native_lib "${bundle.snappy}" "snappy.lib" "snappy" "${snappy.lib}"
-                        bundle_native_lib "${bundle.snappy}" "snappy.lib" "snappy" "${snappy.lib}"
-
-                        echo bundle_native_lib "${bundle.openssl}" "openssl.lib" "crypto" "${openssl.lib}"
-                        bundle_native_lib "${bundle.openssl}" "openssl.lib" "crypto" "${openssl.lib}"
-                      fi
-
-                      BIN_DIR="${BUILD_DIR}/bin"
-                      if [[ -d $${BIN_DIR} ]] ; then
-                        TARGET_BIN_DIR="${BUILD_DIR}/${project.artifactId}-${project.version}/bin"
-                        mkdir -p $${TARGET_BIN_DIR}
-                        cd $${BIN_DIR}
-                        $$TAR * | (cd $${TARGET_BIN_DIR}/; $$UNTAR)
-                        if [[ $? -ne 0 ]]; then
-                          echo "Bundling bin files failed"
-                          exit 1
-                        fi
-
-                        if [[ "${bundle.snappy.in.bin}" == "true" ]]; then
-                          if [[ "${bundle.snappy}" == "true" ]]; then
-                            cd "${snappy.lib}"
-                            $$TAR *snappy* | (cd $${TARGET_BIN_DIR}/; $$UNTAR)
-                            if [[ $? -ne 0 ]]; then
-                              echo "Bundling snappy bin files failed"
-                              exit 1
-                            fi
-                          fi
-                        fi
-                        if [[ "${bundle.openssl.in.bin}" == "true" ]]; then
-                          if [[ "${bundle.openssl}" == "true" ]]; then
-                            cd "${openssl.lib}"
-                            $$TAR *crypto* | (cd $${TARGET_BIN_DIR}/; $$UNTAR)
-                            if [[ $? -ne 0 ]]; then
-                              echo "Bundling openssl bin files failed"
-                              exit 1
-                            fi
-                          fi
-                        fi
-                      fi
-                    </echo>
-
-                    <exec executable="${shell-executable}" dir="${project.build.directory}" failonerror="true">
-                      <arg line="./dist-copynativelibs.sh"/>
-                    </exec>
-                  </target>
+                  <executable>${project.parent.basedir}/../dev-support/bin/dist-copynativelibs</executable>
+                  <workingDirectory>${project.build.directory}</workingDirectory>
+                  <requiresOnline>false</requiresOnline>
+                  <arguments>
+                    <argument>--version=${project.version}</argument>
+                    <argument>--builddir=${project.build.directory}</argument>
+                    <argument>--artifactid=${project.artifactId}</argument>
+                    <argument>--openssllib=${openssl.lib}</argument>
+                    <argument>--opensslbinbundle=${bundle.openssl.in.bin}</argument>
+                    <argument>--openssllibbundle=${bundle.openssl}</argument>
+                    <argument>--snappybinbundle=${bundle.snappy.in.bin}</argument>
+                    <argument>--snappylib=${snappy.lib}</argument>
+                    <argument>--snappylibbundle=${bundle.snappy}</argument>
+                  </arguments>
                 </configuration>
               </execution>
+            </executions>
+          </plugin>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-antrun-plugin</artifactId>
+            <executions>
               <execution>
                 <id>tar</id>
                 <phase>package</phase>


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org