You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by jm...@apache.org on 2016/01/14 18:08:51 UTC

[37/50] [abbrv] hbase git commit: HBASE-13525 replace test-patch with customizations needed to use yetus

HBASE-13525 replace test-patch with customizations needed to use yetus

Signed-off-by: stack <st...@apache.org>


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

Branch: refs/heads/trunk
Commit: 164aeb53992150f0336f17a32ecb2fb733495964
Parents: 28c2b18
Author: Sean Busbey <bu...@apache.org>
Authored: Sun Dec 27 23:26:49 2015 -0600
Committer: Sean Busbey <bu...@cloudera.com>
Committed: Fri Jan 8 07:39:40 2016 -0600

----------------------------------------------------------------------
 dev-support/hbase-personality.sh  |  345 ++++++++++
 dev-support/test-patch.properties |   35 --
 dev-support/test-patch.sh         | 1070 --------------------------------
 pom.xml                           |    3 +
 4 files changed, 348 insertions(+), 1105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/dev-support/hbase-personality.sh
----------------------------------------------------------------------
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
new file mode 100755
index 0000000..2d31fd5
--- /dev/null
+++ b/dev-support/hbase-personality.sh
@@ -0,0 +1,345 @@
+#!/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.
+
+# You'll need a local installation of
+# [Apache Yetus' precommit checker](http://yetus.apache.org/documentation/0.1.0/#yetus-precommit)
+# to use this personality.
+#
+# Download from: http://yetus.apache.org/downloads/ . You can either grab the source artifact and
+# build from it, or use the convenience binaries provided on that download page.
+#
+# To run against, e.g. HBASE-15074 you'd then do
+# ```bash
+# test-patch --personality=dev-support/hbase-personality.sh HBASE-15074
+# ```
+#
+# If you want to skip the ~1 hour it'll take to do all the hadoop API checks, use
+# ```bash
+# test-patch  --plugins=all,-hadoopcheck --personality=dev-support/hbase-personality.sh HBASE-15074
+# ````
+#
+# pass the `--jenkins` flag if you want to allow test-patch to destructively alter local working
+# directory / branch in order to have things match what the issue patch requests.
+
+personality_plugins "all"
+
+function personality_globals
+{
+  #shellcheck disable=SC2034
+  PROJECT_NAME=hbase
+  #shellcheck disable=SC2034
+  PATCH_BRANCH_DEFAULT=master
+  #shellcheck disable=SC2034
+  JIRA_ISSUE_RE='^HBASE-[0-9]+$'
+  #shellcheck disable=SC2034
+  GITHUB_REPO="apache/hbase"
+
+  # TODO use PATCH_BRANCH to select hadoop versions to use.
+  # All supported Hadoop versions that we want to test the compilation with
+  HBASE_HADOOP_VERSIONS="2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.1 2.6.2 2.6.3 2.7.1"
+
+  # TODO use PATCH_BRANCH to select jdk versions to use.
+
+  # Override the maven options
+  MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
+
+}
+
+function personality_modules
+{
+  local repostatus=$1
+  local testtype=$2
+  local extra=""
+
+  yetus_debug "Personality: ${repostatus} ${testtype}"
+
+  clear_personality_queue
+
+  extra="-DHBasePatchProcess"
+
+  if [[ ${repostatus} == branch
+     && ${testtype} == mvninstall ]];then
+     personality_enqueue_module . ${extra}
+     return
+  fi
+
+  if [[ ${testtype} = findbugs ]]; then
+    for module in ${CHANGED_MODULES}; do
+      # skip findbugs on hbase-shell
+      if [[ ${module} == hbase-shell ]]; then
+        continue
+      else
+        # shellcheck disable=SC2086
+        personality_enqueue_module ${module} ${extra}
+      fi
+    done
+    return
+  fi
+
+  if [[ ${testtype} = unit ]]; then
+    extra="${extra} -PrunAllTests"
+
+    # Inject the jenkins build-id for our surefire invocations
+    # Used by zombie detection stuff, even though we're not including that yet.
+    if [ -n "${BUILD_ID}" ]; then
+      extra="${extra} -Dbuild.id=${BUILD_ID}"
+    fi
+  fi
+
+  for module in ${CHANGED_MODULES}; do
+    # shellcheck disable=SC2086
+    personality_enqueue_module ${module} ${extra}
+  done
+}
+
+###################################################
+# Below here are our one-off tests specific to hbase.
+# TODO break them into individual files so it's easier to maintain them?
+
+# TODO line length check? could ignore all java files since checkstyle gets them.
+
+###################################################
+
+add_test_type hadoopcheck
+
+function hadoopcheck_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ \.java$ ]]; then
+    add_test hadoopcheck
+  fi
+}
+
+function hadoopcheck_rebuild
+{
+  local repostatus=$1
+  local hadoopver
+  local logfile
+  local count
+  local result=0
+
+  if [[ "${repostatus}" = branch ]]; then
+    return 0
+  fi
+
+  big_console_header "Compiling against various Hadoop versions"
+
+  export MAVEN_OPTS="${MAVEN_OPTS}"
+  for hadoopver in ${HBASE_HADOOP_VERSIONS}; do
+    logfile="${PATCH_DIR}/patch-javac-${hadoopver}.txt"
+    echo_and_redirect "${logfile}" \
+      "${MAVEN}" clean install \
+        -DskipTests -DHBasePatchProcess \
+        -Dhadoop-two.version="${hadoopver}"
+    count=$(${GREP} -c ERROR "${logfile}")
+    if [[ ${count} -gt 0 ]]; then
+      add_vote_table -1 hadoopcheck "Patch causes ${count} errors with Hadoop v${hadoopver}."
+      ((result=result+1))
+    fi
+  done
+
+  if [[ ${result} -gt 0 ]]; then
+    return 1
+  fi
+
+  add_vote_table +1 hadoopcheck "Patch does not cause any errors with Hadoop ${HBASE_HADOOP_VERSIONS}."
+  return 0
+}
+
+######################################
+
+# TODO if we need th protoc check, we probably need to check building all the modules that rely on hbase-protocol
+add_test_type hbaseprotoc
+
+function hbaseprotoc_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ \.proto$ ]]; then
+    add_test hbaseprotoc
+  fi
+}
+
+function hbaseprotoc_rebuild
+{
+  local i=0
+  local fn
+  local module
+  local logfile
+  local count
+  local result
+
+  if [[ "${repostatus}" = branch ]]; then
+    return 0
+  fi
+
+  verify_needed_test hbaseprotoc
+  if [[ $? == 0 ]]; then
+    return 0
+  fi
+
+  big_console_header "Patch HBase protoc plugin"
+
+  start_clock
+
+
+  personality_modules patch hbaseprotoc
+  modules_workers patch hbaseprotoc compile -DskipTests -Pcompile-protobuf -X -DHBasePatchProcess
+
+  # shellcheck disable=SC2153
+  until [[ $i -eq ${#MODULE[@]} ]]; do
+    if [[ ${MODULE_STATUS[${i}]} == -1 ]]; then
+      ((result=result+1))
+      ((i=i+1))
+      continue
+    fi
+    module=${MODULE[$i]}
+    fn=$(module_file_fragment "${module}")
+    logfile="${PATCH_DIR}/patch-hbaseprotoc-${fn}.txt"
+
+    count=$(${GREP} -c ERROR "${logfile}")
+
+    if [[ ${count} -gt 0 ]]; then
+      module_status ${i} -1 "patch-hbaseprotoc-${fn}.txt" "Patch generated "\
+        "${count} new protoc errors in ${module}."
+      ((result=result+1))
+    fi
+    ((i=i+1))
+  done
+
+  modules_messages patch hbaseprotoc true
+  if [[ ${result} -gt 0 ]]; then
+    return 1
+  fi
+  return 0
+}
+
+######################################
+
+add_test_type hbaseanti
+
+function hbaseanti_filefilter
+{
+  local filename=$1
+
+  if [[ ${filename} =~ \.java$ ]]; then
+    add_test hbaseanti
+  fi
+}
+
+function hbaseanti_patchfile
+{
+  local patchfile=$1
+  local warnings
+  local result
+
+  verify_needed_test hbaseanti
+  if [[ $? == 0 ]]; then
+    return 0
+  fi
+
+  big_console_header "Checking for known anti-patterns"
+
+  start_clock
+
+  warnings=$(${GREP} 'new TreeMap<byte.*()' "${patchfile}")
+  if [[ ${warnings} -gt 0 ]]; then
+    add_vote_table -1 hbaseanti "" "The patch appears to have anti-pattern where BYTES_COMPARATOR was omitted: ${warnings}."
+    ((result=result+1))
+  fi
+
+  warnings=$(${GREP} 'import org.apache.hadoop.classification' "${patchfile}")
+  if [[ ${warnings} -gt 0 ]]; then
+    add_vote_table -1 hbaseanti "" "The patch appears use Hadoop classification instead of HBase: ${warnings}."
+    ((result=result+1))
+  fi
+
+  if [[ ${result} -gt 0 ]]; then
+    return 1
+  fi
+
+  add_vote_table +1 hbaseanti "" "Patch does not have any anti-patterns."
+  return 0
+}
+
+# Work around HBASE-15042
+function mvnsite_filefilter
+{
+  local filename=$1
+
+  if [[ ${BUILDTOOL} = maven ]]; then
+    if [[ ${filename} =~ src/main/site || ${filename} =~ src/main/asciidoc ]]; then
+      yetus_debug "tests/mvnsite: ${filename}"
+      add_test mvnsite
+    fi
+  fi
+}
+
+## This is named so that yetus will check us right after running tests.
+## Essentially, we check for normal failures and then we look for zombies.
+#function hbase_unit_logfilter
+#{
+#  declare testtype="unit"
+#  declare input=$1
+#  declare output=$2
+#  declare processes
+#  declare process_output
+#  declare zombies
+#  declare zombie_count=0
+#  declare zombie_process
+#
+#  yetus_debug "in hbase-specific unit logfilter."
+#
+#  # pass-through to whatever is counting actual failures
+#  if declare -f ${BUILDTOOL}_${testtype}_logfilter >/dev/null; then
+#    "${BUILDTOOL}_${testtype}_logfilter" "${input}" "${output}"
+#  elif declare -f ${testtype}_logfilter >/dev/null; then
+#    "${testtype}_logfilter" "${input}" "${output}"
+#  fi
+#
+#  start_clock
+#  if [ -n "${BUILD_ID}" ]; then
+#    yetus_debug "Checking for zombie test processes."
+#    processes=$(jps -v | "${GREP}" surefirebooter | "${GREP}" -e "hbase.build.id=${BUILD_ID}")
+#    if [ -n "${processes}" ] && [ "$(echo "${processes}" | wc -l)" -gt 0 ]; then
+#      yetus_warn "Found some suspicious process(es). Waiting a bit to see if they're just slow to stop."
+#      yetus_debug "${processes}"
+#      sleep 30
+#      #shellcheck disable=SC2016
+#      for pid in $(echo "${processes}"| ${AWK} '{print $1}'); do
+#        # Test our zombie still running (and that it still an hbase build item)
+#        process_output=$(ps -p "${pid}" | tail +2 | "${GREP}" -e "hbase.build.id=${BUILD_ID}")
+#        if [[ -n "${process_output}" ]]; then
+#          yetus_error "Zombie: ${process_output}"
+#          ((zombie_count = zombie_count + 1))
+#          zombie_process=$(jstack "${pid}" | "${GREP}" -e "\.Test" | "${GREP}" -e "\.java"| head -3)
+#          zombies="${zombies} ${zombie_process}"
+#        fi
+#      done
+#    fi
+#    if [ "${zombie_count}" -ne 0 ]; then
+#      add_vote_table -1 zombies "There are ${zombie_count} zombie test(s)"
+#      populate_test_table "zombie unit tests" "${zombies}"
+#    else
+#      yetus_info "Zombie check complete. All test runs exited normally."
+#      stop_clock
+#    fi
+#  else
+#    add_vote_table -0 zombies "There is no BUILD_ID env variable; can't check for zombies."
+#  fi
+#
+#}

http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/dev-support/test-patch.properties
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.properties b/dev-support/test-patch.properties
deleted file mode 100644
index bc29896..0000000
--- a/dev-support/test-patch.properties
+++ /dev/null
@@ -1,35 +0,0 @@
-# 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.
-
-MAVEN_OPTS="${MAVEN_OPTS:-"-Xmx3100M"}"
-
-# The number of acceptable warning for *all* modules
-# Please update the per-module test-patch.properties if you update this file.
-
-OK_RELEASEAUDIT_WARNINGS=0
-# Allow four warnings.  Javadoc complains about sun.misc.Unsafe use.
-# See HBASE-7457, HBASE-13761
-# Allow 2 additional warnings for Scala stub notice about MR. See HBASE-13992
-OK_JAVADOC_WARNINGS=9
-
-MAX_LINE_LENGTH=100
-
-# All supported branches for testing with precommit build
-# branch-1.x should apprear before branch-1 since the latter is a prefix
-BRANCH_NAMES="0.94 0.98 branch-1.0 branch-1.1 branch-1.2 branch-1 master hbase-12439 hbase-11339"
-
-# All supported Hadoop versions that we want to test the compilation with
-HADOOP2_VERSIONS="2.4.0 2.4.1 2.5.0 2.5.1 2.5.2 2.6.0 2.6.1 2.7.0 2.7.1"
-HADOOP3_VERSIONS="3.0.0-SNAPSHOT"

http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/dev-support/test-patch.sh
----------------------------------------------------------------------
diff --git a/dev-support/test-patch.sh b/dev-support/test-patch.sh
deleted file mode 100755
index d0c0346..0000000
--- a/dev-support/test-patch.sh
+++ /dev/null
@@ -1,1070 +0,0 @@
-#!/usr/bin/env bash
-# Licensed to the Apache Software Foundation (ASF) under one
-# or more contributor license agreements.  See the NOTICE file
-# distributed with this work for additional information
-# regarding copyright ownership.  The ASF licenses this file
-# to you under the Apache License, Version 2.0 (the
-# "License"); you may not use this file except in compliance
-# with the License.  You may obtain a copy of the License at
-#
-#   http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing,
-# software distributed under the License is distributed on an
-# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-# KIND, either express or implied.  See the License for the
-# specific language governing permissions and limitations
-# under the License.
-
-
-#set -x
-
-### Setup some variables.  
-### GIT_COMMIT and BUILD_URL are set by Hudson if it is run by patch process
-### Read variables from properties file
-bindir=$(dirname $0)
-
-# Defaults
-if [ -z "$MAVEN_HOME" ]; then
-  MVN=mvn
-else
-  MVN=$MAVEN_HOME/bin/mvn
-fi
-
-NEWLINE=$'\n'
-
-PROJECT_NAME=HBase
-JENKINS=false
-MOVE_PATCH_DIR=true
-PATCH_DIR=/tmp
-BASEDIR=$(pwd)
-BRANCH_NAME="master"
-
-. $BASEDIR/dev-support/test-patch.properties
-
-PS=${PS:-ps}
-AWK=${AWK:-awk}
-WGET=${WGET:-wget}
-GREP=${GREP:-grep}
-EGREP=${EGREP:-egrep}
-PATCH=${PATCH:-patch}
-JIRACLI=${JIRA:-jira}
-FINDBUGS_HOME=${FINDBUGS_HOME}
-FORREST_HOME=${FORREST_HOME}
-ECLIPSE_HOME=${ECLIPSE_HOME}
-GIT=${GIT:-git}
-
-###############################################################################
-printUsage() {
-  echo "Usage: $0 [options] patch-file | defect-number"
-  echo
-  echo "Where:"
-  echo "  patch-file is a local patch file containing the changes to test"
-  echo "  defect-number is a JIRA defect number (e.g. 'HADOOP-1234') to test (Jenkins only)"
-  echo
-  echo "Options:"
-  echo "--patch-dir=<dir>      The directory for working and output files (default '/tmp')"
-  echo "--basedir=<dir>        The directory to apply the patch to (default current directory)"
-  echo "--mvn-cmd=<cmd>        The 'mvn' command to use (default \$MAVEN_HOME/bin/mvn, or 'mvn')"
-  echo "--ps-cmd=<cmd>         The 'ps' command to use (default 'ps')"
-  echo "--awk-cmd=<cmd>        The 'awk' command to use (default 'awk')"
-  echo "--grep-cmd=<cmd>       The 'grep' command to use (default 'grep')"
-  echo "--patch-cmd=<cmd>      The 'patch' command to use (default 'patch')"
-  echo "--findbugs-home=<path> Findbugs home directory (default FINDBUGS_HOME environment variable)"
-  echo "--forrest-home=<path>  Forrest home directory (default FORREST_HOME environment variable)"
-  echo "--dirty-workspace      Allow the local workspace to have uncommitted changes"
-  echo "--git-cmd=<cmd>        The 'git' command to use (default 'git')"
-  echo
-  echo "Jenkins-only options:"
-  echo "--jenkins              Run by Jenkins (runs tests and posts results to JIRA)"
-  echo "--wget-cmd=<cmd>       The 'wget' command to use (default 'wget')"
-  echo "--jira-cmd=<cmd>       The 'jira' command to use (default 'jira')"
-  echo "--jira-password=<pw>   The password for the 'jira' command"
-  echo "--eclipse-home=<path>  Eclipse home directory (default ECLIPSE_HOME environment variable)"
-}
-
-###############################################################################
-parseArgs() {
-  for i in $*
-  do
-    case $i in
-    --jenkins)
-      JENKINS=true
-      ;;
-    --no-move-patch-dir)
-      MOVE_PATCH_DIR=false
-      ;;
-    --patch-dir=*)
-      PATCH_DIR=${i#*=}
-      ;;
-    --basedir=*)
-      BASEDIR=${i#*=}
-      ;;
-    --mvn-cmd=*)
-      MVN=${i#*=}
-      ;;
-    --ps-cmd=*)
-      PS=${i#*=}
-      ;;
-    --awk-cmd=*)
-      AWK=${i#*=}
-      ;;
-    --wget-cmd=*)
-      WGET=${i#*=}
-      ;;
-    --grep-cmd=*)
-      GREP=${i#*=}
-      ;;
-    --patch-cmd=*)
-      PATCH=${i#*=}
-      ;;
-    --jira-cmd=*)
-      JIRACLI=${i#*=}
-      ;;
-    --jira-password=*)
-      JIRA_PASSWD=${i#*=}
-      ;;
-    --findbugs-home=*)
-      FINDBUGS_HOME=${i#*=}
-      ;;
-    --forrest-home=*)
-      FORREST_HOME=${i#*=}
-      ;;
-    --eclipse-home=*)
-      ECLIPSE_HOME=${i#*=}
-      ;;
-    --dirty-workspace)
-      DIRTY_WORKSPACE=true
-      ;;
-    --git-cmd=*)
-      GIT=${i#*=}
-      ;;
-    *)
-      PATCH_OR_DEFECT=$i
-      ;;
-    esac
-  done
-  if [ -z "$PATCH_OR_DEFECT" ]; then
-    printUsage
-    exit 1
-  fi
-  if [[ $JENKINS == "true" ]] ; then
-    echo "Running in Jenkins mode"
-    defect=$PATCH_OR_DEFECT
-    ECLIPSE_PROPERTY="-Declipse.home=$ECLIPSE_HOME"
-  else
-    echo "Running in developer mode"
-    JENKINS=false
-    ### PATCH_FILE contains the location of the patchfile
-    PATCH_FILE=$PATCH_OR_DEFECT
-    if [[ ! -e "$PATCH_FILE" ]] ; then
-      echo "Unable to locate the patch file $PATCH_FILE"
-      cleanupAndExit 0
-    fi
-    ### Check if $PATCH_DIR exists. If it does not exist, create a new directory
-    if [[ ! -e "$PATCH_DIR" ]] ; then
-      mkdir "$PATCH_DIR"
-      if [[ $? == 0 ]] ; then 
-        echo "$PATCH_DIR has been created"
-      else
-        echo "Unable to create $PATCH_DIR"
-        cleanupAndExit 0
-      fi
-    fi
-    ### Obtain the patch filename to append it to the version number
-    defect=`basename $PATCH_FILE`
-  fi
-}
-
-###############################################################################
-checkout () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Testing patch for ${defect}."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  ### When run by a developer, if the workspace contains modifications, do not continue
-  ### unless the --dirty-workspace option was set
-  if [[ $JENKINS == "false" ]] ; then
-    if [[ -z $DIRTY_WORKSPACE ]] ; then
-      # Ref http://stackoverflow.com/a/2659808 for details on checking dirty status
-      ${GIT} diff-index --quiet HEAD
-      if [[ $? -ne 0 ]] ; then
-        uncommitted=`${GIT} diff --name-only HEAD`
-        uncommitted="You have the following files with uncommitted changes:${NEWLINE}${uncommitted}"
-      fi
-      untracked="$(${GIT} ls-files --exclude-standard --others)" && test -z "${untracked}"
-      if [[ $? -ne 0 ]] ; then
-        untracked="You have untracked and unignored files:${NEWLINE}${untracked}"
-      fi
-      if [[ $uncommitted || $untracked ]] ; then
-        echo "ERROR: can't run in a workspace that contains modifications."
-        echo "Pass the '--dirty-workspace' flag to bypass."
-        echo ""
-        echo "${uncommitted}"
-        echo ""
-        echo "${untracked}"
-        cleanupAndExit 1
-      fi
-    fi
-    echo
-  fi
-  return $?
-}
-
-findBranchNameFromPatchName() {
-  local patchName=$1
-  for LOCAL_BRANCH_NAME in $BRANCH_NAMES; do
-    if [[ $patchName =~ /jira/secure/attachment/[0-9]*/.*$LOCAL_BRANCH_NAME ]]; then
-      BRANCH_NAME=$LOCAL_BRANCH_NAME
-      break
-    fi
-  done
-  return 0
-}
-
-checkoutBranch() {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Testing patch on branch ${BRANCH_NAME}."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  if [[ $JENKINS == "true" ]] ; then
-    if [[ "$BRANCH_NAME" != "master" ]]; then
-      echo "origin/${BRANCH_NAME} HEAD is commit `${GIT} rev-list origin/${BRANCH_NAME} -1`"
-      echo "${GIT} checkout -f  `${GIT} rev-list origin/${BRANCH_NAME} -1`"
-      ${GIT} checkout -f  `${GIT} rev-list origin/${BRANCH_NAME} -1`
-      echo "${GIT} status"
-      ${GIT} status
-    fi
-  fi
-}
-
-###############################################################################
-###  Collect findbugs reports
-collectFindbugsReports() {
-  name=$1
-  basedir=$2
-  patch_dir=$3
-  for file in $(find $basedir -name findbugsXml.xml)
-  do
-    relative_file=${file#$basedir/} # strip leading $basedir prefix
-    if [ ! $relative_file == "target/findbugsXml.xml" ]; then
-      module_suffix=${relative_file%/target/findbugsXml.xml} # strip trailing path
-      module_suffix=`basename ${module_suffix}`
-    fi
-
-    cp $file $patch_dir/${name}FindbugsWarnings${module_suffix}.xml
-    $FINDBUGS_HOME/bin/setBugDatabaseInfo -name $name \
-      $patch_dir/${name}FindbugsWarnings${module_suffix}.xml \
-      $patch_dir/${name}FindbugsWarnings${module_suffix}.xml
-  done
-  xml_file=$patch_dir/${name}FindbugsWarnings.xml
-  html_file=$patch_dir/${name}FindbugsWarnings.html
-  $FINDBUGS_HOME/bin/unionBugs -withMessages \
-	 -output $xml_file $patch_dir/${name}FindbugsWarnings*.xml
-  $FINDBUGS_HOME/bin/convertXmlToText -html $xml_file $html_file
-  file $xml_file $html_file
-}
-
-###############################################################################
-setup () {
-  ### Download latest patch file (ignoring .htm and .html) when run from patch process
-  if [[ $JENKINS == "true" ]] ; then
-    $WGET -q -O $PATCH_DIR/jira http://issues.apache.org/jira/browse/$defect
-    if [[ `$GREP -c 'Patch Available' $PATCH_DIR/jira` == 0 ]] ; then
-      echo "$defect is not \"Patch Available\".  Exiting."
-      cleanupAndExit 0
-    fi
-    relativePatchURL=`$GREP -o '"/jira/secure/attachment/[0-9]*/[^"]*' $PATCH_DIR/jira | $EGREP '(\.txt$|\.patch$|\.diff$)' | sort | tail -1 | $GREP -o '/jira/secure/attachment/[0-9]*/[^"]*'`
-    patchURL="http://issues.apache.org${relativePatchURL}"
-    patchNum=`echo $patchURL | $GREP -o '[0-9]*/' | $GREP -o '[0-9]*'`
-    # ensure attachment has not already been tested
-    ATTACHMENT_ID=$(basename $(dirname $patchURL))
-    if grep -q "ATTACHMENT ID: $ATTACHMENT_ID" $PATCH_DIR/jira
-    then
-      echo "Attachment $ATTACHMENT_ID is already tested for $defect"
-      exit 1
-    fi
-    echo "$defect patch is being downloaded at `date` from"
-    echo "$patchURL"
-    $WGET -q -O $PATCH_DIR/patch $patchURL
-    VERSION=${GIT_COMMIT}_${defect}_PATCH-${patchNum}
-    findBranchNameFromPatchName ${relativePatchURL}
-    checkoutBranch
-    JIRA_COMMENT="Here are the results of testing the latest attachment 
-  $patchURL
-  against ${BRANCH_NAME} branch at commit ${GIT_COMMIT}.
-  ATTACHMENT ID: ${ATTACHMENT_ID}"
-
-  ### Copy the patch file to $PATCH_DIR
-  else
-    VERSION=PATCH-${defect}
-    cp $PATCH_FILE $PATCH_DIR/patch
-    if [[ $? == 0 ]] ; then
-      echo "Patch file $PATCH_FILE copied to $PATCH_DIR"
-    else
-      echo "Could not copy $PATCH_FILE to $PATCH_DIR"
-      cleanupAndExit 0
-    fi
-  fi
-  ### exit if warnings are NOT defined in the properties file
-  if [[ -z "$OK_JAVADOC_WARNINGS" ]] || [[ -z $OK_RELEASEAUDIT_WARNINGS ]] ; then
-    echo "Please define the following properties in test-patch.properties file"
-	 echo  "OK_RELEASEAUDIT_WARNINGS"
-	 echo  "OK_JAVADOC_WARNINGS"
-    cleanupAndExit 1
-  fi
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo " Pre-build master to verify stability and javac warnings"
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package checkstyle:checkstyle-aggregate findbugs:findbugs -DskipTests \
- -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  # build core and tests
-  $MVN clean package checkstyle:checkstyle-aggregate findbugs:findbugs -DskipTests \
-	 -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/trunkJavacWarnings.txt 2>&1
-  if [[ $? != 0 ]] ; then
-    echo "mvn exit code was $?"
-    ERR=`$GREP -A 5 'Compilation failure' $PATCH_DIR/trunkJavacWarnings.txt`
-    if [[ ${#ERR} -ge 1 ]] ; then
-      echo "Trunk compilation is broken?
-      {code}$ERR{code}"
-      cleanupAndExit 1
-    fi
-  fi
-  mv target/checkstyle-result.xml $PATCH_DIR/trunkCheckstyle.xml
-  collectFindbugsReports trunk $BASEDIR $PATCH_DIR
-}
-
-###############################################################################
-### Check for @author tags in the patch
-checkAuthor () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking there are no @author tags in the patch."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  authorTags=`$GREP -c -i '@author' $PATCH_DIR/patch`
-  echo "There appear to be $authorTags @author tags in the patch."
-  if [[ $authorTags != 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 @author{color}.  The patch appears to contain $authorTags @author tags which the Hadoop community has agreed to not allow in code contributions."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 @author{color}.  The patch does not contain any @author tags."
-  return 0
-}
-
-###############################################################################
-### Check for tests in the patch
-checkTests () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking there are new or changed tests in the patch."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  testReferences=`$GREP -c -i '/test' $PATCH_DIR/patch`
-  echo "There appear to be $testReferences test files referenced in the patch."
-  if [[ $testReferences == 0 ]] ; then
-    if [[ $JENKINS == "true" ]] ; then
-      patchIsDoc=`$GREP -c -i 'title="documentation' $PATCH_DIR/jira`
-      if [[ $patchIsDoc != 0 ]] ; then
-        echo "The patch appears to be a documentation patch that doesn't require tests."
-        JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+0 tests included{color}.  The patch appears to be a documentation patch that doesn't require tests."
-        return 0
-      fi
-    fi
-    srcReferences=`${GREP} "diff --git" "${PATCH_DIR}/patch" | ${GREP} "src/main" | \
-        ${GREP} -v "src/main/asciidoc" | ${GREP} -v "src/main/site" -c`
-    if [[ $srcReferences == 0 ]] ; then
-      echo "The patch doesn't appear to alter any code that requires tests."
-      JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+0 tests included{color}.  The patch appears to be a documentation, build,
-                        or dev-support patch that doesn't require tests."
-      return 0
-    fi
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 tests included{color}.  The patch doesn't appear to include any new or modified tests.
-                        Please justify why no new tests are needed for this patch.
-                        Also please list what manual steps were performed to verify this patch."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 tests included{color}.  The patch appears to include $testReferences new or modified tests."
-  return 0
-}
-
-###############################################################################
-### Check there are no compilation errors, passing a file to be parsed.
-checkCompilationErrors() {
-  local file=$1
-  hadoopVersion=""
-  if [ "$#" -ne 1 ]; then
-    hadoopVersion="with Hadoop version $2"
-  fi
-  COMPILATION_ERROR=false
-  eval $(awk '/ERROR/ {print "COMPILATION_ERROR=true"}' $file)
-  if $COMPILATION_ERROR ; then
-    ERRORS=$($AWK '/ERROR/ { print $0 }' $file)
-    echo "======================================================================"
-    echo "There are compilation errors $hadoopVersion."
-    echo "======================================================================"
-    echo "$ERRORS"
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javac{color}.  The patch appears to cause mvn compile goal to fail $hadoopVersion.
-
-    Compilation errors resume:
-    $ERRORS
-    "
-    submitJiraComment 1
-    cleanupAndExit 1
-  fi
-}
-
-###############################################################################
-### Check there are no protoc compilation errors, passing a file to be parsed.
-checkProtocCompilationErrors() {
-  local file=$1
-  COMPILATION_ERROR=false
-  eval $(awk '/\[ERROR/ {print "COMPILATION_ERROR=true"}' $file)
-  if $COMPILATION_ERROR ; then
-    ERRORS=$($AWK '/\[ERROR/ { print $0 }' $file)
-    echo "======================================================================"
-    echo "There are Protoc compilation errors."
-    echo "======================================================================"
-    echo "$ERRORS"
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javac{color}.  The patch appears to cause mvn compile-protobuf profile to fail.
-
-    Protoc Compilation errors resume:
-    $ERRORS
-    "
-    cleanupAndExit 1
-  fi
-}
-
-###############################################################################
-### Attempt to apply the patch
-applyPatch () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Applying patch."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
- 
-  export PATCH
-  $BASEDIR/dev-support/smart-apply-patch.sh $PATCH_DIR/patch
-  if [[ $? != 0 ]] ; then
-    echo "PATCH APPLICATION FAILED"
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 patch{color}.  The patch command could not apply the patch."
-    return 1
-  fi
-  return 0
-}
-
-###############################################################################
-### Check against known anti-patterns
-checkAntiPatterns () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking against known anti-patterns."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  warnings=`$GREP 'new TreeMap<byte.*()' $PATCH_DIR/patch`
-  if [[ $warnings != "" ]]; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 Anti-pattern{color}.  The patch appears to have anti-pattern where BYTES_COMPARATOR was omitted: $warnings."
-    return 1
-  fi
-  return 0
-}
-
-###############################################################################
-### Check that there are no incorrect annotations
-checkInterfaceAudience () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking against hadoop InterfaceAudience."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  warnings=`$GREP 'import org.apache.hadoop.classification' $PATCH_DIR/patch`
-  if [[ $warnings != "" ]]; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 InterfaceAudience{color}.  The patch appears to contain InterfaceAudience from hadoop rather than hbase: $warnings."
-    return 1
-  fi
-  return 0
-}
-
-###############################################################################
-### Check there are no javadoc warnings
-checkJavadocWarnings () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched javadoc warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package javadoc:javadoc -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavadocWarnings.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean package javadoc:javadoc -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavadocWarnings.txt 2>&1
-  javadocWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/patchJavadocWarnings.txt | $AWK '/Javadoc Warnings/,EOF' | $GREP warning | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
-  echo ""
-  echo ""
-  echo "There appear to be $javadocWarnings javadoc warnings generated by the patched build."
-
-  ### if current warnings greater than OK_JAVADOC_WARNINGS
-  if [[ $javadocWarnings -gt $OK_JAVADOC_WARNINGS ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javadoc{color}.  The javadoc tool appears to have generated `expr $(($javadocWarnings-$OK_JAVADOC_WARNINGS))` warning messages."
-    # Add javadoc output url
-    JIRA_COMMENT_FOOTER="Javadoc warnings: $BUILD_URL/artifact/patchprocess/patchJavadocWarnings.txt
-$JIRA_COMMENT_FOOTER"
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 javadoc{color}.  The javadoc tool did not generate any warning messages."
-  return 0
-}
-
-checkBuildWithHadoopVersions() {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Building with all supported Hadoop versions ."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  for HADOOP2_VERSION in $HADOOP2_VERSIONS ; do
-    echo "$MVN clean install -DskipTests -D${PROJECT_NAME}PatchProcess -Dhadoop-two.version=$HADOOP2_VERSION > $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt 2>&1"
-    $MVN clean install -DskipTests -D${PROJECT_NAME}PatchProcess -Dhadoop-two.version=$HADOOP2_VERSION > $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt 2>&1
-    checkCompilationErrors $PATCH_DIR/patchJavacWithHadoop-$HADOOP2_VERSION.txt $HADOOP2_VERSION
-  done
-
-  # TODO: add Hadoop3 versions and compilation here when we get the hadoop.profile=3.0 working
-
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 hadoop versions{color}. The patch compiles with all supported hadoop versions ($HADOOP2_VERSIONS)"
-  return 0
-}
-
-###############################################################################
-### Check there are no changes in the number of Javac warnings
-checkJavacWarnings () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched javac warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchJavacWarnings.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean package -DskipTests -D${PROJECT_NAME}PatchProcess  > $PATCH_DIR/patchJavacWarnings.txt 2>&1
-  checkCompilationErrors $PATCH_DIR/patchJavacWarnings.txt
-  ### Compare trunk and patch javac warning numbers
-  if [[ -f $PATCH_DIR/patchJavacWarnings.txt ]] ; then
-    trunkJavacWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/trunkJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
-    patchJavacWarnings=`$GREP '\[WARNING\]' $PATCH_DIR/patchJavacWarnings.txt | $AWK 'BEGIN {total = 0} {total += 1} END {print total}'`
-    echo "There appear to be $trunkJavacWarnings javac compiler warnings before the patch and $patchJavacWarnings javac compiler warnings after applying the patch."
-    if [[ $patchJavacWarnings != "" && $trunkJavacWarnings != "" ]] ; then
-      if [[ $patchJavacWarnings -gt $trunkJavacWarnings ]] ; then
-        JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 javac{color}.  The applied patch generated $patchJavacWarnings javac compiler warnings (more than the master's current $trunkJavacWarnings warnings)."
-        return 1
-      fi
-    fi
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 javac{color}.  The applied patch does not increase the total number of javac compiler warnings."
-  return 0
-}
-
-checkCheckstyleErrors() {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched Checkstyle errors."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  if [[ -f $PATCH_DIR/trunkCheckstyle.xml ]] ; then
-    $MVN package -DskipTests checkstyle:checkstyle-aggregate > /dev/null 2>&1
-    mv target/checkstyle-result.xml $PATCH_DIR/patchCheckstyle.xml
-    mv target/site/checkstyle-aggregate.html $PATCH_DIR
-    mv target/site/checkstyle.css $PATCH_DIR
-    $BASEDIR/dev-support/checkstyle_report.py $PATCH_DIR/trunkCheckstyle.xml $PATCH_DIR/patchCheckstyle.xml
-    if [[ $? -eq 1 ]] ; then
-                JIRA_COMMENT_FOOTER="Checkstyle Errors: $BUILD_URL/artifact/patchprocess/checkstyle-aggregate.html
-
-                $JIRA_COMMENT_FOOTER"
-
-                JIRA_COMMENT="$JIRA_COMMENT
-
-                {color:red}-1 checkstyle{color}.  The applied patch generated new checkstyle errors. Check build console for list of new errors."
-        return 1
-    fi
-  fi
-  JIRA_COMMENT_FOOTER="Checkstyle Errors: $BUILD_URL/artifact/patchprocess/checkstyle-aggregate.html
-
-  $JIRA_COMMENT_FOOTER"
-
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 checkstyle{color}. The applied patch does not generate new checkstyle errors."
-  return 0
-
-}
-###############################################################################
-checkProtocErrors () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining whether there is patched protoc error."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean install -DskipTests -Pcompile-protobuf -X -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchProtocErrors.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean install -DskipTests -Pcompile-protobuf -X -D${PROJECT_NAME}PatchProcess  > $PATCH_DIR/patchProtocErrors.txt 2>&1
-  checkProtocCompilationErrors $PATCH_DIR/patchProtocErrors.txt
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 protoc{color}.  The applied patch does not increase the total number of protoc compiler warnings."
-  return 0
-}
-
-###############################################################################
-### Check there are no changes in the number of release audit (RAT) warnings
-checkReleaseAuditWarnings () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched release audit warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN apache-rat:check -D${PROJECT_NAME}PatchProcess 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN apache-rat:check -D${PROJECT_NAME}PatchProcess 2>&1
-  find $BASEDIR -name rat.txt | xargs cat > $PATCH_DIR/patchReleaseAuditWarnings.txt
-
-  ### Compare trunk and patch release audit warning numbers
-  if [[ -f $PATCH_DIR/patchReleaseAuditWarnings.txt ]] ; then
-    patchReleaseAuditWarnings=`$GREP -c '\!?????' $PATCH_DIR/patchReleaseAuditWarnings.txt`
-    echo ""
-    echo ""
-    echo "There appear to be $OK_RELEASEAUDIT_WARNINGS release audit warnings before the patch and $patchReleaseAuditWarnings release audit warnings after applying the patch."
-    if [[ $patchReleaseAuditWarnings != "" && $OK_RELEASEAUDIT_WARNINGS != "" ]] ; then
-      if [[ $patchReleaseAuditWarnings -gt $OK_RELEASEAUDIT_WARNINGS ]] ; then
-        JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 release audit{color}.  The applied patch generated $patchReleaseAuditWarnings release audit warnings (more than the master's current $OK_RELEASEAUDIT_WARNINGS warnings)."
-        $GREP '\!?????' $PATCH_DIR/patchReleaseAuditWarnings.txt > $PATCH_DIR/patchReleaseAuditProblems.txt
-        echo "Lines that start with ????? in the release audit report indicate files that do not have an Apache license header." >> $PATCH_DIR/patchReleaseAuditProblems.txt
-        JIRA_COMMENT_FOOTER="Release audit warnings: $BUILD_URL/artifact/patchprocess/patchReleaseAuditWarnings.txt
-$JIRA_COMMENT_FOOTER"
-        return 1
-      fi
-    fi
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 release audit{color}.  The applied patch does not increase the total number of release audit warnings."
-  return 0
-}
-
-###############################################################################
-### Check there are no changes in the number of Findbugs warnings
-checkFindbugsWarnings () {
-  findbugs_version=`${FINDBUGS_HOME}/bin/findbugs -version`
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Determining number of patched Findbugs warnings."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  echo "$MVN clean package findbugs:findbugs -D${PROJECT_NAME}PatchProcess" 
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN clean package findbugs:findbugs -D${PROJECT_NAME}PatchProcess -DskipTests < /dev/null
-
-  if [ $? != 0 ] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 findbugs{color}.  The patch appears to cause Findbugs (version ${findbugs_version}) to fail."
-    return 1
-  fi
-
-  collectFindbugsReports patch $BASEDIR $PATCH_DIR 
-  #this files are generated by collectFindbugsReports() named with its first argument
-  patch_xml=$PATCH_DIR/patchFindbugsWarnings.xml
-  trunk_xml=$PATCH_DIR/trunkFindbugsWarnings.xml
-  # combine them to one database
-  combined_xml=$PATCH_DIR/combinedFindbugsWarnings.xml
-  new_xml=$PATCH_DIR/newFindbugsWarnings.xml
-  new_html=$PATCH_DIR/newFindbugsWarnings.html
-  $FINDBUGS_HOME/bin/computeBugHistory -useAnalysisTimes -withMessages \
-	-output $combined_xml $trunk_xml $patch_xml
-  findbugsWarnings=$($FINDBUGS_HOME/bin/filterBugs -first patch $combined_xml $new_xml)
-  findbugsFixedWarnings=$($FINDBUGS_HOME/bin/filterBugs -fixed patch $combined_xml $new_xml)
-  $FINDBUGS_HOME/bin/convertXmlToText -html  $new_xml $new_html
-  file $new_xml $new_html
-  JIRA_COMMENT_FOOTER="Release Findbugs (version ${findbugs_version}) \
-	warnings: $BUILD_URL/artifact/patchprocess/newFindbugsWarnings.html
-$JIRA_COMMENT_FOOTER"
-  ### if current warnings greater than 0, fail
-  if [[ $findbugsWarnings -gt 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 findbugs{color}.  The patch appears to introduce $findbugsWarnings \
- new Findbugs (version ${findbugs_version}) warnings."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 findbugs{color}.  The patch does not introduce any \
- new Findbugs (version ${findbugs_version}) warnings."
-  return 0
-}
-
-###############################################################################
-### Check line lengths
-checkLineLengths () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking that no line have length > $MAX_LINE_LENGTH"
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  #see http://en.wikipedia.org/wiki/Diff#Unified_format
-
-  MAX_LINE_LENGTH_PATCH=`expr $MAX_LINE_LENGTH + 1`
-  lines=`cat $PATCH_DIR/patch | grep "^+" | grep -v "^@@" | grep -v "^+++" | grep -v "import" | grep -v "org.apache.thrift." | grep -v "com.google.protobuf." | grep -v "protobuf.generated" | awk -v len="$MAX_LINE_LENGTH_PATCH" 'length ($0) > len' | head -n 10`
-  ll=`echo "$lines" | wc -l`
-  if [[ "$ll" -gt "1" ]]; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 lineLengths{color}.  The patch introduces the following lines longer than $MAX_LINE_LENGTH:
-    $lines"
-
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 lineLengths{color}.  The patch does not introduce lines longer than $MAX_LINE_LENGTH"
-  return 0
-}
-
-###############################################################################
-### Run the tests
-runTests () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Running tests."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  failed_tests=""
-  echo "$MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  ulimit -a
-  # Need to export this so the zombie subshell picks up current content
-  export JIRA_COMMENT
-  $MVN clean test -Dsurefire.rerunFailingTestsCount=2 -P runAllTests -D${PROJECT_NAME}PatchProcess
-  if [[ $? != 0 ]] ; then
-     ### Find and format names of failed tests
-     failed_tests=`find . -name 'TEST*.xml' | xargs $GREP  -l -E "<failure|<error" | sed -e "s|.*target/surefire-reports/TEST-|                  |g" | sed -e "s|\.xml||g"`
- 
-     JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 core tests{color}.  The patch failed these unit tests:
-     $failed_tests"
-     JIRA_COMMENT=`$BASEDIR/dev-support/zombie-detector.sh ${BUILD_ID}`
-     return 1
-  else
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 core tests{color}.  The patch passed unit tests in $modules."
-    JIRA_COMMENT=`$BASEDIR/dev-support/zombie-detector.sh ${BUILD_ID}`
-    return $?
-  fi
-}
-
-###############################################################################
-### Check docbook site xml
-checkSiteXml () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking Site generation"
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-
-  echo "$MVN package post-site -DskipTests -D${PROJECT_NAME}PatchProcess > $PATCH_DIR/patchSiteOutput.txt 2>&1"
-  export MAVEN_OPTS="${MAVEN_OPTS}"
-  $MVN package post-site -DskipTests -D${PROJECT_NAME}PatchProcess  > $PATCH_DIR/patchSiteOutput.txt 2>&1
-  if [[ $? != 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 site{color}.  The patch appears to cause mvn post-site goal to fail."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 site{color}.  The mvn post-site goal succeeds with this patch."
-  return 0
-}
-
-###############################################################################
-### Run the inject-system-faults target
-checkInjectSystemFaults () {
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Checking the integrity of system test framework code."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  
-  ### Kill any rogue build processes from the last attempt
-  $PS auxwww | $GREP ${PROJECT_NAME}PatchProcess | $AWK '{print $2}' | /usr/bin/xargs -t -I {} /bin/kill -9 {} > /dev/null
-
-  #echo "$ANT_HOME/bin/ant -Dversion="${VERSION}" -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME inject-system-faults"
-  #$ANT_HOME/bin/ant -Dversion="${VERSION}" -DHadoopPatchProcess= -Dtest.junit.output.format=xml -Dtest.output=no -Dcompile.c++=yes -Dforrest.home=$FORREST_HOME inject-system-faults
-  echo "NOP"
-  return 0
-  if [[ $? != 0 ]] ; then
-    JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:red}-1 system test framework{color}.  The patch failed system test framework compile."
-    return 1
-  fi
-  JIRA_COMMENT="$JIRA_COMMENT
-
-    {color:green}+1 system test framework{color}.  The patch passed system test framework compile."
-  return 0
-}
-
-###############################################################################
-### Submit a comment to the defect's Jira
-submitJiraComment () {
-  local result=$1
-  ### Do not output the value of JIRA_COMMENT_FOOTER when run by a developer
-  if [[  $JENKINS == "false" ]] ; then
-    JIRA_COMMENT_FOOTER=""
-  fi
-  if [[ $result == 0 ]] ; then
-    comment="{color:green}+1 overall{color}.  $JIRA_COMMENT
-
-$JIRA_COMMENT_FOOTER"
-  else
-    comment="{color:red}-1 overall{color}.  $JIRA_COMMENT
-
-$JIRA_COMMENT_FOOTER"
-  fi
-  ### Output the test result to the console
-  echo "
-
-
-
-$comment"  
-
-  if [[ $JENKINS == "true" ]] ; then
-    echo ""
-    echo ""
-    echo "======================================================================"
-    echo "======================================================================"
-    echo "    Adding comment to Jira."
-    echo "======================================================================"
-    echo "======================================================================"
-    echo ""
-    echo ""
-    ### Update Jira with a comment
-    export USER=hudson
-    $JIRACLI -s https://issues.apache.org/jira -a addcomment -u hadoopqa -p $JIRA_PASSWD --comment "$comment" --issue $defect
-    $JIRACLI -s https://issues.apache.org/jira -a logout -u hadoopqa -p $JIRA_PASSWD
-  fi
-}
-
-###############################################################################
-### Cleanup files
-cleanupAndExit () {
-  local result=$1
-  if [[ ${JENKINS} == "true" && ${MOVE_PATCH_DIR} == "true" ]] ; then
-    if [ -e "$PATCH_DIR" ] ; then
-      echo "Relocating patch dir into ${BASEDIR}"
-      mv $PATCH_DIR $BASEDIR
-    fi
-  fi
-  echo ""
-  echo ""
-  echo "======================================================================"
-  echo "======================================================================"
-  echo "    Finished build."
-  echo "======================================================================"
-  echo "======================================================================"
-  echo ""
-  echo ""
-  exit $result
-}
-
-###############################################################################
-###############################################################################
-###############################################################################
-
-JIRA_COMMENT=""
-JIRA_COMMENT_FOOTER="Console output: $BUILD_URL/console
-
-This message is automatically generated."
-
-### Check if arguments to the script have been specified properly or not
-parseArgs $@
-cd $BASEDIR
-
-echo "Version of this script: Wed Oct 14 00:29:04 PDT 2015"
-checkout
-RESULT=$?
-echo "RESULT = " $RESULT
-if [[ $JENKINS == "true" ]] ; then
-  if [[ $RESULT != 0 ]] ; then
-    exit 100
-  fi
-fi
-setup
-checkAuthor
-RESULT=$?
-echo "RESULT = " $RESULT
-checkTests
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-applyPatch
-if [[ $? != 0 ]] ; then
-  submitJiraComment 1
-  cleanupAndExit 1
-fi
-
-checkAntiPatterns
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkBuildWithHadoopVersions
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkJavacWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkProtocErrors
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkJavadocWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkCheckstyleErrors
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkInterfaceAudience
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkFindbugsWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkReleaseAuditWarnings
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkLineLengths
-(( RESULT = RESULT + $? ))
-echo "RESULT = " $RESULT
-checkSiteXml
-(( RESULT = RESULT + $?))
-echo "RESULT = " $RESULT
-### Do not call these when run by a developer
-if [[ $JENKINS == "true" ]] ; then
-  runTests
-  (( RESULT = RESULT + $? ))
-  echo "RESULT = " $RESULT
-JIRA_COMMENT_FOOTER="Test results: $BUILD_URL/testReport/
-$JIRA_COMMENT_FOOTER"
-fi
-submitJiraComment $RESULT
-cleanupAndExit $RESULT

http://git-wip-us.apache.org/repos/asf/hbase/blob/164aeb53/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 897b41c..ccc7eb5 100644
--- a/pom.xml
+++ b/pom.xml
@@ -1826,6 +1826,9 @@
             <name>HBasePatchProcess</name>
         </property>
       </activation>
+      <properties>
+        <surefire.rerunFailingTestsCount>2</surefire.rerunFailingTestsCount>
+      </properties>
       <build>
         <plugins>
           <plugin>