You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by bu...@apache.org on 2018/05/22 22:39:27 UTC

[4/4] hbase git commit: HBASE-20334 add a test that verifies basic client and MR integration

HBASE-20334 add a test that verifies basic client and MR integration


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

Branch: refs/heads/HBASE-20331
Commit: 3e880b72da4282076effb117f49b62c1c1ae19dc
Parents: 6b62c03
Author: Sean Busbey <bu...@apache.org>
Authored: Tue May 1 14:28:52 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue May 22 17:33:48 2018 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile                         | 204 ++++++--
 .../hbase_nightly_pseudo-distributed-test.sh    | 481 +++++++++++++++++++
 dev-support/hbase_nightly_source-artifact.sh    |  14 +-
 .../cache-apache-project-artifact.sh            | 131 +++++
 4 files changed, 774 insertions(+), 56 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/3e880b72/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 821d20e..6c22c41 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -60,54 +60,89 @@ pipeline {
     booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
   }
   stages {
-    stage ('yetus install') {
-      steps {
-        sh  '''#!/usr/bin/env bash
-set -e
-echo "Ensure we have a copy of Apache Yetus."
-if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
-  YETUS_DIR="${WORKSPACE}/yetus-${YETUS_RELEASE}"
-  echo "Checking for Yetus ${YETUS_RELEASE} in '${YETUS_DIR}'"
-  if [ ! -d "${YETUS_DIR}" ]; then
-    echo "New download of Apache Yetus version ${YETUS_RELEASE}."
-    rm -rf "${WORKSPACE}/.gpg"
-    mkdir -p "${WORKSPACE}/.gpg"
-    chmod -R 700 "${WORKSPACE}/.gpg"
-
-    echo "install yetus project KEYS"
-    curl -L --fail -o "${WORKSPACE}/KEYS_YETUS" https://dist.apache.org/repos/dist/release/yetus/KEYS
-    gpg --homedir "${WORKSPACE}/.gpg" --import "${WORKSPACE}/KEYS_YETUS"
-
-    echo "download yetus release ${YETUS_RELEASE}"
-    curl -L --fail -O "https://dist.apache.org/repos/dist/release/yetus/${YETUS_RELEASE}/yetus-${YETUS_RELEASE}-bin.tar.gz"
-    curl -L --fail -O "https://dist.apache.org/repos/dist/release/yetus/${YETUS_RELEASE}/yetus-${YETUS_RELEASE}-bin.tar.gz.asc"
-    echo "verifying yetus release"
-    gpg --homedir "${WORKSPACE}/.gpg" --verify "yetus-${YETUS_RELEASE}-bin.tar.gz.asc"
-    mv "yetus-${YETUS_RELEASE}-bin.tar.gz" yetus.tar.gz
-  else
-    echo "Reusing cached download of Apache Yetus version ${YETUS_RELEASE}."
-  fi
-else
-  YETUS_DIR="${WORKSPACE}/yetus-git"
-  rm -rf "${YETUS_DIR}"
-  echo "downloading from github"
-  curl -L --fail https://api.github.com/repos/apache/yetus/tarball/HEAD -o yetus.tar.gz
-fi
-if [ ! -d "${YETUS_DIR}" ]; then
-  echo "unpacking yetus into '${YETUS_DIR}'"
-  mkdir -p "${YETUS_DIR}"
-  gunzip -c yetus.tar.gz | tar xpf - -C "${YETUS_DIR}" --strip-components 1
-fi
-        '''
-        // Set up the file we need at PERSONALITY_FILE location
-        dir ("tools") {
-          sh """#!/usr/bin/env bash
-set -e
-echo "Downloading Project personality."
-curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
-          """
+    stage ('thirdparty installs') {
+      parallel {
+        stage ('yetus install') {
+          steps {
+            dir('component') {
+              checkout scm
+            }
+            sh  '''#!/usr/bin/env bash
+              set -e
+              echo "Ensure we have a copy of Apache Yetus."
+              if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
+                YETUS_DIR="${WORKSPACE}/yetus-${YETUS_RELEASE}"
+                echo "Checking for Yetus ${YETUS_RELEASE} in '${YETUS_DIR}'"
+                if [ ! -d "${YETUS_DIR}" ]; then
+                  "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \
+                      --keys 'https://www.apache.org/dist/yetus/KEYS' \
+                      "${WORKSPACE}/yetus-${YETUS_RELEASE}-bin.tar.gz" \
+                      "yetus/${YETUS_RELEASE}/yetus-${YETUS_RELEASE}-bin.tar.gz"
+                  mv "yetus-${YETUS_RELEASE}-bin.tar.gz" yetus.tar.gz
+                else
+                  echo "Reusing cached install of Apache Yetus version ${YETUS_RELEASE}."
+                fi
+              else
+                YETUS_DIR="${WORKSPACE}/yetus-git"
+                rm -rf "${YETUS_DIR}"
+                echo "downloading from github"
+                curl -L --fail https://api.github.com/repos/apache/yetus/tarball/HEAD -o yetus.tar.gz
+              fi
+              if [ ! -d "${YETUS_DIR}" ]; then
+                echo "unpacking yetus into '${YETUS_DIR}'"
+                mkdir -p "${YETUS_DIR}"
+                gunzip -c yetus.tar.gz | tar xpf - -C "${YETUS_DIR}" --strip-components 1
+              fi
+            '''
+            // Set up the file we need at PERSONALITY_FILE location
+            dir ("tools") {
+              sh """#!/usr/bin/env bash
+                set -e
+                echo "Downloading Project personality."
+                curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
+              """
+            }
+            stash name: 'yetus', includes: "yetus-*/*,yetus-*/**/*,tools/personality.sh"
+          }
+        }
+        stage ('hadoop 2 cache') {
+          environment {
+            HADOOP2_VERSION="2.7.1"
+          }
+          steps {
+            dir('component') {
+              checkout scm
+            }
+            sh '''#!/usr/bin/env bash
+              set -e
+              echo "Ensure we have a copy of Hadoop ${HADOOP2_VERSION}"
+              "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \
+                  --keys 'http://www.apache.org/dist/hadoop/common/KEYS' \
+                  "${WORKSPACE}/hadoop-${HADOOP2_VERSION}-bin.tar.gz" \
+                  "hadoop/common/hadoop-${HADOOP2_VERSION}/hadoop-${HADOOP2_VERSION}.tar.gz"
+            '''
+            stash name: 'hadoop-2', includes: "hadoop-${HADOOP2_VERSION}-bin.tar.gz"
+          }
+        }
+        stage ('hadoop 3 cache') {
+          environment {
+            HADOOP3_VERSION="3.0.0"
+          }
+          steps {
+            dir('component') {
+              checkout scm
+            }
+            sh '''#!/usr/bin/env bash
+              set -e
+              echo "Ensure we have a copy of Hadoop ${HADOOP3_VERSION}"
+              "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \
+                  --keys 'http://www.apache.org/dist/hadoop/common/KEYS' \
+                  "${WORKSPACE}/hadoop-${HADOOP3_VERSION}-bin.tar.gz" \
+                  "hadoop/common/hadoop-${HADOOP3_VERSION}/hadoop-${HADOOP3_VERSION}.tar.gz"
+            '''
+            stash name: 'hadoop-3', includes: "hadoop-${HADOOP3_VERSION}-bin.tar.gz"
+          }
         }
-        stash name: 'yetus', includes: "yetus-*/*,yetus-*/**/*,tools/personality.sh"
       }
     }
     stage ('init health results') {
@@ -441,7 +476,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
         }
         // This is meant to mimic what a release manager will do to create RCs.
         // See http://hbase.apache.org/book.html#maven.release
-        stage ('create source tarball') {
+        stage ('packaging and integration') {
           tools {
             maven 'Maven (latest)'
             // this needs to be set to the jdk that ought to be used to build releases on the branch the Jenkinsfile is stored in.
@@ -454,10 +489,15 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
             sh '''#!/bin/bash -e
               echo "Setting up directories"
               rm -rf "output-srctarball" && mkdir "output-srctarball"
+              rm -rf "output-integration" && mkdir "output-integration" "output-integration/hadoop-2" "output-integration/hadoop-3" "output-integration/hadoop-3-shaded"
               rm -rf "unpacked_src_tarball" && mkdir "unpacked_src_tarball"
+              rm -rf "hbase-install" && mkdir "hbase-install"
+              rm -rf "hadoop-2" && mkdir "hadoop-2"
+              rm -rf "hadoop-3" && mkdir "hadoop-3"
               rm -rf ".m2-for-repo" && mkdir ".m2-for-repo"
               rm -rf ".m2-for-src" && mkdir ".m2-for-src"
-              echo '(x) {color:red}-1 source release artifact{color}\n-- Something went wrong with this stage, [check relevant console output|${BUILD_URL}/console].' >output-srctarball/commentfile
+              echo "(x) {color:red}-1 source release artifact{color}\n-- Something went wrong with this stage, [check relevant console output|${BUILD_URL}/console]." >output-srctarball/commentfile
+              echo "(x) {color:red}-1 client integration test{color}\n-- Something went wrong with this stage, [check relevant console output|${BUILD_URL}/console]." >output-integration/commentfile
 '''
             dir('component') {
               checkout scm
@@ -470,6 +510,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               ls -lh "output-srctarball/machine"
 '''
             sh """#!/bin/bash -e
+              echo "Checking the steps for an RM to make a source artifact, then a binary artifact."
               if "${env.BASEDIR}/dev-support/hbase_nightly_source-artifact.sh" \
                   --intermediate-file-dir output-srctarball \
                   --unpack-temp-dir unpacked_src_tarball \
@@ -480,14 +521,74 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
                 echo '(/) {color:green}+1 source release artifact{color}\n-- See build output for details.' >output-srctarball/commentfile
               else
                 echo '(x) {color:red}-1 source release artifact{color}\n-- See build output for details.' >output-srctarball/commentfile
+                exit 1
               fi
 """
+            echo "unpacking the hbase bin tarball into 'hbase-install'"
+            sh '''#!/bin/bash -e
+              if [ 1 -ne $(ls -1 "${WORKSPACE}"/unpacked_src_tarball/hbase-assembly/target/hbase-*-bin.tar.gz | wc -l) ]; then
+                echo '(x) {color:red}-1 testing binary artifact{color}\n-- source tarball did not produce a single binary.' >>output-srctarball/commentfile
+                exit 1
+              fi
+              artifact=$(ls -1 "${WORKSPACE}"/unpacked_src_tarball/hbase-assembly/target/hbase-*-bin.tar.gz | head -n 1)
+              tar --stripcomponent=1 -xzf "${artifact}" -C "hbase-install"
+'''
+            unstash 'hadoop-2'
+            echo "Attempting to use run an instance on top of Hadoop 2."
+            sh '''#!/bin/bash -e
+              artifact=$(ls -1 "${WORKSPACE}"/hadoop-2*.tar.gz | head -n 1)
+              tar --strip-components=1 -xzf "${artifact}" -C "hadoop-2"
+              if ! "${BASEDIR}/dev-support/hbase_nightly_pseudo-distributed-test.sh" \
+                  --single-process \
+                  --working-dir output-integration/hadoop-2 \
+                  hbase-install \
+                  hadoop-2/bin/hadoop \
+                  hadoop-2/share/hadoop/yarn/test/hadoop-yarn-server-tests-*-tests.jar \
+                  hadoop-2/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-*-tests.jar \
+                  >output-integration/hadoop-2.log 2>&1 ; then
+                echo "(x) {color:red}-1 client integration test{color}\n--Failed when running client tests on top of Hadoop 2. [see log for details|${BUILD_URL}/artifacts/output-integration/hadoop-2.log]. (note that this means we didn't run on Hadoop 3)" >output-integration/commentfile
+                exit 2
+              fi
+'''
+            unstash 'hadoop-3'
+            sh '''#!/bin/bash -e
+              artifact=$(ls -1 "${WORKSPACE}"/hadoop-3*.tar.gz | head -n 1)
+              tar --strip-components=1 -xzf "${artifact}" -C "hadoop-3"
+              if ! "${BASEDIR}/dev-support/hbase_nightly_pseudo-distributed-test.sh" \
+                  --single-process \
+                  --working-dir output-integration/hadoop-3 \
+                  hbase-install \
+                  hadoop-3/bin/hadoop \
+                  hadoop-3/share/hadoop/yarn/test/hadoop-yarn-server-tests-*-tests.jar \
+                  hadoop-3/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-*-tests.jar \
+                  >output-integration/hadoop-3.log 2>&1 ; then
+                echo "(x) {color:red}-1 client integration test{color}\n--Failed when running client tests on top of Hadoop 3. [see log for details|${BUILD_URL}/artifacts/output-integration/hadoop-3.log]. (note that this means we didn't check the Hadoop 3 shaded client)" >output-integration/commentfile
+                exit 2
+              fi
+              if ! "${BASEDIR}/dev-support/hbase_nightly_pseudo-distributed-test.sh" \
+                  --single-process \
+                  --hadoop-client-classpath hadoop-3/share/hadoop/client/hadoop-client-api-*.jar:hadoop-3/share/hadoop/client/hadoop-client-runtime-*.jar \
+                  --working-dir output-integration/hadoop-3-shaded \
+                  hbase-install \
+                  hadoop-3/bin/hadoop \
+                  hadoop-3/share/hadoop/yarn/test/hadoop-yarn-server-tests-*-tests.jar \
+                  hadoop-3/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-*-tests.jar \
+                  >output-integration/hadoop-3-shaded.log 2>&1 ; then
+                echo "(x) {color:red}-1 client integration test{color}\n--Failed when running client tests on top of Hadoop 3 using Hadoop's shaded client. [see log for details|${BUILD_URL}/artifacts/output-integration/hadoop-3-shaded.log]." >output-integration/commentfile
+                exit 2
+              fi
+              echo "(/) {color:green}+1 client integration test{color}" >output-integration/commentfile
+'''
+
+
           }
           post {
             always {
-              stash name: 'srctarball-result', includes: "output-srctarball/commentfile"
+              stash name: 'srctarball-result', includes: "output-srctarball/commentfile,output-integration/commentfile"
               archive 'output-srctarball/*'
               archive 'output-srctarball/**/*'
+              archive 'output-integration/*'
+              archive 'output-integration/**/*'
             }
           }
         }
@@ -508,7 +609,8 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
                           "${env.OUTPUT_DIR_RELATIVE_JDK7}/commentfile",
                           "${env.OUTPUT_DIR_RELATIVE_HADOOP2}/commentfile",
                           "${env.OUTPUT_DIR_RELATIVE_HADOOP3}/commentfile",
-                          'output-srctarball/commentfile']
+                          'output-srctarball/commentfile',
+                          'output-integration/commentfile']
            echo env.BRANCH_NAME
            echo env.BUILD_URL
            echo currentBuild.result

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e880b72/dev-support/hbase_nightly_pseudo-distributed-test.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_pseudo-distributed-test.sh b/dev-support/hbase_nightly_pseudo-distributed-test.sh
new file mode 100755
index 0000000..11fa523
--- /dev/null
+++ b/dev-support/hbase_nightly_pseudo-distributed-test.sh
@@ -0,0 +1,481 @@
+#!/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 -e
+function usage {
+  echo "Usage: ${0} [options] /path/to/component/bin-install /path/to/hadoop/executable /path/to/hadoop/hadoop-yarn-server-tests-tests.jar /path/to/hadoop/hadoop-mapreduce-client-jobclient-tests.jar"
+  echo ""
+  echo "    --zookeeper-data /path/to/use                                     Where the embedded zookeeper instance should write its data."
+  echo "                                                                      defaults to 'zk-data' in the working-dir."
+  echo "    --working-dir /path/to/use                                        Path for writing configs and logs. must exist."
+  echo "                                                                      defaults to making a directory via mktemp."
+  echo "    --hadoop-client-classpath /path/to/some.jar:/path/to/another.jar  classpath for hadoop jars."
+  echo "                                                                      defaults to 'hadoop classpath'"
+  echo "    --force-data-clean                                                Delete all data in HDFS and ZK prior to starting up hbase"
+  echo "    --single-process                                                  Run as single process instead of pseudo-distributed"
+  echo ""
+  exit 1
+}
+# if no args specified, show usage
+if [ $# -lt 4 ]; then
+  usage
+fi
+
+# Get arguments
+declare component_install
+declare hadoop_exec
+declare working_dir
+declare zk_data_dir
+declare clean
+declare distributed="true"
+declare hadoop_jars
+while [ $# -gt 0 ]
+do
+  case "$1" in
+    --working-dir) shift; working_dir=$1; shift;;
+    --force-data-clean) shift; clean="true";;
+    --zookeeper-data) shift; zk_data_dir=$1; shift;;
+    --single-process) shift; distributed="false";;
+    --hadoop-client-classpath) shift; hadoop_jars="$1"; shift;;
+    --) shift; break;;
+    -*) usage ;;
+    *)  break;;  # terminate while loop
+  esac
+done
+
+# should still have where component checkout is.
+if [ $# -lt 4 ]; then
+  usage
+fi
+component_install="$(cd "$(dirname "$1")"; pwd)/$(basename "$1")"
+hadoop_exec="$2"
+yarn_server_tests_test_jar="$3"
+mapred_jobclient_test_jar="$4"
+
+if [ ! -x "${hadoop_exec}" ]; then
+  echo "hadoop cli does not appear to be executable." >&2
+  exit 1
+fi
+
+if [ ! -d "${component_install}" ]; then
+  echo "Path to HBase binary install should be a directory." >&2
+  exit 1
+fi
+
+if [ -z "${working_dir}" ]; then
+  if ! working_dir="$(mktemp -d -t hbase-pseudo-dist-test)" ; then
+    echo "Failed to create temporary working directory. Please specify via --working-dir" >&2
+    exit 1
+  fi
+else
+  # absolutes please
+  working_dir="$(cd "$(dirname "${working_dir}")"; pwd)/$(basename "${working_dir}")"
+  if [ ! -d "${working_dir}" ]; then
+    echo "passed working directory '${working_dir}' must already exist." >&2
+    exit 1
+  fi
+fi
+
+if [ -z "${zk_data_dir}" ]; then
+  zk_data_dir="${working_dir}/zk-data"
+  mkdir "${zk_data_dir}"
+else
+  # absolutes please
+  zk_data_dir="$(cd "$(dirname "${zk_data_dir}")"; pwd)/$(basename "${zk_data_dir}")"
+  if [ ! -d "${zk_data_dir}" ]; then
+    echo "passed directory for unpacking the source tarball '${zk_data_dir}' must already exist."
+    exit 1
+  fi
+fi
+
+echo "You'll find logs and temp files in ${working_dir}"
+
+function redirect_and_run {
+  log_base=$1
+  shift
+  echo "$*" >"${log_base}.err"
+  "$@" >"${log_base}.out" 2>>"${log_base}.err"
+}
+
+(cd "${working_dir}"
+
+echo "Hadoop version information:"
+"${hadoop_exec}" version
+hadoop_version=$("${hadoop_exec}" version | head -n 1)
+hadoop_version="${hadoop_version#Hadoop }"
+if [ "${hadoop_version%.*.*}" -gt 2 ]; then
+  "${hadoop_exec}" envvars
+else
+  echo "JAVA_HOME: ${JAVA_HOME}"
+fi
+
+# Ensure that if some other Hadoop install happens to be present in the environment we ignore it.
+HBASE_DISABLE_HADOOP_CLASSPATH_LOOKUP="true"
+export HBASE_DISABLE_HADOOP_CLASSPATH_LOOKUP
+
+if [ -n "${clean}" ]; then
+  echo "Cleaning out ZooKeeper..."
+  rm -rf "${zk_data_dir:?}/*"
+fi
+
+echo "HBase version information:"
+"${component_install}/bin/hbase" version 2>/dev/null
+hbase_version=$("${component_install}/bin/hbase" version | head -n 1 2>/dev/null)
+hbase_version="${hbase_version#HBase }"
+
+if [ ! -s "${component_install}/lib/shaded-clients/hbase-shaded-mapreduce-${hbase_version}.jar" ]; then
+  echo "HBase binary install doesn't appear to include a shaded mapreduce artifact." >&2
+  exit 1
+fi
+
+if [ ! -s "${component_install}/lib/shaded-clients/hbase-shaded-client-${hbase_version}.jar" ]; then
+  echo "HBase binary install doesn't appear to include a shaded client artifact." >&2
+  exit 1
+fi
+
+if [ ! -s "${component_install}/lib/shaded-clients/hbase-shaded-client-byo-hadoop-${hbase_version}.jar" ]; then
+  echo "HBase binary install doesn't appear to include a shaded client artifact." >&2
+  exit 1
+fi
+
+echo "Writing out configuration for HBase."
+rm -rf "${working_dir}/hbase-conf"
+mkdir "${working_dir}/hbase-conf"
+
+if [ -f "${component_install}/conf/log4j.properties" ]; then
+  cp "${component_install}/conf/log4j.properties" "${working_dir}/hbase-conf/log4j.properties"
+else
+  cat >"${working_dir}/hbase-conf/log4j.properties" <<EOF
+# Define some default values that can be overridden by system properties
+hbase.root.logger=INFO,console
+
+# Define the root logger to the system property "hbase.root.logger".
+log4j.rootLogger=${hbase.root.logger}
+
+# Logging Threshold
+log4j.threshold=ALL
+# console
+log4j.appender.console=org.apache.log4j.ConsoleAppender
+log4j.appender.console.target=System.err
+log4j.appender.console.layout=org.apache.log4j.PatternLayout
+log4j.appender.console.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2}: %.1000m%n
+EOF
+fi
+
+cat >"${working_dir}/hbase-conf/hbase-site.xml" <<EOF
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+/**
+ *
+ * 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.
+ */
+-->
+<configuration>
+  <property>
+    <name>hbase.rootdir</name>
+    <!-- We rely on the defaultFS being set in our hadoop confs -->
+    <value>/hbase</value>
+  </property>
+  <property>
+    <name>hbase.zookeeper.property.dataDir</name>
+    <value>${zk_data_dir}</value>
+  </property>
+  <property>
+    <name>hbase.cluster.distributed</name>
+    <value>${distributed}</value>
+  </property>
+</configuration>
+EOF
+
+if [ "true" = "${distributed}" ]; then
+  cat >"${working_dir}/hbase-conf/regionservers" <<EOF
+localhost
+EOF
+fi
+
+function cleanup {
+
+  echo "Shutting down HBase"
+  HBASE_CONF_DIR="${working_dir}/hbase-conf/" "${component_install}/bin/stop-hbase.sh"
+
+  if [ -f "${working_dir}/hadoop.pid" ]; then
+    echo "Shutdown: listing HDFS contents"
+    redirect_and_run "${working_dir}/hadoop_listing_at_end" \
+    "${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -ls -R /
+
+    echo "Shutting down Hadoop"
+    kill -6 "$(cat "${working_dir}/hadoop.pid")"
+  fi
+}
+
+trap cleanup EXIT SIGQUIT
+
+echo "Starting up Hadoop"
+
+HADOOP_CLASSPATH="${yarn_server_tests_test_jar}" "${hadoop_exec}" jar "${mapred_jobclient_test_jar}" minicluster -writeConfig "${working_dir}/hbase-conf/core-site.xml" -writeDetails "${working_dir}/hadoop_cluster_info.json" >"${working_dir}/hadoop_cluster_command.out" 2>"${working_dir}/hadoop_cluster_command.err" &
+echo "$!" > "${working_dir}/hadoop.pid"
+
+sleep_time=2
+until [ -s "${working_dir}/hbase-conf/core-site.xml" ]; do
+  printf '\twaiting for Hadoop to finish starting up.\n'
+  sleep "${sleep_time}"
+  sleep_time="$((sleep_time*2))"
+done
+
+if [ "${hadoop_version%.*.*}" -gt 2 ]; then
+  echo "Verifying configs"
+  "${hadoop_exec}" --config "${working_dir}/hbase-conf/" conftest
+fi
+
+if [ -z "${hadoop_jars}" ]; then
+  echo "Hadoop client jars not given; getting them from 'hadoop classpath'"
+  hadoop_jars=$("${hadoop_exec}" --config "${working_dir}/hbase-conf/" classpath)
+fi
+
+if [ -n "${clean}" ]; then
+  echo "Cleaning out HDFS..."
+  "${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -rm -r /hbase
+  "${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -rm -r example/
+  "${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -rm -r example-region-listing.data
+fi
+
+echo "Listing HDFS contents"
+redirect_and_run "${working_dir}/hadoop_cluster_smoke" \
+    "${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -ls -R /
+
+echo "Starting up HBase"
+HBASE_CONF_DIR="${working_dir}/hbase-conf/" "${component_install}/bin/start-hbase.sh"
+
+sleep_time=2
+until "${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive >"${working_dir}/waiting_hbase_startup.log" 2>&1 <<EOF
+  count 'hbase:meta'
+EOF
+do
+  printf '\tretry waiting for hbase to come up.\n'
+  sleep "${sleep_time}"
+  sleep_time="$((sleep_time*2))"
+done
+
+echo "Setting up table 'test:example' with 1,000 regions"
+"${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive >"${working_dir}/table_create.log" 2>&1 <<EOF
+  create_namespace 'test'
+  create 'test:example', 'family1', 'family2', {NUMREGIONS => 1000, SPLITALGO => 'UniformSplit'}
+EOF
+
+echo "writing out example TSV to example.tsv"
+cat >"${working_dir}/example.tsv" <<EOF
+row1	value8	value8	
+row3			value2
+row2	value9		
+row10		value1	
+pow1	value8		value8
+pow3		value2	
+pow2			value9
+pow10	value1		
+paw1		value8	value8
+paw3	value2		
+paw2		value9	
+paw10			value1
+raw1	value8	value8	
+raw3			value2
+raw2	value9		
+raw10		value1	
+aow1	value8		value8
+aow3		value2	
+aow2			value9
+aow10	value1		
+aaw1		value8	value8
+aaw3	value2		
+aaw2		value9	
+aaw10			value1
+how1	value8	value8	
+how3			value2
+how2	value9		
+how10		value1	
+zow1	value8		value8
+zow3		value2	
+zow2			value9
+zow10	value1		
+zaw1		value8	value8
+zaw3	value2		
+zaw2		value9	
+zaw10			value1
+haw1	value8	value8	
+haw3			value2
+haw2	value9		
+haw10		value1	
+low1	value8		value8
+low3		value2	
+low2			value9
+low10	value1		
+law1		value8	value8
+law3	value2		
+law2		value9	
+law10			value1
+EOF
+
+echo "uploading example.tsv to HDFS"
+"${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -mkdir example
+"${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -copyFromLocal "${working_dir}/example.tsv" "example/"
+
+echo "Importing TSV via shaded client artifact for HBase - MapReduce integration."
+hbase_thirdparty_jars=("${component_install}"/lib/htrace-core4*.jar \
+    "${component_install}"/lib/slf4j-api-*.jar \
+    "${component_install}"/lib/commons-logging-*.jar \
+    "${component_install}"/lib/slf4j-log4j12-*.jar \
+    "${component_install}"/lib/log4j-1.2.*.jar \
+    "${working_dir}/hbase-conf/log4j.properties")
+hbase_dep_classpath=$(IFS=:; echo "${hbase_thirdparty_jars[*]}")
+HADOOP_CLASSPATH="${hbase_dep_classpath}" redirect_and_run "${working_dir}/mr-importtsv" \
+    "${hadoop_exec}" --config "${working_dir}/hbase-conf/" jar "${component_install}/lib/shaded-clients/hbase-shaded-mapreduce-${hbase_version}.jar" importtsv -Dimporttsv.columns=HBASE_ROW_KEY,family1:column1,family1:column4,family1:column3 test:example example/ -libjars "${hbase_dep_classpath}"
+"${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive >"${working_dir}/scan_import.out" 2>"${working_dir}/scan_import.err" <<EOF
+  scan 'test:example'
+EOF
+
+echo "Verifying row count from import."
+import_rowcount=$(echo 'count "test:example"' | "${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive 2>/dev/null | tail -n 1)
+if [ ! "${import_rowcount}" -eq 48 ]; then
+  echo "ERROR: Instead of finding 48 rows, we found ${import_rowcount}."
+  exit 2
+fi
+
+echo "Building shaded client example."
+cat >"${working_dir}/HBaseClientReadWriteExample.java" <<EOF
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.CellBuilder;
+import org.apache.hadoop.hbase.CellBuilderFactory;
+import org.apache.hadoop.hbase.CellBuilderType;
+import org.apache.hadoop.hbase.ClusterMetrics;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.RegionMetrics;
+import org.apache.hadoop.hbase.ServerMetrics;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.client.Admin;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.util.Bytes;
+
+import java.util.LinkedList;
+import java.util.List;
+
+
+public class HBaseClientReadWriteExample {
+  private static final byte[] FAMILY_BYTES = Bytes.toBytes("family2");
+
+  public static void main(String[] args) throws Exception {
+    Configuration hbase = HBaseConfiguration.create();
+    Configuration hadoop = new Configuration();
+    try (Connection connection = ConnectionFactory.createConnection(hbase)) {
+      System.out.println("Generating list of regions");
+      final List<String> regions = new LinkedList<>();
+      try (Admin admin = connection.getAdmin()) {
+        final ClusterMetrics cluster = admin.getClusterMetrics();
+        System.out.println(String.format("\tCluster reports version %s, ave load %f, region count %d", cluster.getHBaseVersion(), cluster.getAverageLoad(), cluster.getRegionCount()));
+        for (ServerMetrics server : cluster.getLiveServerMetrics().values()) {
+          for (RegionMetrics region : server.getRegionMetrics().values()) {
+            regions.add(region.getNameAsString());
+          }
+        }
+      }
+      final Path listing = new Path("example-region-listing.data");
+      System.out.println("Writing list to HDFS");
+      try (FileSystem fs = FileSystem.newInstance(hadoop)) {
+        final Path path = fs.makeQualified(listing);
+        try (FSDataOutputStream out = fs.create(path)) {
+          out.writeInt(regions.size());
+          for (String region : regions) {
+            out.writeUTF(region);
+          }
+          out.hsync();
+        }
+      }
+      final List<Put> puts = new LinkedList<>();
+      final Put marker = new Put(new byte[] { (byte)0 });
+      System.out.println("Reading list from HDFS");
+      try (FileSystem fs = FileSystem.newInstance(hadoop)) {
+        final Path path = fs.makeQualified(listing);
+        final CellBuilder builder = CellBuilderFactory.create(CellBuilderType.SHALLOW_COPY);
+        try (FSDataInputStream in = fs.open(path)) {
+          final int count = in.readInt();
+          marker.addColumn(FAMILY_BYTES, Bytes.toBytes("count"), Bytes.toBytes(count));
+          for(int i = 0; i < count; i++) {
+            builder.clear();
+            final byte[] row = Bytes.toBytes(in.readUTF());
+            final Put put = new Put(row);
+            builder.setRow(row);
+            builder.setFamily(FAMILY_BYTES);
+            builder.setType(Cell.Type.Put);
+            put.add(builder.build());
+            puts.add(put);
+          }
+        }
+      }
+      System.out.println("Writing list into HBase table");
+      try (Table table = connection.getTable(TableName.valueOf("test:example"))) {
+        table.put(marker);
+        table.put(puts);
+      }
+    }
+  }
+}
+EOF
+redirect_and_run "${working_dir}/hbase-shaded-client-compile" \
+    javac -cp "${component_install}/lib/shaded-clients/hbase-shaded-client-byo-hadoop-${hbase_version}.jar:${hadoop_jars}" "${working_dir}/HBaseClientReadWriteExample.java"
+echo "Running shaded client example. It'll fetch the set of regions, round-trip them to a file in HDFS, then write them one-per-row into the test table."
+# The order of classpath entries here is important. if we're using non-shaded Hadoop 3 / 2.9.0 jars, we have to work around YARN-2190.
+redirect_and_run "${working_dir}/hbase-shaded-client-example" \
+    java -cp "${working_dir}/hbase-conf/:${component_install}/lib/shaded-clients/hbase-shaded-client-byo-hadoop-${hbase_version}.jar:${hbase_dep_classpath}:${working_dir}:${hadoop_jars}" HBaseClientReadWriteExample
+
+echo "Checking on results of example program."
+"${hadoop_exec}" --config "${working_dir}/hbase-conf/" fs -copyToLocal "example-region-listing.data" "${working_dir}/example-region-listing.data"
+
+"${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive >"${working_dir}/scan_example.out" 2>"${working_dir}/scan_example.err" <<EOF
+  scan 'test:example'
+EOF
+
+echo "Verifying row count from example."
+example_rowcount=$(echo 'count "test:example"' | "${component_install}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive 2>/dev/null | tail -n 1)
+if [ "${example_rowcount}" -gt "1050" ]; then
+  echo "Found ${example_rowcount} rows, which is enough to cover 48 for import, 1000 example's use of user table regions, 2 for example's use of meta/root regions, and 1 for example's count record"
+else
+  echo "ERROR: Only found ${example_rowcount} rows."
+fi
+
+)

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e880b72/dev-support/hbase_nightly_source-artifact.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase_nightly_source-artifact.sh b/dev-support/hbase_nightly_source-artifact.sh
index 9458324..c435c48 100755
--- a/dev-support/hbase_nightly_source-artifact.sh
+++ b/dev-support/hbase_nightly_source-artifact.sh
@@ -174,9 +174,13 @@ cd "${unpack_dir}"
 echo "Follow the ref guide section on making a RC: Step 8 Build the binary tarball."
 if mvn -DskipTests -Prelease --batch-mode -Dmaven.repo.local="${m2_tarbuild}" clean install \
     assembly:single >"${working_dir}/srctarball_install.log" 2>&1; then
-  echo "Building a binary tarball from the source tarball succeeded."
-else
-  echo "Building a binary tarball from the source tarball failed. see srtarball_install.log for details."
-  exit 1
+  for artifact in "${unpack_dir}"/hbase-assembly/target/hbase-*-bin.tar.gz; do
+    if [ -f "${artifact}" ]; then
+      # TODO check the layout of the binary artifact we just made.
+      echo "Building a binary tarball from the source tarball succeeded."
+      exit 0
+    fi
+  done
 fi
-# TODO check the layout of the binary artifact we just made.
+echo "Building a binary tarball from the source tarball failed. see srtarball_install.log for details."
+exit 1

http://git-wip-us.apache.org/repos/asf/hbase/blob/3e880b72/dev-support/jenkins-scripts/cache-apache-project-artifact.sh
----------------------------------------------------------------------
diff --git a/dev-support/jenkins-scripts/cache-apache-project-artifact.sh b/dev-support/jenkins-scripts/cache-apache-project-artifact.sh
new file mode 100755
index 0000000..57853c3
--- /dev/null
+++ b/dev-support/jenkins-scripts/cache-apache-project-artifact.sh
@@ -0,0 +1,131 @@
+#!/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 -e
+function usage {
+  echo "Usage: ${0} [options] /path/to/download/file.tar.gz download/fragment/eg/project/subdir/some-artifact-version.tar.gz"
+  echo ""
+  echo "    --force                       for a redownload even if /path/to/download/file.tar.gz exists."
+  echo "    --working-dir /path/to/use    Path for writing tempfiles. must exist."
+  echo "                                  defaults to making a directory via mktemp that we clean."
+  echo "    --keys url://to/project/KEYS  where to get KEYS. needed to check signature on download."
+  echo ""
+  exit 1
+}
+# if no args specified, show usage
+if [ $# -lt 2 ]; then
+  usage
+fi
+
+
+# Get arguments
+declare done_if_cached="true"
+declare working_dir
+declare cleanup="true"
+declare keys
+while [ $# -gt 0 ]
+do
+  case "$1" in
+    --force) shift; done_if_cached="false";;
+    --working-dir) shift; working_dir=$1; cleanup="false"; shift;;
+    --keys) shift; keys=$1; shift;;
+    --) shift; break;;
+    -*) usage ;;
+    *)  break;;  # terminate while loop
+  esac
+done
+
+# should still have required args
+if [ $# -lt 2 ]; then
+  usage
+fi
+
+target="$1"
+artifact="$2"
+
+if [ -f "${target}" ] && [ "true" = "${done_if_cached}" ]; then
+  echo "Reusing existing download of '${artifact}'."
+  exit 0
+fi
+
+if [ -z "${working_dir}" ]; then
+  if ! working_dir="$(mktemp -d -t hbase-download-apache-artifact)" ; then
+    echo "Failed to create temporary working directory. Please specify via --working-dir" >&2
+    exit 1
+  fi
+else
+  # absolutes please
+  working_dir="$(cd "$(dirname "${working_dir}")"; pwd)/$(basename "${working_dir}")"
+  if [ ! -d "${working_dir}" ]; then
+    echo "passed working directory '${working_dir}' must already exist." >&2
+    exit 1
+  fi
+fi
+
+function cleanup {
+  if [ "true" = "${cleanup}" ]; then
+    echo "cleaning up temp space."
+    rm -rf "${working_dir}"
+  fi
+}
+trap cleanup EXIT SIGQUIT
+
+echo "New download of '${artifact}'"
+
+# N.B. this comes first so that if gpg falls over we skip the expensive download.
+if [ -n "${keys}" ]; then
+  if [ ! -d "${working_dir}/.gpg" ]; then
+    rm -rf "${working_dir}/.gpg"
+    mkdir -p "${working_dir}/.gpg"
+    chmod -R 700 "${working_dir}/.gpg"
+  fi
+
+  echo "installing project KEYS"
+  curl -L --fail -o "${working_dir}/KEYS" "${keys}"
+  if ! gpg --homedir "${working_dir}/.gpg" --import "${working_dir}/KEYS" ; then
+    echo "ERROR importing the keys via gpg failed. If the output above mentions this error:" >&2
+    echo "    gpg: can't connect to the agent: File name too long" >&2
+    # we mean to give them the command to run, not to run it.
+    #shellcheck disable=SC2016
+    echo 'then you prolly need to create /var/run/user/$(id -u)' >&2
+    echo "see this thread on gnupg-users: https://s.apache.org/uI7x" >&2
+    exit 2
+  fi
+
+  echo "downloading signature"
+  curl -L --fail -o "${working_dir}/artifact.asc" "https://archive.apache.org/dist/${artifact}.asc"
+fi
+
+echo "downloading artifact"
+if ! curl --dump-header "${working_dir}/artifact_download_headers.txt" -L --fail -o "${working_dir}/artifact" "https://www.apache.org/dyn/closer.lua?filename=${artifact}&action=download" ; then
+  echo "Artifact wasn't in mirror system. falling back to archive.a.o."
+  curl --dump-header "${working_dir}/artifact_fallback_headers.txt" -L --fail -o "${working_dir}/artifact" "http://archive.apache.org/dist/${artifact}"
+fi
+
+if [ -n "${keys}" ]; then
+  echo "verifying artifact signature"
+  gpg --homedir "${working_dir}/.gpg" --verify "${working_dir}/artifact.asc"
+  echo "signature good."
+fi
+
+echo "moving artifact into place at '${target}'"
+# ensure we're on the same filesystem
+mv "${working_dir}/artifact" "${target}.copying"
+# attempt atomic move
+mv "${target}.copying" "${target}"
+echo "all done!"