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/23 18:33:52 UTC

[1/6] hbase git commit: HBASE-20591 nightly job needs to enable maven build system. [Forced Update!]

Repository: hbase
Updated Branches:
  refs/heads/HBASE-20331 734dd4055 -> 390d2840a (forced update)


HBASE-20591 nightly job needs to enable maven build system.

Signed-off-by: Mike Drob <md...@apache.org>


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

Branch: refs/heads/HBASE-20331
Commit: dace8ff2a4d1855c90e3b6e2da0aaa34ba110656
Parents: 6c1097e
Author: Sean Busbey <bu...@apache.org>
Authored: Wed May 16 08:58:00 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Tue May 22 17:40:56 2018 -0500

----------------------------------------------------------------------
 dev-support/Jenkinsfile | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/dace8ff2/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 821d20e..9c775f3 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -202,7 +202,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
           }
           environment {
             BASEDIR = "${env.WORKSPACE}/component"
-            TESTS = 'mvninstall,compile,javac,unit,htmlout'
+            TESTS = 'maven,mvninstall,compile,javac,unit,htmlout'
             OUTPUT_DIR_RELATIVE = "${env.OUTPUT_DIR_RELATIVE_JDK7}"
             OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE_JDK7}"
             // On branches where we do jdk7 checks, jdk7 will be JAVA_HOME already.
@@ -280,7 +280,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
           }
           environment {
             BASEDIR = "${env.WORKSPACE}/component"
-            TESTS = 'mvninstall,compile,javac,unit,findbugs,htmlout'
+            TESTS = 'maven,mvninstall,compile,javac,unit,findbugs,htmlout'
             OUTPUT_DIR_RELATIVE = "${env.OUTPUT_DIR_RELATIVE_HADOOP2}"
             OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE_HADOOP2}"
             // This isn't strictly needed on branches that only support jdk8, but doesn't hurt
