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 2019/11/27 15:15:25 UTC

[hbase] branch HBASE-23162-branch-1 created (now ac52e35)

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

busbey pushed a change to branch HBASE-23162-branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git.


      at ac52e35  HBASE-23162 backport nightly pseudo-distributed hbase test to branches-1.

This branch includes the following new commits:

     new ac52e35  HBASE-23162 backport nightly pseudo-distributed hbase test to branches-1.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[hbase] 01/01: HBASE-23162 backport nightly pseudo-distributed hbase test to branches-1.

Posted by bu...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

busbey pushed a commit to branch HBASE-23162-branch-1
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit ac52e3503fa3cb661cacb05650f581aee96132c8
Author: Sean Busbey <bu...@apache.org>
AuthorDate: Thu Oct 10 17:01:01 2019 -0500

    HBASE-23162 backport nightly pseudo-distributed hbase test to branches-1.
    
    * Includes backport of HBASE-20334 and current state of updates to master branch for test script
    * Changes specific to branch-1 APIs and expectations noted in test file
    * skips hadoop 3 since we don't do hadoop 3 for branches-1 yet.
    * skips using the hbase client tarball since branches-1 don't make one.
    * pass location of shaded artifacts from build area, since branches-1 don't include them in the convenience tarball
---
 dev-support/Jenkinsfile                            | 121 ++++-
 .../hbase_nightly_pseudo-distributed-test.sh       | 549 +++++++++++++++++++++
 dev-support/hbase_nightly_source-artifact.sh       |  14 +-
 .../cache-apache-project-artifact.sh               | 137 +++++
 4 files changed, 789 insertions(+), 32 deletions(-)

diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index ea7d1c6..6966f93 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -57,8 +57,24 @@ pipeline {
     booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
   }
   stages {
-    stage ('yetus install') {
+    stage ('scm-checkout') {
       steps {
+            dir('component') {
+              checkout scm
+            }
+      }
+    }
+    stage ('thirdparty installs') {
+      parallel {
+        stage ('yetus install') {
+          steps {
+            // directory must be unique for each parallel stage, because jenkins runs them in the same workspace :(
+            dir('downloads-yetus') {
+              // can't just do a simple echo or the directory won't be created. :(
+              sh '''#!/usr/bin/env bash
+                echo "Make sure we have a directory for downloading dependencies: $(pwd)"
+'''
+            }
         sh  '''#!/usr/bin/env bash
 set -e
 echo "Ensure we have a copy of Apache Yetus."
@@ -66,25 +82,13 @@ if [[ true !=  "${USE_YETUS_PRERELEASE}" ]]; then
   YETUS_DIR="${WORKSPACE}/yetus-${YETUS_RELEASE}"
   echo "Checking for Yetus ${YETUS_RELEASE} in '${YETUS_DIR}'"
   if ! "${YETUS_DIR}/bin/test-patch" --version >/dev/null 2>&1 ; then
-    echo "New download of Apache Yetus version ${YETUS_RELEASE}."
     rm -rf "${YETUS_DIR}"
-    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}/apache-yetus-${YETUS_RELEASE}-bin.tar.gz"
-    curl -L --fail -O "https://dist.apache.org/repos/dist/release/yetus/${YETUS_RELEASE}/apache-yetus-${YETUS_RELEASE}-bin.tar.gz.asc"
-    echo "verifying yetus release"
-    gpg --homedir "${WORKSPACE}/.gpg" --verify "apache-yetus-${YETUS_RELEASE}-bin.tar.gz.asc"
-    mv "apache-yetus-${YETUS_RELEASE}-bin.tar.gz" yetus.tar.gz
-  else
-    echo "Reusing cached download of Apache Yetus version ${YETUS_RELEASE}."
-  fi
+    "${WORKSPACE}/component/dev-support/jenkins-scripts/cache-apache-project-artifact.sh" \
+        --working-dir "${WORKSPACE}/downloads-yetus" \
+        --keys 'https://www.apache.org/dist/yetus/KEYS' \
+        "${WORKSPACE}/yetus-${YETUS_RELEASE}-bin.tar.gz" \
+        "yetus/${YETUS_RELEASE}/apache-yetus-${YETUS_RELEASE}-bin.tar.gz"
+    mv "yetus-${YETUS_RELEASE}-bin.tar.gz" yetus.tar.gz
 else
   YETUS_DIR="${WORKSPACE}/yetus-git"
   rm -rf "${YETUS_DIR}"
@@ -104,8 +108,33 @@ 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.8.5"
+          }
+          steps {
+            // directory must be unique for each parallel stage, because jenkins runs them in the same workspace :(
+            dir('downloads-hadoop-2') {
+              sh '''#!/usr/bin/env bash
+                echo "Make sure we have a directory for downloading dependencies: $(pwd)"
+'''
+            }
+            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" \
+                  --working-dir "${WORKSPACE}/downloads-hadoop-2" \
+                  --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"
+          }
         }
-        stash name: 'yetus', includes: "yetus-*/*,yetus-*/**/*,tools/personality.sh"
       }
     }
     stage ('init health results') {
@@ -150,6 +179,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
               echo "-- Something went wrong running this stage, please [check relevant console output|${BUILD_URL}/console]." >> "${OUTPUT_DIR}/commentfile"
 '''
             unstash 'yetus'
+            // since we have a new node definition we need to re-do the scm checkout
             dir('component') {
               checkout scm
             }
@@ -441,7 +471,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,14 +484,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"
               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 ".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
-            }
             sh '''#!/usr/bin/env bash
               set -e
               rm -rf "output-srctarball/machine" && mkdir "output-srctarball/machine"
@@ -470,6 +501,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 \
@@ -483,12 +515,46 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
                 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 the expected binaries.' >>output-srctarball/commentfile
+                exit 1
+              fi
+              install_artifact=$(ls -1 "${WORKSPACE}"/unpacked_src_tarball/hbase-assembly/target/hbase-*-bin.tar.gz | sort | head -n 1)
+              tar --strip-component=1 -xzf "${install_artifact}" -C "hbase-install"
+'''
+            unstash 'hadoop-2'
+            echo "Attempting to use run an instance on top of Hadoop 2."
+            sh '''#!/bin/bash -xe
+              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-shaded-server "${WORKSPACE}"/unpacked_src_tarball/hbase-shaded/hbase-shaded-server/target/hbase-shaded-server-*.jar \
+                  --hbase-shaded-client "${WORKSPACE}"/hbase-shaded/hbase-shaded-client/target/hbase-shaded-client-*.jar \
+                  "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}/artifact/output-integration/hadoop-2.log]. (note that this means we didn't run on Hadoop 3)" >output-integration/commentfile
+                exit 2
+              fi
+'''
+            sh '''#!/bin/bash -e
+              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"
               archiveArtifacts 'output-srctarball/*'
               archiveArtifacts 'output-srctarball/**/*'
+              archiveArtifacts 'output-integration/*'
+              archiveArtifacts 'output-integration/**/*'
             }
           }
         }
@@ -509,7 +575,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
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..6a8f4d8
--- /dev/null
+++ b/dev-support/hbase_nightly_pseudo-distributed-test.sh
@@ -0,0 +1,549 @@
+#!/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 /path/to/mapred/executable"
+  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 "    --hbase-client-install /path/to/unpacked/client/tarball           if given we'll look here for hbase client jars instead of the bin-install"
+  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 5 ]; 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
+declare hbase_client
+declare hbase_shaded_mapreduce
+declare hbase_shaded_client
+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;;
+    --hbase-client-install) shift; hbase_client="$1"; shift;;
+    --hbase-shaded-server) shift; hbase_shaded_mapreduce="$1"; shift;;
+    --hbase-shaded-client) shift; hbase_shaded_client="$1"; shift;;
+    --) shift; break;;
+    -*) usage ;;
+    *)  break;;  # terminate while loop
+  esac
+done
+
+# should still have where component checkout is.
+if [ $# -lt 5 ]; then
+  usage
+fi
+component_install="$(cd "$(dirname "$1")"; pwd)/$(basename "$1")"
+hadoop_exec="$(cd "$(dirname "$2")"; pwd)/$(basename "$2")"
+yarn_server_tests_test_jar="$(cd "$(dirname "$3")"; pwd)/$(basename "$3")"
+mapred_jobclient_test_jar="$(cd "$(dirname "$4")"; pwd)/$(basename "$4")"
+mapred_exec="$(cd "$(dirname "$5")"; pwd)/$(basename "$5")"
+
+if [ ! -x "${hadoop_exec}" ]; then
+  echo "hadoop cli does not appear to be executable." >&2
+  exit 1
+fi
+
+if [ ! -x "${mapred_exec}" ]; then
+  echo "mapred 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 [ ! -f "${yarn_server_tests_test_jar}" ]; then
+  echo "Specified YARN server tests test jar is not a file." >&2
+  exit 1
+fi
+
+if [ ! -f "${mapred_jobclient_test_jar}" ]; then
+  echo "Specified MapReduce jobclient test jar is not a file." >&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
+
+if [ -z "${hbase_client}" ]; then
+  hbase_client="${component_install}"
+else
+  echo "Using HBase client-side artifact"
+  # absolutes please
+  hbase_client="$(cd "$(dirname "${hbase_client}")"; pwd)/$(basename "${hbase_client}")"
+  if [ ! -d "${hbase_client}" ]; then
+    echo "If given hbase client install should be a directory with contents of the client tarball." >&2
+    exit 1
+  fi
+fi
+
+if [ -n "${hadoop_jars}" ]; then
+  declare -a tmp_jars
+  for entry in $(echo "${hadoop_jars}" | tr ':' '\n'); do
+    tmp_jars=("${tmp_jars[@]}" "$(cd "$(dirname "${entry}")"; pwd)/$(basename "${entry}")")
+  done
+  hadoop_jars="$(IFS=:; echo "${tmp_jars[*]}")"
+fi
+
+if [ -n "${hbase_shaded_mapreduce}" ]; then
+  echo "Using HBase shaded artifact for mapreduce"
+  # absolutes please
+  hbase_shaded_mapreduce="$(cd "$(dirname "${hbase_shaded_mapreduce}")"; pwd)/$(basename "${hbase_shaded_mapreduce}")"
+  if [ ! -f "${hbase_shaded_mapreduce}" ]; then
+    echo "If given the HBase shaded artifact for mapreduce should be a plain file." >&2
+    exit 1
+  fi
+fi
+
+if [ -n "${hbase_shaded_client}" ]; then
+  echo "Using HBase shaded artifact for downstream clients"
+  # absolutes please
+  hbase_shaded_client="$(cd "$(dirname "${hbase_shaded_client}")"; pwd)/$(basename "${hbase_shaded_client}")"
+  if [ ! -f "${hbase_shaded_client}" ]; then
+    echo "If given the HBase shaded artifact for downstream clients should be a plain file." >&2
+    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 [ -z "${hbase_shaded_mapreduce}" ]; then
+  if [ ! -s "${hbase_client}/lib/shaded-clients/hbase-shaded-server-${hbase_version}.jar" ]; then
+    echo "HBase binary install doesn't appear to include a shaded mapreduce artifact." >&2
+    exit 1
+  else
+    hbase_shaded_mapreduce="${hbase_client}/lib/shaded-clients/hbase-shaded-server-${hbase_version}.jar"
+  fi
+fi
+
+if [ -z "${hbase_shaded_client}" ]; then
+  if [ ! -s "${hbase_client}/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
+  else
+    hbase_shaded_client="${hbase_client}/lib/shaded-clients/hbase-shaded-client-${hbase_version}.jar"
+  fi
+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"
+
+if [ "${hadoop_version%.*.*}" -gt 2 ]; then
+  "${mapred_exec}" minicluster -format -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" &
+else
+  HADOOP_CLASSPATH="${yarn_server_tests_test_jar}" "${hadoop_exec}" jar "${mapred_jobclient_test_jar}" minicluster -format -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" &
+fi
+
+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 [ -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"
+"${hbase_client}/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[*]}")
+hbase_dep_classpath="$("${hbase_client}/bin/hbase" --config "${working_dir}/hbase-conf/" mapredcp)"
+# Work around HBASE-23147 by specifying the Driver
+HADOOP_CLASSPATH="${hbase_dep_classpath}" redirect_and_run "${working_dir}/mr-importtsv" \
+    "${hadoop_exec}" --config "${working_dir}/hbase-conf/" jar "${hbase_shaded_mapreduce}" org.apache.hadoop.hbase.mapreduce.Driver importtsv -Dimporttsv.columns=HBASE_ROW_KEY,family1:column1,family1:column4,family1:column3 test:example example/ -libjars "${hbase_dep_classpath}"
+"${hbase_client}/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"' | "${hbase_client}/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
+
+if [ -z "${hadoop_jars}" ]; then
+  echo "Hadoop client jars not given; getting them from 'hadoop classpath' for the example."
+  hadoop_jars=$("${hadoop_exec}" --config "${working_dir}/hbase-conf/" classpath)
+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.ClusterStatus;
+import org.apache.hadoop.hbase.HBaseConfiguration;
+import org.apache.hadoop.hbase.RegionLoad;
+import org.apache.hadoop.hbase.ServerName;
+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");
+  private static final byte[] EMPTY = new byte[] {};
+
+  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 ClusterStatus cluster = admin.getClusterStatus();
+        System.out.println(String.format("\tCluster reports version %s, ave load %f, region count %d", cluster.getHBaseVersion(), cluster.getAverageLoad(), cluster.getRegionsCount()));
+        for (ServerName server : cluster.getServers()) {
+          for (RegionLoad region : cluster.getLoad(server).getRegionsLoad().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);
+        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++) {
+            final byte[] row = Bytes.toBytes(in.readUTF());
+            final Put put = new Put(row);
+            put.addColumn(FAMILY_BYTES, EMPTY, EMPTY);
+            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
+# 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-7190.
+# However we also have to work around the branch-1 equivalent of HBASE-20333. Since YARN-7190 got fixed and we're unlikely to further clean up
+# branches-1's shaded artifacts, put hadoop classes first
+redirect_and_run "${working_dir}/hbase-shaded-client-compile" \
+    javac -cp "${hadoop_jars}:${hbase_shaded_client}" "${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."
+redirect_and_run "${working_dir}/hbase-shaded-client-example" \
+    java -cp "${working_dir}/hbase-conf/:${hadoop_jars}:${hbase_shaded_client}:${hbase_dep_classpath}:${working_dir}" 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"
+
+"${hbase_client}/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"' | "${hbase_client}/bin/hbase" --config "${working_dir}/hbase-conf/" shell --noninteractive 2>/dev/null | tail -n 1)
+if [ "${example_rowcount}" -gt "1049" ]; then
+  echo "Found ${example_rowcount} rows, which is enough to cover 48 for import, 1000 example's use of user table regions, 1 for example's use of meta region, and 1 for example's count record"
+else
+  echo "ERROR: Only found ${example_rowcount} rows."
+fi
+
+)
diff --git a/dev-support/hbase_nightly_source-artifact.sh b/dev-support/hbase_nightly_source-artifact.sh
index f3655e0..719a70f 100755
--- a/dev-support/hbase_nightly_source-artifact.sh
+++ b/dev-support/hbase_nightly_source-artifact.sh
@@ -175,9 +175,13 @@ echo "Follow the ref guide section on making a RC: Step 8 Build the binary tarba
 # N.B. Older JDK7 requires the TLSv1.2 in order to talk to maven central
 if mvn -Dhttps.protocols=TLSv1.2 -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
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..5653b05
--- /dev/null
+++ b/dev-support/jenkins-scripts/cache-apache-project-artifact.sh
@@ -0,0 +1,137 @@
+#!/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 [ -n "${keys}" ]; then
+    echo "Stopping gpg agent daemon"
+    gpgconf --homedir "${working_dir}/.gpg" --kill gpg-agent
+    echo "Stopped gpg agent daemon"
+  fi
+
+  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!"