@@ -365,7 +365,7 @@ curl -L  -o personality.sh "${env.PROJECT_PERSONALITY}"
           }
           environment {
             BASEDIR = "${env.WORKSPACE}/component"
-            TESTS = 'mvninstall,compile,javac,unit,htmlout'
+            TESTS = 'maven,mvninstall,compile,javac,unit,htmlout'
             OUTPUT_DIR_RELATIVE = "${env.OUTPUT_DIR_RELATIVE_HADOOP3}"
             OUTPUT_DIR = "${env.WORKSPACE}/${env.OUTPUT_DIR_RELATIVE_HADOOP3}"
             // This isn't strictly needed on branches that only support jdk8, but doesn't hurt


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

Posted by bu...@apache.org.
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/390d2840
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/390d2840
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/390d2840

Branch: refs/heads/HBASE-20331
Commit: 390d2840ae6e677fe3123dd2d2ebe637f3326bf6
Parents: c696a56
Author: Sean Busbey <bu...@apache.org>
Authored: Tue May 1 14:28:52 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed May 23 13:32:53 2018 -0500

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


http://git-wip-us.apache.org/repos/asf/hbase/blob/390d2840/dev-support/Jenkinsfile
----------------------------------------------------------------------
diff --git a/dev-support/Jenkinsfile b/dev-support/Jenkinsfile
index 9c775f3..9de6b6c 100644
--- a/dev-support/Jenkinsfile
+++ b/dev-support/Jenkinsfile
@@ -60,54 +60,108 @@ pipeline {
     booleanParam(name: 'DEBUG', defaultValue: false, description: 'Produce a lot more meta-information.')
   }
   stages {
-    stage ('yetus install') {
+    stage ('scm-checkout') {
       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}"
-          """
+            dir('component') {
+              checkout scm
+            }
+      }
+    }
+    stage ('thirdparty installs') {
+      parallel {
+        stage ('yetus install') {
+          steps {
+      //      dir('component') {
+      //        checkout scm
+      //      }
+            dir('downloads') {
+              echo "Make sure we have a directory for downloading dependencies."
+            }
+            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" \
+                      --working-dir "${WORKSPACE}/downloads" \
+                      --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
+//            }
+            dir('downloads') {
+              echo "Make sure we have a directory for downloading dependencies."
+            }
+            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" \
+                  --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
+//            }
+            dir('downloads') {
+              echo "Make sure we have a directory for downloading dependencies."
+            }
+            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" \
+                  --working-dir "${WORKSPACE}/downloads" \
+                  --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') {
@@ -149,9 +203,9 @@ 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'
-            dir('component') {
-              checkout scm
-            }
+//            dir('component') {
+//              checkout scm
+//            }
             sh '''#!/usr/bin/env bash
               set -e
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
@@ -216,9 +270,9 @@ 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'
-            dir('component') {
-              checkout scm
-            }
+//            dir('component') {
+//              checkout scm
+//            }
             sh '''#!/usr/bin/env bash
               set -e
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
@@ -296,9 +350,9 @@ 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'
-            dir('component') {
-              checkout scm
-            }
+//            dir('component') {
+//              checkout scm
+//            }
             sh '''#!/usr/bin/env bash
               set -e
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
@@ -383,9 +437,9 @@ 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'
-            dir('component') {
-              checkout scm
-            }
+            //dir('component') {
+            //  checkout scm
+            //}
             sh '''#!/usr/bin/env bash
               set -e
               rm -rf "${OUTPUT_DIR}/machine" && mkdir "${OUTPUT_DIR}/machine"
@@ -441,7 +495,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 +508,19 @@ 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
-            }
+            //dir('component') {
+            //  checkout scm
+            //}
             sh '''#!/usr/bin/env bash
               set -e
               rm -rf "output-srctarball/machine" && mkdir "output-srctarball/machine"
@@ -470,6 +529,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 +540,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 +628,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/390d2840/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/390d2840/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/390d2840/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!"


[4/6] hbase git commit: HBASE-20332 shaded mapreduce module shouldn't include hadoop

Posted by bu...@apache.org.
HBASE-20332 shaded mapreduce module shouldn't include hadoop

* modify the jar checking script to take args; make hadoop stuff optional
* separate out checking the artifacts that have hadoop vs those that don't.
* * Unfortunately means we need two modules for checking things
* * put in a safety check that the support script for checking jar contents is maintained in both modules
* * have to carve out an exception for o.a.hadoop.metrics2. :(
* fix duplicated class warning
* clean up dependencies in hbase-server and some modules that depend on it.
* allow Hadoop to have its own htrace where it needs it
* add a precommit check to make sure we're not using old htrace imports


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

Branch: refs/heads/HBASE-20331
Commit: 93b034a07c173745782ac6244ab2a2cfd6c2fff2
Parents: 5721150
Author: Sean Busbey <bu...@apache.org>
Authored: Mon Apr 9 13:37:44 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed May 23 13:32:52 2018 -0500

----------------------------------------------------------------------
 dev-support/hbase-personality.sh                |  12 ++
 hbase-backup/pom.xml                            |  12 +-
 .../src/main/resources/hbase/checkstyle.xml     |   3 +-
 hbase-client/pom.xml                            |   4 -
 hbase-common/pom.xml                            |  12 --
 .../org/apache/hadoop/hbase/net/Address.java    |   2 +-
 hbase-endpoint/pom.xml                          |  14 --
 hbase-examples/pom.xml                          |  12 --
 hbase-external-blockcache/pom.xml               |   4 -
 hbase-hadoop2-compat/pom.xml                    |   6 -
 hbase-it/pom.xml                                |   6 -
 hbase-mapreduce/pom.xml                         |  30 +--
 hbase-replication/pom.xml                       |   4 -
 hbase-rest/pom.xml                              |  19 +-
 hbase-rsgroup/pom.xml                           |   4 -
 hbase-server/pom.xml                            | 103 ++++-----
 .../hbase-shaded-check-invariants/pom.xml       |  54 +++--
 .../ensure-jars-have-correct-contents.sh        |  92 ++++++--
 hbase-shaded/hbase-shaded-mapreduce/pom.xml     | 190 +++++++++++++++-
 .../pom.xml                                     | 215 +++++++++++++++++++
 .../ensure-jars-have-correct-contents.sh        | 129 +++++++++++
 hbase-shaded/pom.xml                            |  13 ++
 hbase-shell/pom.xml                             |  14 --
 hbase-testing-util/pom.xml                      |  16 --
 hbase-thrift/pom.xml                            |  16 --
 pom.xml                                         |  65 +++---
 26 files changed, 785 insertions(+), 266 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 2b1e2c3..77bbc22 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -649,6 +649,18 @@ function hbaseanti_patchfile
     ((result=result+1))
   fi
 
+  warnings=$(${GREP} -c -E 'import org.apache.htrace.[^c]' "${patchfile}")
+  if [[ ${warnings} -gt 0 ]]; then
+    add_vote_table -1 hbaseanti "" "The patch appears use HTrace 3 classes instead of HTrace 4."
+    echo "Use of HTrace 3 in the patch"
+    {
+      printf 'Use of HTrace 3 in patchfile\n------\n'
+      ${GREP} -n -E 'import org.apache.htrace.[^c]' "${patchfile}"
+      echo "------"
+    } >>"${PATCH_DIR}/${logfile}"
+    ((result=result+1))
+  fi
+
   if [[ ${result} -gt 0 ]]; then
     return 1
   fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-backup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-backup/pom.xml b/hbase-backup/pom.xml
index 7afd51e..00a996f 100644
--- a/hbase-backup/pom.xml
+++ b/hbase-backup/pom.xml
@@ -155,10 +155,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>
@@ -264,9 +260,6 @@
           <value>3.0</value>
         </property>
       </activation>
-      <properties>
-        <hadoop.version>3.0-SNAPSHOT</hadoop.version>
-      </properties>
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -276,6 +269,11 @@
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-mapreduce-client-core</artifactId>
         </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-distcp</artifactId>
+          <version>${hadoop.version}</version>
+        </dependency>
       </dependencies>
     </profile>
   </profiles>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
----------------------------------------------------------------------
diff --git a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
index c77d46b9..601b302 100644
--- a/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
+++ b/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml
@@ -85,7 +85,8 @@
           org.apache.commons.collections4,
           org.apache.commons.lang,
           org.apache.curator.shaded,
-          org.apache.htrace.shaded"/>
+          org.apache.htrace.shaded,
+          org.htrace"/>
     </module>
     <!-- Javadoc Checks
     http://checkstyle.sourceforge.net/config_javadoc.html -->

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-client/pom.xml b/hbase-client/pom.xml
index f6247e3..bb99eec 100644
--- a/hbase-client/pom.xml
+++ b/hbase-client/pom.xml
@@ -235,10 +235,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-common/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-common/pom.xml b/hbase-common/pom.xml
index 5ae8e0b..0aaccb8 100644
--- a/hbase-common/pom.xml
+++ b/hbase-common/pom.xml
@@ -314,12 +314,6 @@
           <artifactId>hadoop-common</artifactId>
           <!--FYI This pulls in hadoop's guava. Its needed for Configuration
                at least-->
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
       </dependencies>
       <build>
@@ -366,12 +360,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
       </dependencies>
       <build>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
----------------------------------------------------------------------
diff --git a/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java b/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
index ab7fa3b..b7931a4 100644
--- a/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
+++ b/hbase-common/src/main/java/org/apache/hadoop/hbase/net/Address.java
@@ -26,7 +26,7 @@ import org.apache.hbase.thirdparty.com.google.common.net.HostAndPort;
  * An immutable type to hold a hostname and port combo, like an Endpoint
  * or java.net.InetSocketAddress (but without danger of our calling
  * resolve -- we do NOT want a resolve happening every time we want
- * to hold a hostname and port combo). This class is also <<Comparable>>.
+ * to hold a hostname and port combo). This class is also @{code Comparable}.
  * <p>In implementation this class is a facade over Guava's {@link HostAndPort}.
  * We cannot have Guava classes in our API hence this Type.
  */

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-endpoint/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-endpoint/pom.xml b/hbase-endpoint/pom.xml
index e9a8cf7..a831d3a 100644
--- a/hbase-endpoint/pom.xml
+++ b/hbase-endpoint/pom.xml
@@ -260,12 +260,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -297,10 +291,6 @@
           <scope>test</scope>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>com.google.guava</groupId>
               <artifactId>guava</artifactId>
             </exclusion>
@@ -344,10 +334,6 @@
           <artifactId>hadoop-minicluster</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>com.google.guava</groupId>
               <artifactId>guava</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-examples/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-examples/pom.xml b/hbase-examples/pom.xml
index 8814491..b7847ef 100644
--- a/hbase-examples/pom.xml
+++ b/hbase-examples/pom.xml
@@ -228,12 +228,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
       </dependencies>
       <build>
@@ -283,12 +277,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-minicluster</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
       </dependencies>
       <build>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-external-blockcache/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-external-blockcache/pom.xml b/hbase-external-blockcache/pom.xml
index 24c33eb..2479b46 100644
--- a/hbase-external-blockcache/pom.xml
+++ b/hbase-external-blockcache/pom.xml
@@ -225,10 +225,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>com.google.guava</groupId>
               <artifactId>guava</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-hadoop2-compat/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-hadoop2-compat/pom.xml b/hbase-hadoop2-compat/pom.xml
index 6a313ab..5174f26 100644
--- a/hbase-hadoop2-compat/pom.xml
+++ b/hbase-hadoop2-compat/pom.xml
@@ -160,12 +160,6 @@ limitations under the License.
       <groupId>org.apache.hadoop</groupId>
       <artifactId>hadoop-common</artifactId>
       <version>${hadoop.version}</version>
-      <exclusions>
-        <exclusion>
-          <groupId>org.apache.htrace</groupId>
-          <artifactId>htrace-core</artifactId>
-        </exclusion>
-      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-it/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-it/pom.xml b/hbase-it/pom.xml
index 9b1a167..4c17245 100644
--- a/hbase-it/pom.xml
+++ b/hbase-it/pom.xml
@@ -335,12 +335,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-mapreduce/pom.xml b/hbase-mapreduce/pom.xml
index af80737..2bf693b 100644
--- a/hbase-mapreduce/pom.xml
+++ b/hbase-mapreduce/pom.xml
@@ -196,6 +196,15 @@
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-server</artifactId>
+      <exclusions>
+        <!-- commons-logging is only used by hbase-http's HttpRequestLog and hbase-server's
+             HBaseTestingUtil. We don't need either of those here, so execlude it.
+          -->
+        <exclusion>
+          <groupId>commons-logging</groupId>
+          <artifactId>commons-logging</artifactId>
+        </exclusion>
+      </exclusions>
     </dependency>
     <dependency>
       <groupId>org.apache.hbase</groupId>
@@ -246,10 +255,19 @@
       <artifactId>junit</artifactId>
       <scope>test</scope>
     </dependency>
+    <!-- jackson(s) used by PerformanceEvaluation and it looks like TableMapReduceUtil -->
     <dependency>
       <groupId>com.fasterxml.jackson.core</groupId>
       <artifactId>jackson-databind</artifactId>
     </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
+    </dependency>
   </dependencies>
   <profiles>
     <!-- Skip the tests in this module -->
@@ -285,10 +303,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>
@@ -335,10 +349,6 @@
           <artifactId>hadoop-hdfs</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>javax.servlet.jsp</groupId>
               <artifactId>jsp-api</artifactId>
             </exclusion>
@@ -378,10 +388,6 @@
           <scope>test</scope>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>org.apache.zookeeper</groupId>
               <artifactId>zookeeper</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-replication/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-replication/pom.xml b/hbase-replication/pom.xml
index d05c60e..b999c1d 100644
--- a/hbase-replication/pom.xml
+++ b/hbase-replication/pom.xml
@@ -156,10 +156,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-rest/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rest/pom.xml b/hbase-rest/pom.xml
index 617f254..d06feec 100644
--- a/hbase-rest/pom.xml
+++ b/hbase-rest/pom.xml
@@ -300,6 +300,19 @@
       <artifactId>jackson-jaxrs-json-provider</artifactId>
     </dependency>
     <dependency>
+      <!-- We *might* need this for XMLStreamReader use in RemoteAdmin
+           TODO figure out if we can remove it.
+        -->
+      <groupId>org.codehaus.jettison</groupId>
+      <artifactId>jettison</artifactId>
+        <exclusions>
+          <exclusion>
+            <groupId>stax</groupId>
+            <artifactId>stax-api</artifactId>
+          </exclusion>
+        </exclusions>
+    </dependency>
+    <dependency>
       <!--For JspC used in ant task-->
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
@@ -374,12 +387,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-rsgroup/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-rsgroup/pom.xml b/hbase-rsgroup/pom.xml
index 2d9a10d..1cc3854 100644
--- a/hbase-rsgroup/pom.xml
+++ b/hbase-rsgroup/pom.xml
@@ -199,10 +199,6 @@
           <artifactId>hadoop-common</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>net.java.dev.jets3t</groupId>
               <artifactId>jets3t</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-server/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-server/pom.xml b/hbase-server/pom.xml
index 11361d8..e9daf93 100644
--- a/hbase-server/pom.xml
+++ b/hbase-server/pom.xml
@@ -71,6 +71,14 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-remote-resources-plugin</artifactId>
         <version>1.5</version>
+        <dependencies>
+          <!-- resource bundle only needed at build time -->
+          <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-resource-bundle</artifactId>
+            <version>${project.version}</version>
+          </dependency>
+        </dependencies>
         <executions>
           <execution>
             <id>default</id>
@@ -392,12 +400,6 @@
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-metrics</artifactId>
     </dependency>
-    <!-- resource bundle only needed at build time -->
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-resource-bundle</artifactId>
-      <optional>true</optional>
-    </dependency>
     <dependency>
       <groupId>commons-codec</groupId>
       <artifactId>commons-codec</artifactId>
@@ -437,19 +439,16 @@
       <artifactId>jetty-webapp</artifactId>
     </dependency>
     <dependency>
-      <!--For JspC used in ant task-->
+      <!-- For JspC used in ant task, then needed at compile /runtime
+           because the source code made from the JSP refers to its runtime
+        -->
       <groupId>org.glassfish.web</groupId>
       <artifactId>javax.servlet.jsp</artifactId>
     </dependency>
+      <!-- Also used by generated sources from our JSP -->
     <dependency>
-      <groupId>org.codehaus.jettison</groupId>
-      <artifactId>jettison</artifactId>
-        <exclusions>
-          <exclusion>
-            <groupId>stax</groupId>
-            <artifactId>stax-api</artifactId>
-          </exclusion>
-        </exclusions>
+      <groupId>javax.servlet.jsp</groupId>
+      <artifactId>javax.servlet.jsp-api</artifactId>
     </dependency>
     <!-- General dependencies -->
     <dependency>
@@ -501,9 +500,20 @@
       <groupId>javax.servlet</groupId>
       <artifactId>javax.servlet-api</artifactId>
     </dependency>
+    <!-- Jackson only used in compile/runtime scope by BlockCacheUtil class
+         also used by some tests
+      -->
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-databind</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-core</artifactId>
+    </dependency>
     <dependency>
-      <groupId>javax.ws.rs</groupId>
-      <artifactId>javax.ws.rs-api</artifactId>
+      <groupId>com.fasterxml.jackson.core</groupId>
+      <artifactId>jackson-annotations</artifactId>
     </dependency>
 
     <!-- tracing Dependencies -->
@@ -512,11 +522,6 @@
       <artifactId>htrace-core4</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core</artifactId>
-      <version>${htrace-hadoop.version}</version>
-    </dependency>
-    <dependency>
       <groupId>com.lmax</groupId>
       <artifactId>disruptor</artifactId>
     </dependency>
@@ -556,6 +561,15 @@
       <artifactId>httpcore</artifactId>
       <scope>test</scope>
     </dependency>
+    <!-- commons-logging is used by HBTU to monkey with log levels
+         have to put it at compile scope because Hadoop's IOUtils uses it
+         both for hadoop 2.7 and 3.0, so we'll fail at compile if it's at test scope.
+      -->
+    <dependency>
+      <groupId>commons-logging</groupId>
+      <artifactId>commons-logging</artifactId>
+      <scope>compile</scope>
+    </dependency>
     <dependency>
       <groupId>org.apache.commons</groupId>
       <artifactId>commons-crypto</artifactId>
@@ -676,34 +690,10 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-distcp</artifactId>
-          <version>${hadoop-two.version}</version>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-auth</artifactId>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-          <version>${hadoop-two.version}</version>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-client</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>com.google.guava</groupId>
-              <artifactId>guava</artifactId>
-            </exclusion>
-          </exclusions>
-        </dependency>
-        <dependency>
-          <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-mapreduce-client-core</artifactId>
           <exclusions>
             <exclusion>
@@ -796,21 +786,32 @@
       <dependencies>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-distcp</artifactId>
-          <version>${hadoop-three.version}</version>
+          <artifactId>hadoop-common</artifactId>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-common</artifactId>
+          <artifactId>hadoop-hdfs</artifactId>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-hdfs-client</artifactId>
+          <version>${hadoop.version}</version>
+        </dependency>
+        <dependency>
+          <groupId>org.apache.hadoop</groupId>
+          <artifactId>hadoop-mapreduce-client-core</artifactId>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-hdfs</artifactId>
+          <type>test-jar</type>
+          <scope>test</scope>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
-          <artifactId>hadoop-annotations</artifactId>
-          <version>${hadoop-three.version}</version>
+          <artifactId>hadoop-mapreduce-client-jobclient</artifactId>
+          <type>test-jar</type>
+          <scope>test</scope>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shaded/hbase-shaded-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-check-invariants/pom.xml b/hbase-shaded/hbase-shaded-check-invariants/pom.xml
index 7322769..7ba4a41 100644
--- a/hbase-shaded/hbase-shaded-check-invariants/pom.xml
+++ b/hbase-shaded/hbase-shaded-check-invariants/pom.xml
@@ -26,7 +26,7 @@
   Enforces our invariants for our shaded artifacts. e.g. shaded clients have
   a specific set of transitive dependencies and shaded clients only contain
   classes that are in particular packages. Does the enforcement through
-  the maven-enforcer-plugin and and integration test.
+  the maven-enforcer-plugin and integration test.
   </description>
   <name>Apache HBase Shaded Packaging Invariants</name>
 
@@ -34,11 +34,15 @@
   </properties>
 
   <dependencies>
-    <dependency>
-      <groupId>org.apache.hbase</groupId>
-      <artifactId>hbase-shaded-client</artifactId>
-      <version>${project.version}</version>
-    </dependency>
+    <!-- Include here any client facing artifacts that presume
+         the runtime environment will have hadoop.
+
+         If our checks fail for the shaded mapreduce artifact,
+         then probably a dependency from hadoop has shown up
+         in the hbase-mapreduce module without being flagged
+         as 'provided' scope. See the note by the relevant
+         hadoop profile in that module.
+      -->
     <dependency>
       <groupId>org.apache.hbase</groupId>
       <artifactId>hbase-shaded-mapreduce</artifactId>
@@ -113,6 +117,8 @@
                     <exclude>com.github.stephenc.findbugs:*</exclude>
                     <!-- We leave HTrace as an unshaded dependnecy on purpose so that tracing within a JVM will work -->
                     <exclude>org.apache.htrace:*</exclude>
+                    <!-- Our public API requires Hadoop at runtime to work -->
+                    <exclude>org.apache.hadoop:*</exclude>
                   </excludes>
                 </banTransitiveDependencies>
                 <banDuplicateClasses>
@@ -158,18 +164,37 @@
           </execution>
         </executions>
       </plugin>
-      <!--
-        Check that we actually relocated everything we included.
-        It's critical that we don't ship third party dependencies that haven't
-        been relocated under our pacakge space, since this will lead to
-        difficult to debug classpath errors for downstream. Unfortunately, that
-        means inspecting all the jars.
-        -->
       <plugin>
         <groupId>org.codehaus.mojo</groupId>
         <artifactId>exec-maven-plugin</artifactId>
         <version>1.6.0</version>
         <executions>
+          <!-- It's easier to have two copies of our validation
+               script than to copy it via remote-resources-plugin, but
+               we need to make sure they stay the same.
+            -->
+          <execution>
+            <id>make-sure-validation-files-are-in-sync</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>diff</executable>
+              <requiresOnline>false</requiresOnline>
+              <arguments>
+                <argument>../hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh</argument>
+                <argument>../hbase-shaded-with-hadoop-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh</argument>
+              </arguments>
+            </configuration>
+          </execution>
+          <!--
+            Check that we actually relocated everything we included.
+            It's critical that we don't ship third party dependencies that haven't
+            been relocated under our package space, since this will lead to
+            difficult to debug classpath errors for downstream. Unfortunately, that
+            means inspecting all the jars.
+            -->
           <execution>
             <id>check-jar-contents</id>
             <phase>integration-test</phase>
@@ -180,6 +205,9 @@
               <executable>${shell-executable}</executable>
               <workingDirectory>${project.build.testOutputDirectory}</workingDirectory>
               <requiresOnline>false</requiresOnline>
+              <!-- Important that we don't pass the 'allow-hadoop' flag here, because
+                   we allowed it as a provided dependency above.
+                -->
               <arguments>
                 <argument>ensure-jars-have-correct-contents.sh</argument>
                 <argument>${hbase-client-artifacts}</argument>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh b/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
index 8bda8ce..eff1d20 100644
--- a/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
+++ b/hbase-shaded/hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
@@ -15,33 +15,67 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Usage: $0 [/path/to/some/example.jar:/path/to/another/example/created.jar]
-#
-# accepts a single command line argument with a colon separated list of
-# paths to jars to check. Iterates through each such passed jar and checks
-# all the contained paths to make sure they follow the below constructed
-# safe list.
+set -e
+function usage {
+  echo "Usage: ${0} [options] [/path/to/some/example.jar:/path/to/another/example/created.jar]"
+  echo ""
+  echo "  accepts a single command line argument with a colon separated list of"
+  echo "  paths to jars to check. Iterates through each such passed jar and checks"
+  echo "  all the contained paths to make sure they follow the below constructed"
+  echo "  safe list."
+  echo ""
+  echo "    --allow-hadoop     Include stuff from the Apache Hadoop project in the list"
+  echo "                       of allowed jar contents. default: false"
+  echo "    --debug            print more info to stderr"
+  exit 1
+}
+# if no args specified, show usage
+if [ $# -lt 1 ]; then
+  usage
+fi
+
+# Get arguments
+declare allow_hadoop
+declare debug
+while [ $# -gt 0 ]
+do
+  case "$1" in
+    --allow-hadoop) shift; allow_hadoop="true";;
+    --debug) shift; debug="true";;
+    --) shift; break;;
+    -*) usage ;;
+    *)  break;;  # terminate while loop
+  esac
+done
+
+# should still have jars to check.
+if [ $# -lt 1 ]; then
+  usage
+fi
+if [ -n "${debug}" ]; then
+  echo "[DEBUG] Checking on jars: $*" >&2
+  echo "jar command is: $(which jar)" >&2
+  echo "grep command is: $(which grep)" >&2
+  grep -V >&2 || true
+fi
+
+IFS=: read -r -d '' -a artifact_list < <(printf '%s\0' "$1")
 
-# we have to allow the directories that lead to the org/apache/hadoop dir
-allowed_expr="(^org/$|^org/apache/$"
+# we have to allow the directories that lead to the hbase dirs
+allowed_expr="(^org/$|^org/apache/$|^org/apache/hadoop/$"
 # We allow the following things to exist in our client artifacts:
-#   * classes in packages that start with org.apache.hadoop, which by
-#     convention should be in a path that looks like org/apache/hadoop
-allowed_expr+="|^org/apache/hadoop/"
+#   * classes in packages that start with org.apache.hadoop.hbase, which by
+#     convention should be in a path that looks like org/apache/hadoop/hbase
+allowed_expr+="|^org/apache/hadoop/hbase"
 #   * classes in packages that start with org.apache.hbase
 allowed_expr+="|^org/apache/hbase/"
 #   * whatever in the "META-INF" directory
 allowed_expr+="|^META-INF/"
 #   * the folding tables from jcodings
 allowed_expr+="|^tables/"
-#   * Hadoop's and HBase's default configuration files, which have the form
+#   * HBase's default configuration files, which have the form
 #     "_module_-default.xml"
-allowed_expr+="|^[^-]*-default.xml$"
-#   * Hadoop's versioning properties files, which have the form
-#     "_module_-version-info.properties"
-allowed_expr+="|^[^-]*-version-info.properties$"
-#   * Hadoop's application classloader properties file.
-allowed_expr+="|^org.apache.hadoop.application-classloader.properties$"
+allowed_expr+="|^hbase-default.xml$"
 # public suffix list used by httpcomponents
 allowed_expr+="|^mozilla/$"
 allowed_expr+="|^mozilla/public-suffix-list.txt$"
@@ -51,12 +85,30 @@ allowed_expr+="|^properties.dtd$"
 allowed_expr+="|^PropertyList-1.0.dtd$"
 
 
+if [ -n "${allow_hadoop}" ]; then
+  #   * classes in packages that start with org.apache.hadoop, which by
+  #     convention should be in a path that looks like org/apache/hadoop
+  allowed_expr+="|^org/apache/hadoop/"
+  #   * Hadoop's default configuration files, which have the form
+  #     "_module_-default.xml"
+  allowed_expr+="|^[^-]*-default.xml$"
+  #   * Hadoop's versioning properties files, which have the form
+  #     "_module_-version-info.properties"
+  allowed_expr+="|^[^-]*-version-info.properties$"
+  #   * Hadoop's application classloader properties file.
+  allowed_expr+="|^org.apache.hadoop.application-classloader.properties$"
+else
+  # We have some classes for integrating with the Hadoop Metrics2 system
+  # that have to be in a particular package space due to access rules.
+  allowed_expr+="|^org/apache/hadoop/metrics2"
+fi
+
+
 allowed_expr+=")"
 declare -i bad_artifacts=0
 declare -a bad_contents
-IFS=: read -r -d '' -a artifact_list < <(printf '%s\0' "$1")
 for artifact in "${artifact_list[@]}"; do
-  bad_contents=($(jar tf "${artifact}" | grep -v -E "${allowed_expr}"))
+  bad_contents=($(jar tf "${artifact}" | grep -v -E "${allowed_expr}" || true))
   if [ ${#bad_contents[@]} -gt 0 ]; then
     echo "[ERROR] Found artifact with unexpected contents: '${artifact}'"
     echo "    Please check the following and either correct the build or update"

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shaded/hbase-shaded-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-mapreduce/pom.xml b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
index cfcc357..edc80cd 100644
--- a/hbase-shaded/hbase-shaded-mapreduce/pom.xml
+++ b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
@@ -62,6 +62,10 @@
         </plugins>
     </build>
     <dependencies>
+        <!--
+             We want to ensure needed hadoop bits are at provided scope for our shaded
+             artifact, so we list them below in hadoop specific profiles.
+          -->
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-mapreduce</artifactId>
@@ -138,10 +142,6 @@
                 <artifactId>jetty-webapp</artifactId>
               </exclusion>
               <exclusion>
-                <groupId>org.glassfish.web</groupId>
-                <artifactId>javax.servlet.jsp</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>org.glassfish.jersey.core</groupId>
                 <artifactId>jersey-server</artifactId>
               </exclusion>
@@ -149,6 +149,17 @@
                 <groupId>org.glassfish.jersey.containers</groupId>
                 <artifactId>jersey-container-servlet-core</artifactId>
               </exclusion>
+              <!-- We excluded the server-side generated classes for JSP, so exclude
+                   their runtime support libraries too
+                -->
+              <exclusion>
+                <groupId>org.glassfish.web</groupId>
+                <artifactId>javax.servlet.jsp</artifactId>
+              </exclusion>
+              <exclusion>
+                <groupId>javax.servlet.jsp</groupId>
+                <artifactId>javax.servlet.jsp-api</artifactId>
+              </exclusion>
             </exclusions>
         </dependency>
     </dependencies>
@@ -158,12 +169,175 @@
             <id>release</id>
             <build>
                 <plugins>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-shade-plugin</artifactId>
-                    </plugin>
+                <!-- Tell the shade plugin we want to leave Hadoop as a dependency -->
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-shade-plugin</artifactId>
+                    <executions>
+                        <execution>
+                            <id>aggregate-into-a-jar-with-relocated-third-parties</id>
+                            <configuration>
+                                <artifactSet>
+                                    <excludes>
+                                        <exclude>org.apache.hadoop:*</exclude>
+                                        <!-- The rest of these should be kept in sync with the parent pom -->
+                                        <exclude>org.apache.hbase:hbase-resource-bundle</exclude>
+                                        <exclude>org.slf4j:*</exclude>
+                                        <exclude>com.google.code.findbugs:*</exclude>
+                                        <exclude>com.github.stephenc.findbugs:*</exclude>
+                                        <exclude>org.apache.htrace:*</exclude>
+                                        <exclude>org.apache.yetus:*</exclude>
+                                        <exclude>log4j:*</exclude>
+                                        <exclude>commons-logging:*</exclude>
+                                    </excludes>
+                                </artifactSet>
+                            </configuration>
+                        </execution>
+                    </executions>
+                </plugin>
                 </plugins>
             </build>
         </profile>
+        <!-- These hadoop profiles should be derived from those in the hbase-mapreduce
+             module. Essentially, you must list the same hadoop-* dependencies
+             since provided dependencies are not transitively included.
+        -->
+        <!-- profile against Hadoop 2.x: This is the default. -->
+        <profile>
+          <id>hadoop-2.0</id>
+          <activation>
+            <property>
+                <!--Below formatting for dev-support/generate-hadoopX-poms.sh-->
+                <!--h2--><name>!hadoop.profile</name>
+            </property>
+          </activation>
+          <dependencies>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-common</artifactId>
+              <scope>provided</scope>
+              <exclusions>
+                <exclusion>
+                  <groupId>net.java.dev.jets3t</groupId>
+                  <artifactId>jets3t</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>javax.servlet.jsp</groupId>
+                  <artifactId>jsp-api</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>org.mortbay.jetty</groupId>
+                  <artifactId>jetty</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>com.sun.jersey</groupId>
+                  <artifactId>jersey-server</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>com.sun.jersey</groupId>
+                  <artifactId>jersey-core</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>com.sun.jersey</groupId>
+                  <artifactId>jersey-json</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>javax.servlet</groupId>
+                  <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>tomcat</groupId>
+                  <artifactId>jasper-compiler</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>tomcat</groupId>
+                  <artifactId>jasper-runtime</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>com.google.code.findbugs</groupId>
+                  <artifactId>jsr305</artifactId>
+                </exclusion>
+              </exclusions>
+            </dependency>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-hdfs</artifactId>
+              <scope>provided</scope>
+              <exclusions>
+                <exclusion>
+                  <groupId>javax.servlet.jsp</groupId>
+                  <artifactId>jsp-api</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>javax.servlet</groupId>
+                  <artifactId>servlet-api</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>io.netty</groupId>
+                  <artifactId>netty</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>stax</groupId>
+                  <artifactId>stax-api</artifactId>
+                </exclusion>
+                <exclusion>
+                  <groupId>xerces</groupId>
+                  <artifactId>xercesImpl</artifactId>
+                </exclusion>
+              </exclusions>
+              <version>${hadoop-two.version}</version>
+            </dependency>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-mapreduce-client-core</artifactId>
+              <scope>provided</scope>
+              <exclusions>
+                <exclusion>
+                  <groupId>com.google.guava</groupId>
+                  <artifactId>guava</artifactId>
+                </exclusion>
+              </exclusions>
+            </dependency>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-auth</artifactId>
+              <scope>provided</scope>
+            </dependency>
+          </dependencies>
+        </profile>
+
+        <!--
+          profile for building against Hadoop 3.0.x. Activate using:
+           mvn -Dhadoop.profile=3.0
+        -->
+        <profile>
+          <id>hadoop-3.0</id>
+          <activation>
+            <property>
+              <name>hadoop.profile</name>
+              <value>3.0</value>
+            </property>
+          </activation>
+          <properties>
+            <hadoop.version>${hadoop-three.version}</hadoop.version>
+          </properties>
+          <dependencies>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-common</artifactId>
+              <scope>provided</scope>
+            </dependency>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-hdfs</artifactId>
+              <scope>provided</scope>
+            </dependency>
+            <dependency>
+              <groupId>org.apache.hadoop</groupId>
+              <artifactId>hadoop-auth</artifactId>
+              <scope>provided</scope>
+            </dependency>
+          </dependencies>
+        </profile>
     </profiles>
 </project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/pom.xml b/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/pom.xml
new file mode 100644
index 0000000..07789f4
--- /dev/null
+++ b/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/pom.xml
@@ -0,0 +1,215 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+ Licensed 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. See accompanying LICENSE file.
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>hbase</artifactId>
+    <groupId>org.apache.hbase</groupId>
+    <version>3.0.0-SNAPSHOT</version>
+    <relativePath>../..</relativePath>
+  </parent>
+  <artifactId>hbase-shaded-with-hadoop-check-invariants</artifactId>
+  <packaging>pom</packaging>
+
+  <description>
+  Enforces our invariants for our shaded artifacts. e.g. shaded clients have
+  a specific set of transitive dependencies and shaded clients only contain
+  classes that are in particular packages. Does the enforcement through
+  the maven-enforcer-plugin and integration test.
+  </description>
+  <name>Apache HBase Shaded Packaging Invariants (with Hadoop bundled)</name>
+
+  <properties>
+  </properties>
+
+  <dependencies>
+    <!-- This should only be client facing artifacts that bundle
+         Apache Hadoop related artifacts.
+      -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-client</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <!-- parent pom defines these for children. :( :( :( -->
+    <dependency>
+      <groupId>com.github.stephenc.findbugs</groupId>
+      <artifactId>findbugs-annotations</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>log4j</groupId>
+      <artifactId>log4j</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>junit</groupId>
+      <artifactId>junit</artifactId>
+      <scope>provided</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.mockito</groupId>
+      <artifactId>mockito-core</artifactId>
+      <scope>provided</scope>
+    </dependency>
+  </dependencies>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-site-plugin</artifactId>
+        <configuration>
+          <skip>true</skip>
+        </configuration>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-enforcer-plugin</artifactId>
+        <dependencies>
+          <dependency>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>extra-enforcer-rules</artifactId>
+            <version>1.0-beta-6</version>
+          </dependency>
+        </dependencies>
+        <executions>
+          <execution>
+            <id>enforce-banned-dependencies</id>
+            <goals>
+              <goal>enforce</goal>
+            </goals>
+            <configuration>
+              <skip>true</skip>
+              <rules>
+                <banTransitiveDependencies>
+<!--
+                  <message>
+    Our client-facing artifacts are not supposed to have additional dependencies
+    and one or more of them do. The output from the enforcer plugin should give
+    specifics.
+                  </message>
+-->
+                  <excludes>
+                    <!-- We leave logging stuff alone -->
+                    <exclude>org.slf4j:*</exclude>
+                    <exclude>log4j:*</exclude>
+                    <exclude>commons-logging:*</exclude>
+                    <!-- annotations that never change -->
+                    <exclude>com.google.code.findbugs:*</exclude>
+                    <exclude>com.github.stephenc.findbugs:*</exclude>
+                    <!-- We leave HTrace as an unshaded dependnecy on purpose so that tracing within a JVM will work -->
+                    <exclude>org.apache.htrace:*</exclude>
+                    <!-- NB we don't exclude Hadoop from this check here, because the assumption is any needed classes
+                         are contained in our artifacts.
+                      -->
+                  </excludes>
+                </banTransitiveDependencies>
+                <banDuplicateClasses>
+                  <findAllDuplicates>true</findAllDuplicates>
+                </banDuplicateClasses>
+              </rules>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-resources-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>test-resources</id>
+            <phase>pre-integration-test</phase>
+            <goals>
+              <goal>testResources</goal>
+            </goals>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <!-- create a maven pom property that has all of our dependencies.
+             below in the integration-test phase we'll pass this list
+             of paths to our jar checker script.
+          -->
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-dependency-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>put-client-artifacts-in-a-property</id>
+            <phase>pre-integration-test</phase>
+            <goals>
+              <goal>build-classpath</goal>
+            </goals>
+            <configuration>
+              <excludeScope>provided</excludeScope>
+              <excludeTransitive>true</excludeTransitive>
+              <outputProperty>hbase-client-artifacts</outputProperty>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>exec-maven-plugin</artifactId>
+        <version>1.6.0</version>
+        <executions>
+          <!-- It's easier to have two copies of our validation
+               script than to copy it via remote-resources-plugin, but
+               we need to make sure they stay the same.
+            -->
+          <execution>
+            <id>make-sure-validation-files-are-in-sync</id>
+            <phase>validate</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>diff</executable>
+              <requiresOnline>false</requiresOnline>
+              <arguments>
+                <argument>../hbase-shaded-with-hadoop-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh</argument>
+                <argument>../hbase-shaded-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh</argument>
+              </arguments>
+            </configuration>
+          </execution>
+          <!--
+            Check that we actually relocated everything we included.
+            It's critical that we don't ship third party dependencies that haven't
+            been relocated under our package space, since this will lead to
+            difficult to debug classpath errors for downstream. Unfortunately, that
+            means inspecting all the jars.
+            -->
+          <execution>
+            <id>check-jar-contents-for-stuff-with-hadoop</id>
+            <phase>integration-test</phase>
+            <goals>
+              <goal>exec</goal>
+            </goals>
+            <configuration>
+              <executable>${shell-executable}</executable>
+              <workingDirectory>${project.build.testOutputDirectory}</workingDirectory>
+              <requiresOnline>false</requiresOnline>
+              <arguments>
+                <argument>ensure-jars-have-correct-contents.sh</argument>
+                <argument>--allow-hadoop</argument>
+                <argument>${hbase-client-artifacts}</argument>
+              </arguments>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+    </plugins>
+  </build>
+
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh b/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
new file mode 100644
index 0000000..eff1d20
--- /dev/null
+++ b/hbase-shaded/hbase-shaded-with-hadoop-check-invariants/src/test/resources/ensure-jars-have-correct-contents.sh
@@ -0,0 +1,129 @@
+#!/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/some/example.jar:/path/to/another/example/created.jar]"
+  echo ""
+  echo "  accepts a single command line argument with a colon separated list of"
+  echo "  paths to jars to check. Iterates through each such passed jar and checks"
+  echo "  all the contained paths to make sure they follow the below constructed"
+  echo "  safe list."
+  echo ""
+  echo "    --allow-hadoop     Include stuff from the Apache Hadoop project in the list"
+  echo "                       of allowed jar contents. default: false"
+  echo "    --debug            print more info to stderr"
+  exit 1
+}
+# if no args specified, show usage
+if [ $# -lt 1 ]; then
+  usage
+fi
+
+# Get arguments
+declare allow_hadoop
+declare debug
+while [ $# -gt 0 ]
+do
+  case "$1" in
+    --allow-hadoop) shift; allow_hadoop="true";;
+    --debug) shift; debug="true";;
+    --) shift; break;;
+    -*) usage ;;
+    *)  break;;  # terminate while loop
+  esac
+done
+
+# should still have jars to check.
+if [ $# -lt 1 ]; then
+  usage
+fi
+if [ -n "${debug}" ]; then
+  echo "[DEBUG] Checking on jars: $*" >&2
+  echo "jar command is: $(which jar)" >&2
+  echo "grep command is: $(which grep)" >&2
+  grep -V >&2 || true
+fi
+
+IFS=: read -r -d '' -a artifact_list < <(printf '%s\0' "$1")
+
+# we have to allow the directories that lead to the hbase dirs
+allowed_expr="(^org/$|^org/apache/$|^org/apache/hadoop/$"
+# We allow the following things to exist in our client artifacts:
+#   * classes in packages that start with org.apache.hadoop.hbase, which by
+#     convention should be in a path that looks like org/apache/hadoop/hbase
+allowed_expr+="|^org/apache/hadoop/hbase"
+#   * classes in packages that start with org.apache.hbase
+allowed_expr+="|^org/apache/hbase/"
+#   * whatever in the "META-INF" directory
+allowed_expr+="|^META-INF/"
+#   * the folding tables from jcodings
+allowed_expr+="|^tables/"
+#   * HBase's default configuration files, which have the form
+#     "_module_-default.xml"
+allowed_expr+="|^hbase-default.xml$"
+# public suffix list used by httpcomponents
+allowed_expr+="|^mozilla/$"
+allowed_expr+="|^mozilla/public-suffix-list.txt$"
+# Comes from commons-configuration, not sure if relocatable.
+allowed_expr+="|^digesterRules.xml$"
+allowed_expr+="|^properties.dtd$"
+allowed_expr+="|^PropertyList-1.0.dtd$"
+
+
+if [ -n "${allow_hadoop}" ]; then
+  #   * classes in packages that start with org.apache.hadoop, which by
+  #     convention should be in a path that looks like org/apache/hadoop
+  allowed_expr+="|^org/apache/hadoop/"
+  #   * Hadoop's default configuration files, which have the form
+  #     "_module_-default.xml"
+  allowed_expr+="|^[^-]*-default.xml$"
+  #   * Hadoop's versioning properties files, which have the form
+  #     "_module_-version-info.properties"
+  allowed_expr+="|^[^-]*-version-info.properties$"
+  #   * Hadoop's application classloader properties file.
+  allowed_expr+="|^org.apache.hadoop.application-classloader.properties$"
+else
+  # We have some classes for integrating with the Hadoop Metrics2 system
+  # that have to be in a particular package space due to access rules.
+  allowed_expr+="|^org/apache/hadoop/metrics2"
+fi
+
+
+allowed_expr+=")"
+declare -i bad_artifacts=0
+declare -a bad_contents
+for artifact in "${artifact_list[@]}"; do
+  bad_contents=($(jar tf "${artifact}" | grep -v -E "${allowed_expr}" || true))
+  if [ ${#bad_contents[@]} -gt 0 ]; then
+    echo "[ERROR] Found artifact with unexpected contents: '${artifact}'"
+    echo "    Please check the following and either correct the build or update"
+    echo "    the allowed list with reasoning."
+    echo ""
+    for bad_line in "${bad_contents[@]}"; do
+      echo "    ${bad_line}"
+    done
+    bad_artifacts=${bad_artifacts}+1
+  else
+    echo "[INFO] Artifact looks correct: '$(basename "${artifact}")'"
+  fi
+done
+
+# if there was atleast one bad artifact, exit with failure
+if [ "${bad_artifacts}" -gt 0 ]; then
+  exit 1
+fi

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml
index 24c5158..93b122f 100644
--- a/hbase-shaded/pom.xml
+++ b/hbase-shaded/pom.xml
@@ -42,6 +42,7 @@
         <module>hbase-shaded-client</module>
         <module>hbase-shaded-mapreduce</module>
         <module>hbase-shaded-check-invariants</module>
+        <module>hbase-shaded-with-hadoop-check-invariants</module>
     </modules>
     <dependencies>
       <dependency>
@@ -118,6 +119,7 @@
                     <artifactId>maven-shade-plugin</artifactId>
                     <executions>
                         <execution>
+                            <id>aggregate-into-a-jar-with-relocated-third-parties</id>
                             <phase>package</phase>
                             <goals>
                                 <goal>shade</goal>
@@ -450,11 +452,22 @@
                                     </excludes>
                                   </filter>
                                   <filter>
+                                    <!-- Duplication of classes that ship in commons-collections 2.x and 3.x
+                                         If we stop bundling a relevant commons-collections artifact we'll
+                                         need to revisit. See: https://s.apache.org/e09o
+                                    -->
+                                    <artifact>commons-beanutils:commons-beanutils-core</artifact>
+                                    <excludes>
+                                      <exclude>org/apache/commons/collections/*.class</exclude>
+                                    </excludes>
+                                  </filter>
+                                  <filter>
                                     <!-- server side webapps that we don't need -->
                                     <artifact>org.apache.hbase:hbase-server</artifact>
                                     <excludes>
                                       <exclude>hbase-webapps/*</exclude>
                                       <exclude>hbase-webapps/**/*</exclude>
+                                      <exclude>**/*_jsp.class</exclude>
                                     </excludes>
                                   </filter>
                                   <filter>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-shell/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shell/pom.xml b/hbase-shell/pom.xml
index 8eaefaa..85f0415 100644
--- a/hbase-shell/pom.xml
+++ b/hbase-shell/pom.xml
@@ -277,12 +277,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -342,10 +336,6 @@
           <scope>test</scope>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>com.google.guava</groupId>
               <artifactId>guava</artifactId>
             </exclusion>
@@ -409,10 +399,6 @@
           <artifactId>hadoop-minicluster</artifactId>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>com.google.guava</groupId>
               <artifactId>guava</artifactId>
             </exclusion>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-testing-util/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-testing-util/pom.xml b/hbase-testing-util/pom.xml
index 0f1b86c..032de61 100644
--- a/hbase-testing-util/pom.xml
+++ b/hbase-testing-util/pom.xml
@@ -140,12 +140,6 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-common</artifactId>
                     <scope>compile</scope>
-                    <exclusions>
-                        <exclusion>
-                            <groupId>org.apache.htrace</groupId>
-                            <artifactId>htrace-core</artifactId>
-                        </exclusion>
-                    </exclusions>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.hadoop</groupId>
@@ -202,10 +196,6 @@
                     <scope>compile</scope>
                     <exclusions>
                       <exclusion>
-                        <groupId>org.apache.htrace</groupId>
-                        <artifactId>htrace-core</artifactId>
-                      </exclusion>
-                      <exclusion>
                         <groupId>com.google.guava</groupId>
                         <artifactId>guava</artifactId>
                       </exclusion>
@@ -242,12 +232,6 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-minicluster</artifactId>
                     <scope>compile</scope>
-                    <exclusions>
-                        <exclusion>
-                            <groupId>org.apache.htrace</groupId>
-                            <artifactId>htrace-core</artifactId>
-                        </exclusion>
-                    </exclusions>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.hadoop</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/hbase-thrift/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-thrift/pom.xml b/hbase-thrift/pom.xml
index 0142ccd..aec3cb4 100644
--- a/hbase-thrift/pom.xml
+++ b/hbase-thrift/pom.xml
@@ -498,12 +498,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-common</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
         <dependency>
           <groupId>org.apache.hadoop</groupId>
@@ -511,10 +505,6 @@
           <scope>test</scope>
           <exclusions>
             <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-            <exclusion>
               <groupId>com.google.guava</groupId>
               <artifactId>guava</artifactId>
             </exclusion>
@@ -571,12 +561,6 @@
         <dependency>
           <groupId>org.apache.hadoop</groupId>
           <artifactId>hadoop-minicluster</artifactId>
-          <exclusions>
-            <exclusion>
-              <groupId>org.apache.htrace</groupId>
-              <artifactId>htrace-core</artifactId>
-            </exclusion>
-          </exclusions>
         </dependency>
       </dependencies>
       <build>

http://git-wip-us.apache.org/repos/asf/hbase/blob/93b034a0/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index c5d8148..46dc8d3 100755
--- a/pom.xml
+++ b/pom.xml
@@ -1442,8 +1442,11 @@
     <hadoop.guava.version>11.0.2</hadoop.guava.version>
     <compat.module>hbase-hadoop2-compat</compat.module>
     <assembly.file>src/main/assembly/hadoop-two-compat.xml</assembly.file>
-    <audience-annotations.version>0.5.0</audience-annotations.version>
+    <!--This property is for hadoops netty. HBase netty
+         comes in via hbase-thirdparty hbase-shaded-netty-->
+    <netty.hadoop.version>3.6.2.Final</netty.hadoop.version>
     <!-- end HBASE-15925 default hadoop compatibility values -->
+    <audience-annotations.version>0.5.0</audience-annotations.version>
     <avro.version>1.7.7</avro.version>
     <commons-codec.version>1.10</commons-codec.version>
     <!-- pretty outdated -->
@@ -1471,7 +1474,6 @@
     <junit.version>4.12</junit.version>
     <hamcrest.version>1.3</hamcrest.version>
     <htrace.version>4.2.0-incubating</htrace.version>
-    <htrace-hadoop.version>3.2.0-incubating</htrace-hadoop.version>
     <log4j.version>1.2.17</log4j.version>
     <mockito-core.version>2.1.0</mockito-core.version>
     <!--Internally we use a different version of protobuf. See hbase-protocol-shaded-->
@@ -1599,7 +1601,8 @@
           org.mortbay.jetty:servlet-api, javax.servlet:servlet-api: These are excluded because they are
           the same implementations. I chose org.mortbay.jetty:servlet-api-2.5 instead, which is a third
           implementation of the same, because Hadoop also uses this version
-          javax.servlet:jsp-api in favour of org.mortbay.jetty:jsp-api-2.1
+          javax.servlet:jsp-api in favour of javax.servlet.jsp:javax.servlet.jsp-api:2.3.1 since it
+          is what glassfish's jspC jar uses and that's where we get our own need for a jsp-api.
         -->
       <!-- Intra-module dependencies -->
       <dependency>
@@ -1915,6 +1918,14 @@
         <version>${commons-math.version}</version>
       </dependency>
       <dependency>
+        <!-- commons-logging is only used by hbase-http's HttpRequestLog and hbase-server's
+             HBaseTestingUtil.
+          -->
+        <groupId>commons-logging</groupId>
+        <artifactId>commons-logging</artifactId>
+        <version>1.2</version>
+      </dependency>
+      <dependency>
         <groupId>org.apache.zookeeper</groupId>
         <artifactId>zookeeper</artifactId>
         <version>${zookeeper.version}</version>
@@ -1979,6 +1990,16 @@
       </dependency>
       <dependency>
         <groupId>com.fasterxml.jackson.core</groupId>
+        <artifactId>jackson-annotations</artifactId>
+        <version>${jackson.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.fasterxml.jackson.core</groupId>
+        <artifactId>jackson-core</artifactId>
+        <version>${jackson.version}</version>
+      </dependency>
+      <dependency>
+        <groupId>com.fasterxml.jackson.core</groupId>
         <artifactId>jackson-databind</artifactId>
         <version>${jackson.version}</version>
       </dependency>
@@ -2081,6 +2102,12 @@
         <version>${glassfish.jsp.version}</version>
       </dependency>
       <dependency>
+        <!-- this lib is used by the compiled Jsp from the above JspC -->
+        <groupId>javax.servlet.jsp</groupId>
+        <artifactId>javax.servlet.jsp-api</artifactId>
+        <version>2.3.1</version>
+      </dependency>
+      <dependency>
         <groupId>org.glassfish</groupId>
         <artifactId>javax.el</artifactId>
         <version>${glassfish.el.version}</version>
@@ -2545,10 +2572,6 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
-                <groupId>org.apache.htrace</groupId>
-                <artifactId>htrace-core</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
               </exclusion>
@@ -2590,10 +2613,6 @@
             <scope>test</scope>
             <exclusions>
               <exclusion>
-                <groupId>org.apache.htrace</groupId>
-                <artifactId>htrace-core</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>javax.servlet.jsp</groupId>
                 <artifactId>jsp-api</artifactId>
               </exclusion>
@@ -2638,10 +2657,6 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
-                <groupId>org.apache.htrace</groupId>
-                <artifactId>htrace-core</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>commons-beanutils</groupId>
                 <artifactId>commons-beanutils</artifactId>
               </exclusion>
@@ -2692,10 +2707,6 @@
             <version>${hadoop-two.version}</version>
             <exclusions>
               <exclusion>
-                <groupId>org.apache.htrace</groupId>
-                <artifactId>htrace-core</artifactId>
-              </exclusion>
-              <exclusion>
                 <groupId>commons-httpclient</groupId>
                 <artifactId>commons-httpclient</artifactId>
               </exclusion>
@@ -2814,10 +2825,6 @@
            <version>${hadoop-three.version}</version>
            <exclusions>
              <exclusion>
-               <groupId>org.apache.htrace</groupId>
-               <artifactId>htrace-core</artifactId>
-             </exclusion>
-             <exclusion>
                <groupId>com.sun.jersey</groupId>
                <artifactId>jersey-core</artifactId>
              </exclusion>
@@ -2863,10 +2870,6 @@
            <scope>test</scope>
            <exclusions>
              <exclusion>
-               <groupId>org.apache.htrace</groupId>
-               <artifactId>htrace-core</artifactId>
-             </exclusion>
-             <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>
              </exclusion>
@@ -2947,10 +2950,6 @@
                <artifactId>jersey-server</artifactId>
              </exclusion>
              <exclusion>
-               <groupId>org.apache.htrace</groupId>
-               <artifactId>htrace-core</artifactId>
-             </exclusion>
-             <exclusion>
                <groupId>javax.servlet.jsp</groupId>
                <artifactId>jsp-api</artifactId>
              </exclusion>
@@ -3015,10 +3014,6 @@
            <version>${hadoop-three.version}</version>
            <exclusions>
              <exclusion>
-               <groupId>org.apache.htrace</groupId>
-               <artifactId>htrace-core</artifactId>
-             </exclusion>
-             <exclusion>
                <groupId>commons-httpclient</groupId>
                <artifactId>commons-httpclient</artifactId>
              </exclusion>


[5/6] hbase git commit: HBASE-20615 ship shaded client artifacts in binary tarball.

Posted by bu...@apache.org.
HBASE-20615 ship shaded client artifacts in binary tarball.


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

Branch: refs/heads/HBASE-20331
Commit: c696a5671aca8861406ce6d1320fcf1c4a470679
Parents: 6eb691b
Author: Sean Busbey <bu...@apache.org>
Authored: Fri May 18 11:11:42 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed May 23 13:32:52 2018 -0500

----------------------------------------------------------------------
 hbase-assembly/pom.xml                          | 16 +++++++++++++
 .../src/main/assembly/hadoop-two-compat.xml     | 25 ++++++++++++++++++++
 2 files changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/c696a567/hbase-assembly/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/pom.xml b/hbase-assembly/pom.xml
index 5da105b..acb6f53 100644
--- a/hbase-assembly/pom.xml
+++ b/hbase-assembly/pom.xml
@@ -189,6 +189,22 @@
     </plugins>
   </build>
   <dependencies>
+    <!-- client artifacts for downstream use -->
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-client</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-client-byo-hadoop</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-mapreduce</artifactId>
+      <version>${project.version}</version>
+    </dependency>
 	<!-- Intra-project dependencies -->
     <dependency>
       <groupId>org.apache.hbase</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/c696a567/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
----------------------------------------------------------------------
diff --git a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
index 69a800b..6ab621a 100644
--- a/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
+++ b/hbase-assembly/src/main/assembly/hadoop-two-compat.xml
@@ -73,11 +73,36 @@
               <exclude>com.sun.jersey:*</exclude>
               <exclude>com.sun.jersey.contribs:*</exclude>
               <exclude>jline:jline</exclude>
+        <exclude>org.apache.hbase:hbase-shaded-client</exclude>
+        <exclude>org.apache.hbase:hbase-shaded-client-byo-hadoop</exclude>
+        <exclude>org.apache.hbase:hbase-shaded-mapreduce</exclude>
             </excludes>
           </dependencySet>
         </dependencySets>
       </binaries>
     </moduleSet>
+    <!-- Include shaded clients in their own directory -->
+    <moduleSet>
+      <useAllReactorProjects>true</useAllReactorProjects>
+      <includes>
+        <include>org.apache.hbase:hbase-shaded-client</include>
+        <include>org.apache.hbase:hbase-shaded-mapreduce</include>
+        <include>org.apache.hbase:hbase-shaded-client-byo-hadoop</include>
+      </includes>
+      <binaries>
+        <outputDirectory>lib/shaded-clients</outputDirectory>
+        <unpack>false</unpack>
+        <dependencySets>
+          <dependencySet>
+            <includes>
+        <include>org.apache.hbase:hbase-shaded-client</include>
+        <include>org.apache.hbase:hbase-shaded-mapreduce</include>
+        <include>org.apache.hbase:hbase-shaded-client-byo-hadoop</include>
+            </includes>
+          </dependencySet>
+        </dependencySets>
+      </binaries>
+    </moduleSet>
   </moduleSets>
   <!-- Include the generated LICENSE and NOTICE files -->
   <files>


[2/6] hbase git commit: HBASE-20612 TestReplicationKillSlaveRSWithSeparateOldWALs sometimes fail because it uses an expired cluster conn

Posted by bu...@apache.org.
HBASE-20612 TestReplicationKillSlaveRSWithSeparateOldWALs sometimes fail because it uses an expired cluster conn


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

Branch: refs/heads/HBASE-20331
Commit: 5721150c6d910ef95c30c0dd3126f58b6096f1b5
Parents: dace8ff
Author: huzheng <op...@gmail.com>
Authored: Tue May 22 19:39:28 2018 +0800
Committer: huzheng <op...@gmail.com>
Committed: Wed May 23 12:07:01 2018 +0800

----------------------------------------------------------------------
 .../hbase/client/RpcRetryingCallerImpl.java     |  1 -
 .../replication/TestReplicationKillRS.java      | 98 ++++++++++----------
 2 files changed, 50 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/5721150c/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
----------------------------------------------------------------------
diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
index 2d05dbb..96bb45d 100644
--- a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
+++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/RpcRetryingCallerImpl.java
@@ -108,7 +108,6 @@ public class RpcRetryingCallerImpl<T> implements RpcRetryingCaller<T> {
       } catch (PreemptiveFastFailException e) {
         throw e;
       } catch (Throwable t) {
-        Throwable e = t.getCause();
         ExceptionUtil.rethrowIfInterrupt(t);
         Throwable cause = t.getCause();
         if (cause instanceof DoNotRetryIOException) {

http://git-wip-us.apache.org/repos/asf/hbase/blob/5721150c/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
index 643b629..5b4fa2a 100644
--- a/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
+++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/replication/TestReplicationKillRS.java
@@ -22,9 +22,12 @@ import static org.junit.Assert.fail;
 import org.apache.hadoop.hbase.HBaseClassTestRule;
 import org.apache.hadoop.hbase.HBaseTestingUtility;
 import org.apache.hadoop.hbase.UnknownScannerException;
+import org.apache.hadoop.hbase.client.Connection;
+import org.apache.hadoop.hbase.client.ConnectionFactory;
 import org.apache.hadoop.hbase.client.Result;
 import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.testclassification.LargeTests;
 import org.apache.hadoop.hbase.testclassification.ReplicationTests;
 import org.junit.ClassRule;
@@ -32,7 +35,7 @@ import org.junit.experimental.categories.Category;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-@Category({ReplicationTests.class, LargeTests.class})
+@Category({ ReplicationTests.class, LargeTests.class })
 public class TestReplicationKillRS extends TestReplicationBase {
 
   @ClassRule
@@ -42,38 +45,34 @@ public class TestReplicationKillRS extends TestReplicationBase {
   private static final Logger LOG = LoggerFactory.getLogger(TestReplicationKillRS.class);
 
   /**
-   * Load up 1 tables over 2 region servers and kill a source during
-   * the upload. The failover happens internally.
-   *
-   * WARNING this test sometimes fails because of HBASE-3515
-   *
-   * @throws Exception
+   * Load up 1 tables over 2 region servers and kill a source during the upload. The failover
+   * happens internally. WARNING this test sometimes fails because of HBASE-3515
    */
   public void loadTableAndKillRS(HBaseTestingUtility util) throws Exception {
     // killing the RS with hbase:meta can result into failed puts until we solve
     // IO fencing
-    int rsToKill1 =
-        util.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
+    int rsToKill1 = util.getHBaseCluster().getServerWithMeta() == 0 ? 1 : 0;
 
     // Takes about 20 secs to run the full loading, kill around the middle
     Thread killer = killARegionServer(util, 5000, rsToKill1);
-
-    LOG.info("Start loading table");
-    int initialCount = utility1.loadTable(htable1, famName);
-    LOG.info("Done loading table");
-    killer.join(5000);
-    LOG.info("Done waiting for threads");
-
     Result[] res;
-    while (true) {
-      try {
-        Scan scan = new Scan();
-        ResultScanner scanner = htable1.getScanner(scan);
-        res = scanner.next(initialCount);
-        scanner.close();
-        break;
-      } catch (UnknownScannerException ex) {
-        LOG.info("Cluster wasn't ready yet, restarting scanner");
+    int initialCount;
+    try (Connection conn = ConnectionFactory.createConnection(conf1)) {
+      try (Table table = conn.getTable(tableName)) {
+        LOG.info("Start loading table");
+        initialCount = utility1.loadTable(table, famName);
+        LOG.info("Done loading table");
+        killer.join(5000);
+        LOG.info("Done waiting for threads");
+
+        while (true) {
+          try (ResultScanner scanner = table.getScanner(new Scan())) {
+            res = scanner.next(initialCount);
+            break;
+          } catch (UnknownScannerException ex) {
+            LOG.info("Cluster wasn't ready yet, restarting scanner");
+          }
+        }
       }
     }
     // Test we actually have all the rows, we may miss some because we
@@ -85,36 +84,39 @@ public class TestReplicationKillRS extends TestReplicationBase {
     }
 
     int lastCount = 0;
-
     final long start = System.currentTimeMillis();
     int i = 0;
-    while (true) {
-      if (i==NB_RETRIES-1) {
-        fail("Waited too much time for queueFailover replication. " +
-            "Waited "+(System.currentTimeMillis() - start)+"ms.");
-      }
-      Scan scan2 = new Scan();
-      ResultScanner scanner2 = htable2.getScanner(scan2);
-      Result[] res2 = scanner2.next(initialCount * 2);
-      scanner2.close();
-      if (res2.length < initialCount) {
-        if (lastCount < res2.length) {
-          i--; // Don't increment timeout if we make progress
-        } else {
-          i++;
+    try (Connection conn = ConnectionFactory.createConnection(conf2)) {
+      try (Table table = conn.getTable(tableName)) {
+        while (true) {
+          if (i == NB_RETRIES - 1) {
+            fail("Waited too much time for queueFailover replication. " + "Waited "
+                + (System.currentTimeMillis() - start) + "ms.");
+          }
+          Result[] res2;
+          try (ResultScanner scanner = table.getScanner(new Scan())) {
+            res2 = scanner.next(initialCount * 2);
+          }
+          if (res2.length < initialCount) {
+            if (lastCount < res2.length) {
+              i--; // Don't increment timeout if we make progress
+            } else {
+              i++;
+            }
+            lastCount = res2.length;
+            LOG.info(
+              "Only got " + lastCount + " rows instead of " + initialCount + " current i=" + i);
+            Thread.sleep(SLEEP_TIME * 2);
+          } else {
+            break;
+          }
         }
-        lastCount = res2.length;
-        LOG.info("Only got " + lastCount + " rows instead of " +
-            initialCount + " current i=" + i);
-        Thread.sleep(SLEEP_TIME*2);
-      } else {
-        break;
       }
     }
   }
 
-  private static Thread killARegionServer(final HBaseTestingUtility utility,
-                                          final long timeout, final int rs) {
+  private static Thread killARegionServer(final HBaseTestingUtility utility, final long timeout,
+      final int rs) {
     Thread killer = new Thread() {
       @Override
       public void run() {


[3/6] hbase git commit: HBASE-20333 Provide a shaded client that allows downstream to provide Hadoop needs.

Posted by bu...@apache.org.
HBASE-20333 Provide a shaded client that allows downstream to provide Hadoop needs.


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

Branch: refs/heads/HBASE-20331
Commit: 6eb691b5e1a8a608888cf6e9edf6a5e408f7ca13
Parents: 93b034a
Author: Sean Busbey <bu...@apache.org>
Authored: Tue Apr 24 14:51:12 2018 -0500
Committer: Sean Busbey <bu...@apache.org>
Committed: Wed May 23 13:32:52 2018 -0500

----------------------------------------------------------------------
 .../hbase-shaded-check-invariants/pom.xml       |  5 ++
 .../hbase-shaded-client-byo-hadoop/pom.xml      | 70 ++++++++++++++++++++
 hbase-shaded/hbase-shaded-client/pom.xml        | 35 ++++++++--
 hbase-shaded/hbase-shaded-mapreduce/pom.xml     | 30 ++-------
 hbase-shaded/pom.xml                            |  6 ++
 5 files changed, 115 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/6eb691b5/hbase-shaded/hbase-shaded-check-invariants/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-check-invariants/pom.xml b/hbase-shaded/hbase-shaded-check-invariants/pom.xml
index 7ba4a41..287a986 100644
--- a/hbase-shaded/hbase-shaded-check-invariants/pom.xml
+++ b/hbase-shaded/hbase-shaded-check-invariants/pom.xml
@@ -48,6 +48,11 @@
       <artifactId>hbase-shaded-mapreduce</artifactId>
       <version>${project.version}</version>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hbase</groupId>
+      <artifactId>hbase-shaded-client-byo-hadoop</artifactId>
+      <version>${project.version}</version>
+    </dependency>
     <!-- parent pom defines these for children. :( :( :( -->
     <dependency>
       <groupId>com.github.stephenc.findbugs</groupId>

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eb691b5/hbase-shaded/hbase-shaded-client-byo-hadoop/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-client-byo-hadoop/pom.xml b/hbase-shaded/hbase-shaded-client-byo-hadoop/pom.xml
new file mode 100644
index 0000000..c51a1af
--- /dev/null
+++ b/hbase-shaded/hbase-shaded-client-byo-hadoop/pom.xml
@@ -0,0 +1,70 @@
+<project xmlns="http://maven.apache.org/POM/4.0.0"
+         xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <!--
+      /**
+       * 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.
+       */
+      -->
+    <modelVersion>4.0.0</modelVersion>
+    <parent>
+        <artifactId>hbase-shaded</artifactId>
+        <groupId>org.apache.hbase</groupId>
+        <version>3.0.0-SNAPSHOT</version>
+        <relativePath>..</relativePath>
+    </parent>
+    <artifactId>hbase-shaded-client-byo-hadoop</artifactId>
+    <name>Apache HBase - Shaded - Client</name>
+    <build>
+        <plugins>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-site-plugin</artifactId>
+                <configuration>
+                    <skip>true</skip>
+                </configuration>
+            </plugin>
+            <plugin>
+                <!--Make it so assembly:single does nothing in here-->
+                <artifactId>maven-assembly-plugin</artifactId>
+                <configuration>
+                    <skipAssembly>true</skipAssembly>
+                </configuration>
+            </plugin>
+        </plugins>
+    </build>
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.hbase</groupId>
+            <artifactId>hbase-client</artifactId>
+        </dependency>
+    </dependencies>
+
+    <profiles>
+        <profile>
+            <id>release</id>
+            <build>
+                <plugins>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-shade-plugin</artifactId>
+                    </plugin>
+                </plugins>
+            </build>
+        </profile>
+    </profiles>
+</project>

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eb691b5/hbase-shaded/hbase-shaded-client/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-client/pom.xml b/hbase-shaded/hbase-shaded-client/pom.xml
index 72a5b60..5ac3ef5 100644
--- a/hbase-shaded/hbase-shaded-client/pom.xml
+++ b/hbase-shaded/hbase-shaded-client/pom.xml
@@ -28,7 +28,7 @@
         <relativePath>..</relativePath>
     </parent>
     <artifactId>hbase-shaded-client</artifactId>
-    <name>Apache HBase - Shaded - Client</name>
+    <name>Apache HBase - Shaded - Client (with Hadoop bundled)</name>
     <build>
         <plugins>
             <plugin>
@@ -51,6 +51,7 @@
         <dependency>
             <groupId>org.apache.hbase</groupId>
             <artifactId>hbase-client</artifactId>
+            <version>${project.version}</version>
         </dependency>
     </dependencies>
 
@@ -59,10 +60,34 @@
             <id>release</id>
             <build>
                 <plugins>
-                    <plugin>
-                        <groupId>org.apache.maven.plugins</groupId>
-                        <artifactId>maven-shade-plugin</artifactId>
-                    </plugin>
+                <plugin>
+                    <groupId>org.apache.maven.plugins</groupId>
+                    <artifactId>maven-shade-plugin</artifactId>
+                    <executions>
+                        <execution>
+                            <id>aggregate-into-a-jar-with-relocated-third-parties</id>
+                            <configuration>
+                                <artifactSet>
+                                    <excludes>
+                                        <!--
+                                          Tell the shade plugin that in this case we want to include hadoop 
+                                          by leaving out the exclude.
+                                          -->
+                                        <!-- The rest of these should be kept in sync with the parent pom -->
+                                        <exclude>org.apache.hbase:hbase-resource-bundle</exclude>
+                                        <exclude>org.slf4j:*</exclude>
+                                        <exclude>com.google.code.findbugs:*</exclude>
+                                        <exclude>com.github.stephenc.findbugs:*</exclude>
+                                        <exclude>org.apache.htrace:*</exclude>
+                                        <exclude>org.apache.yetus:*</exclude>
+                                        <exclude>log4j:*</exclude>
+                                        <exclude>commons-logging:*</exclude>
+                                    </excludes>
+                                </artifactSet>
+                            </configuration>
+                        </execution>
+                    </executions>
+                </plugin>
                 </plugins>
             </build>
         </profile>

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eb691b5/hbase-shaded/hbase-shaded-mapreduce/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/hbase-shaded-mapreduce/pom.xml b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
index edc80cd..598f3af 100644
--- a/hbase-shaded/hbase-shaded-mapreduce/pom.xml
+++ b/hbase-shaded/hbase-shaded-mapreduce/pom.xml
@@ -169,32 +169,10 @@
             <id>release</id>
             <build>
                 <plugins>
-                <!-- Tell the shade plugin we want to leave Hadoop as a dependency -->
-                <plugin>
-                    <groupId>org.apache.maven.plugins</groupId>
-                    <artifactId>maven-shade-plugin</artifactId>
-                    <executions>
-                        <execution>
-                            <id>aggregate-into-a-jar-with-relocated-third-parties</id>
-                            <configuration>
-                                <artifactSet>
-                                    <excludes>
-                                        <exclude>org.apache.hadoop:*</exclude>
-                                        <!-- The rest of these should be kept in sync with the parent pom -->
-                                        <exclude>org.apache.hbase:hbase-resource-bundle</exclude>
-                                        <exclude>org.slf4j:*</exclude>
-                                        <exclude>com.google.code.findbugs:*</exclude>
-                                        <exclude>com.github.stephenc.findbugs:*</exclude>
-                                        <exclude>org.apache.htrace:*</exclude>
-                                        <exclude>org.apache.yetus:*</exclude>
-                                        <exclude>log4j:*</exclude>
-                                        <exclude>commons-logging:*</exclude>
-                                    </excludes>
-                                </artifactSet>
-                            </configuration>
-                        </execution>
-                    </executions>
-                </plugin>
+                    <plugin>
+                        <groupId>org.apache.maven.plugins</groupId>
+                        <artifactId>maven-shade-plugin</artifactId>
+                    </plugin>
                 </plugins>
             </build>
         </profile>

http://git-wip-us.apache.org/repos/asf/hbase/blob/6eb691b5/hbase-shaded/pom.xml
----------------------------------------------------------------------
diff --git a/hbase-shaded/pom.xml b/hbase-shaded/pom.xml
index 93b122f..9eb30e0 100644
--- a/hbase-shaded/pom.xml
+++ b/hbase-shaded/pom.xml
@@ -39,6 +39,7 @@
       <shaded.prefix>org.apache.hadoop.hbase.shaded</shaded.prefix>
     </properties>
     <modules>
+        <module>hbase-shaded-client-byo-hadoop</module>
         <module>hbase-shaded-client</module>
         <module>hbase-shaded-mapreduce</module>
         <module>hbase-shaded-check-invariants</module>
@@ -131,6 +132,11 @@
                                 <shadeTestJar>false</shadeTestJar>
                                 <artifactSet>
                                     <excludes>
+                                        <!-- default to excluding Hadoop, have module that want
+                                             to include it redefine the exclude list -->
+                                        <exclude>org.apache.hadoop:*</exclude>
+                                        <!-- the rest of this needs to be kept in sync with any
+                                             hadoop-including module -->
                                         <exclude>org.apache.hbase:hbase-resource-bundle</exclude>
                                         <exclude>org.slf4j:*</exclude>
                                         <exclude>com.google.code.findbugs:*</exclude>