You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by nd...@apache.org on 2020/03/04 17:13:03 UTC

[hbase] 01/02: HBASE-23767 Add JDK11 compilation and unit test support to Github precommit

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

ndimiduk pushed a commit to branch HBASE-23876/jdk11-nightly-master
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 31f8086aa7d8b98d3db500bb9b3c8fc0df61a488
Author: Nick Dimiduk <nd...@apache.org>
AuthorDate: Wed Feb 19 16:10:57 2020 -0800

    HBASE-23767 Add JDK11 compilation and unit test support to Github precommit
    
    Rebuild our Dockerfile with support for multiple JDK versions. Use
    multiple stages in the Jenkinsfile instead of yetus's multijdk because
    of YETUS-953. Run those multiple stages in parallel to speed up
    results.
    
    Note that multiple stages means multiple Yetus invocations means
    multiple comments on the PreCommit. This should become more obvious to
    users once we can make use of GitHub Checks API, HBASE-23902.
---
 .editorconfig                                 |   1 +
 dev-support/Jenkinsfile_GitHub                | 463 +++++++++++++++++---------
 dev-support/docker/Dockerfile                 | 194 ++++++++---
 dev-support/hbase-personality.sh              |   3 -
 dev-support/jenkins_precommit_github_yetus.sh | 144 ++++++++
 pom.xml                                       |   1 +
 6 files changed, 610 insertions(+), 196 deletions(-)

diff --git a/.editorconfig b/.editorconfig
index aa6adaa..8c65c96 100644
--- a/.editorconfig
+++ b/.editorconfig
@@ -460,6 +460,7 @@ ij_javascript_while_on_new_line = false
 ij_javascript_wrap_comments = false
 
 [{*.gradle,*.groovy,*.gant,*.gdsl,*.gy,*.gson,Jenkinsfile*}]
+indent_size = 4
 ij_groovy_align_group_field_declarations = false
 ij_groovy_align_multiline_array_initializer_expression = false
 ij_groovy_align_multiline_assignment = false
diff --git a/dev-support/Jenkinsfile_GitHub b/dev-support/Jenkinsfile_GitHub
index df1c8e9..fdd6c60 100644
--- a/dev-support/Jenkinsfile_GitHub
+++ b/dev-support/Jenkinsfile_GitHub
@@ -27,17 +27,31 @@ pipeline {
         buildDiscarder(logRotator(numToKeepStr: '15'))
         timeout (time: 7, unit: 'HOURS')
         timestamps()
-        checkoutToSubdirectory('src')
+        skipDefaultCheckout()
     }
 
     environment {
-        SOURCEDIR = 'src'
-        // will also need to change notification section below
-        PATCHDIR = 'out'
-        DOCKERFILE = "${SOURCEDIR}/dev-support/docker/Dockerfile"
-        YETUS='yetus'
+        SRC_REL = 'src'
+        PATCH_REL = 'output'
+        YETUS_REL = 'yetus'
+        DOCKERFILE_REL = "${SRC_REL}/dev-support/docker/Dockerfile"
+        YETUS_DRIVER_REL = "${SRC_REL}/dev-support/jenkins_precommit_github_yetus.sh"
         // Branch or tag name.  Yetus release tags are 'rel/X.Y.Z'
-        YETUS_VERSION='rel/0.11.1'
+        YETUS_VERSION = 'rel/0.11.1'
+        GENERAL_CHECK_PLUGINS = 'all,-compile,-javac,-javadoc,-jira,-shadedjars,-unit'
+        JDK_SPECIFIC_PLUGINS = 'compile,github,htmlout,javac,javadoc,maven,mvninstall,shadedjars,unit'
+        // output from surefire; sadly the archive function in yetus only works on file names.
+        ARCHIVE_PATTERN_LIST = 'TEST-*.xml,org.apache.h*.txt,*.dumpstream,*.dump'
+        // These tests currently have known failures. Once they burn down to 0, remove from here so that new problems will cause a failure.
+        TESTS_FILTER = 'cc,checkstyle,javac,javadoc,pylint,shellcheck,whitespace,perlcritic,ruby-lint,rubocop,mvnsite'
+        EXCLUDE_TESTS_URL = "${JENKINS_URL}/job/HBase-Find-Flaky-Tests/job/${CHANGE_TARGET}/lastSuccessfulBuild/artifact/excludes"
+
+        // a global view of paths. parallel stages can land on the same host concurrently, so each
+        // stage works in its own subdirectory. there is an "output" under each of these
+        // directories, which we retrieve after the build is complete.
+        WORKDIR_REL_GENERAL_CHECK = 'yetus-general-check'
+        WORKDIR_REL_JDK8_HADOOP2_CHECK = 'yetus-jdk8-hadoop2-check'
+        WORKDIR_REL_JDK11_HADOOP3_CHECK = 'yetus-jdk11-hadoop3-check'
     }
 
     parameters {
@@ -47,163 +61,304 @@ pipeline {
     }
 
     stages {
-        stage ('install yetus') {
-            steps {
-                dir("${WORKSPACE}/${YETUS}") {
-                    checkout([
-                        $class: 'GitSCM',
-                        branches: [[name: "${env.YETUS_VERSION}"]],
-                        userRemoteConfigs: [[ url: 'https://github.com/apache/yetus.git']]]
-                    )
-                }
-            }
-        }
-
-        stage ('precommit-run') {
-            steps {
-                withCredentials(
-                    [usernamePassword(credentialsId: 'apache-hbase-at-github.com',
+        stage ('precommit checks') {
+            parallel {
+                stage ('yetus general check') {
+                    agent {
+                        node {
+                            label 'Hadoop'
+                        }
+                    }
+                    environment {
+                        // customized per parallel stage
+                        PLUGINS = "${GENERAL_CHECK_PLUGINS}"
+                        SET_JAVA_HOME = '/usr/lib/jvm/java-8'
+                        WORKDIR_REL = "${WORKDIR_REL_GENERAL_CHECK}"
+                        // identical for all parallel stages
+                        WORKDIR = "${WORKSPACE}/${WORKDIR_REL}"
+                        YETUSDIR = "${WORKDIR}/${YETUS_REL}"
+                        SOURCEDIR = "${WORKDIR}/${SRC_REL}"
+                        PATCHDIR = "${WORKDIR}/${PATCH_REL}"
+                        BUILD_URL_ARTIFACTS = "artifact/${WORKDIR_REL}/${PATCH_REL}"
+                        DOCKERFILE = "${WORKDIR}/${DOCKERFILE_REL}"
+                        YETUS_DRIVER = "${WORKDIR}/${YETUS_DRIVER_REL}"
+                    }
+                    steps {
+                        dir("${SOURCEDIR}") {
+                            checkout scm
+                        }
+                        dir("${YETUSDIR}") {
+                            checkout([
+                              $class           : 'GitSCM',
+                              branches         : [[name: "${YETUS_VERSION}"]],
+                              userRemoteConfigs: [[url: 'https://github.com/apache/yetus.git']]]
+                            )
+                        }
+                        dir("${WORKDIR}") {
+                            withCredentials([
+                                usernamePassword(
+                                  credentialsId: 'apache-hbase-at-github.com',
                                   passwordVariable: 'GITHUB_PASSWORD',
-                                  usernameVariable: 'GITHUB_USER'),
-                    usernamePassword(credentialsId: 'hbaseqa-at-asf-jira',
-                                        passwordVariable: 'JIRA_PASSWORD',
-                                        usernameVariable: 'JIRA_USER')]) {
-                        sh '''#!/usr/bin/env bash
-                        set -e
-                        TESTPATCHBIN="${WORKSPACE}/${YETUS}/precommit/src/main/shell/test-patch.sh"
-                        # this must be clean for every run
-                        if [[ -d "${WORKSPACE}/${PATCHDIR}" ]]; then
-                          rm -rf "${WORKSPACE}/${PATCHDIR}"
-                        fi
-                        mkdir -p "${WORKSPACE}/${PATCHDIR}"
-
-                        ## Checking on H* machine nonsense
-                        echo "JAVA_HOME: ${JAVA_HOME}"
-                        ls -l "${JAVA_HOME}" || true
-                        echo "MAVEN_HOME: ${MAVEN_HOME}"
-                        echo "maven version:"
-                        mvn --offline --version  || true
-                        mkdir "${PATCHDIR}/machine"
-                       "${SOURCEDIR}/dev-support/gather_machine_environment.sh" "${PATCHDIR}/machine"
-                        ## /H*
-
-                        # If CHANGE_URL is set (e.g., Github Branch Source plugin), process it.
-                        # Otherwise exit, because we don't want HBase to do a
-                        # full build.  We wouldn't normally do this check for smaller
-                        # projects. :)
-                        if [[ -z "${CHANGE_URL}" ]]; then
-                            echo "Full build skipped" > "${WORKSPACE}/${PATCHDIR}/report.html"
-                            exit 0
-                        fi
-                        # enable debug output for yetus
-                        if [[ "true" = "${DEBUG}" ]]; then
-                            YETUS_ARGS+=("--debug")
-                        fi
-                        # If we're doing docker, make sure we don't accidentally pollute the image with a host java path
-                        if [ -n "${JAVA_HOME}" ]; then
-                          unset JAVA_HOME
-                        fi
-                        YETUS_ARGS+=("--patch-dir=${WORKSPACE}/${PATCHDIR}")
-                        # where the source is located
-                        YETUS_ARGS+=("--basedir=${WORKSPACE}/${SOURCEDIR}")
-                        # our project defaults come from a personality file
-                        # which will get loaded automatically by setting the project name
-                        YETUS_ARGS+=("--project=hbase")
-                        # lots of different output formats
-                        YETUS_ARGS+=("--brief-report-file=${WORKSPACE}/${PATCHDIR}/brief.txt")
-                        YETUS_ARGS+=("--console-report-file=${WORKSPACE}/${PATCHDIR}/console.txt")
-                        YETUS_ARGS+=("--html-report-file=${WORKSPACE}/${PATCHDIR}/report.html")
-                        # enable writing back to Github
-                        YETUS_ARGS+=(--github-password="${GITHUB_PASSWORD}")
-                        YETUS_ARGS+=(--github-user=${GITHUB_USER})
-                        # enable writing back to ASF JIRA
-                        YETUS_ARGS+=(--jira-password="${JIRA_PASSWORD}")
-                        YETUS_ARGS+=(--jira-user="${JIRA_USER}")
-                        # auto-kill any surefire stragglers during unit test runs
-                        YETUS_ARGS+=("--reapermode=kill")
-                        YETUS_ARGS+=("--multijdktests=compile")
-                        # set relatively high limits for ASF machines
-                        # changing these to higher values may cause problems
-                        # with other jobs on systemd-enabled machines
-                        YETUS_ARGS+=("--proclimit=10000")
-                        YETUS_ARGS+=("--dockermemlimit=20g")
-                        # -1 findbugs issues that show up prior to the patch being applied
-                        YETUS_ARGS+=("--findbugs-strict-precheck")
-                        # rsync these files back into the archive dir
-                        YETUS_ARGS+=("--archive-list=rat.txt")
-                        # URL for user-side presentation in reports and such to our artifacts
-                        # (needs to match the archive bits below)
-                        YETUS_ARGS+=("--build-url-artifacts=artifact/out")
-                        # plugins to enable
-                        YETUS_ARGS+=("--plugins=all")
-                        # don't let these tests cause -1s because we aren't really paying that
-                        # much attention to them
-                        YETUS_ARGS+=("--tests-filter=ruby-lint,test4tests")
-                        # run in docker mode and specifically point to our
-                        # Dockerfile since we don't want to use the auto-pulled version.
-                        YETUS_ARGS+=("--docker")
-                        YETUS_ARGS+=("--dockerfile=${DOCKERFILE}")
-                        YETUS_ARGS+=("--mvn-custom-repos")
-                        YETUS_ARGS+=("--multijdkdirs=/usr/lib/jvm/java-8-openjdk-amd64")
-                        YETUS_ARGS+=("--findbugs-home=/usr")
-                        YETUS_ARGS+=("--whitespace-eol-ignore-list=.*/generated/.*")
-                        YETUS_ARGS+=("--whitespace-tabs-ignore-list=.*/generated/.*")
-                        YETUS_ARGS+=("--personality=${SOURCEDIR}/dev-support/hbase-personality.sh")
-                        YETUS_ARGS+=("--quick-hadoopcheck")
-                        YETUS_ARGS+=("--skip-errorprone")
-                        # effectively treat dev-support as a custom maven module
-                        YETUS_ARGS+=("--skip-dirs=dev-support")
-                        # help keep the ASF boxes clean
-                        YETUS_ARGS+=("--sentinel")
-                        # use emoji vote so it is easier to find the broken line
-                        YETUS_ARGS+=("--github-use-emoji-vote")
-                        "${TESTPATCHBIN}" "${YETUS_ARGS[@]}"
-                        '''
+                                  usernameVariable: 'GITHUB_USER'
+                                )]) {
+                                sh label: 'test-patch', script: '''
+                                    hostname -a ; pwd ; ls -la
+                                    printenv 2>&1 | sort
+                                    echo "[INFO] Launching Yetus via ${YETUS_DRIVER}"
+                                    "${YETUS_DRIVER}"
+                                '''
+                            }
+                        }
+                    }
+                    post {
+                        always {
+                            // Has to be relative to WORKSPACE.
+                            archiveArtifacts artifacts: "${WORKDIR_REL}/${PATCH_REL}/*", excludes: "${WORKDIR_REL}/${PATCH_REL}/precommit"
+                            archiveArtifacts artifacts: "${WORKDIR_REL}/${PATCH_REL}/**/*", excludes: "${WORKDIR_REL}/${PATCH_REL}/precommit/**/*"
+                            publishHTML target: [
+                              allowMissing: true,
+                              keepAll: true,
+                              alwaysLinkToLastBuild: true,
+                              // Has to be relative to WORKSPACE
+                              reportDir: "${WORKDIR_REL}/${PATCH_REL}",
+                              reportFiles: 'report.html',
+                              reportName: 'PR General Check Report'
+                            ]
+                        }
+                        // Jenkins pipeline jobs fill slaves on PRs without this :(
+                        cleanup() {
+                            script {
+                                sh label: 'Cleanup workspace', script: '''
+                                    # See YETUS-764
+                                    if [ -f "${PATCHDIR}/pidfile.txt" ]; then
+                                      echo "test-patch process appears to still be running: killing"
+                                      kill `cat "${PATCHDIR}/pidfile.txt"` || true
+                                      sleep 10
+                                    fi
+                                    if [ -f "${PATCHDIR}/cidfile.txt" ]; then
+                                      echo "test-patch container appears to still be running: killing"
+                                      docker kill `cat "${PATCHDIR}/cidfile.txt"` || true
+                                    fi
+                                    # See HADOOP-13951
+                                    chmod -R u+rxw "${WORKSPACE}"
+                                '''
+                                dir ("${WORKDIR}") {
+                                    deleteDir()
+                                }
+                            }
+                        }
+                    }
+                }
+                stage ('yetus jdk8 Hadoop2 checks') {
+                    agent {
+                        node {
+                            label 'Hadoop'
+                        }
+                    }
+                    environment {
+                        // customized per parallel stage
+                        PLUGINS = "${JDK_SPECIFIC_PLUGINS}"
+                        SET_JAVA_HOME = '/usr/lib/jvm/java-8'
+                        WORKDIR_REL = "${WORKDIR_REL_JDK8_HADOOP2_CHECK}"
+                        // identical for all parallel stages
+                        WORKDIR = "${WORKSPACE}/${WORKDIR_REL}"
+                        YETUSDIR = "${WORKDIR}/${YETUS_REL}"
+                        SOURCEDIR = "${WORKDIR}/${SRC_REL}"
+                        PATCHDIR = "${WORKDIR}/${PATCH_REL}"
+                        BUILD_URL_ARTIFACTS = "artifact/${WORKDIR_REL}/${PATCH_REL}"
+                        DOCKERFILE = "${WORKDIR}/${DOCKERFILE_REL}"
+                        YETUS_DRIVER = "${WORKDIR}/${YETUS_DRIVER_REL}"
+                    }
+                    steps {
+                        dir("${SOURCEDIR}") {
+                            checkout scm
+                        }
+                        dir("${YETUSDIR}") {
+                            checkout([
+                              $class           : 'GitSCM',
+                              branches         : [[name: "${YETUS_VERSION}"]],
+                              userRemoteConfigs: [[url: 'https://github.com/apache/yetus.git']]]
+                            )
+                        }
+                        dir("${WORKDIR}") {
+                            withCredentials([
+                              usernamePassword(
+                                credentialsId: 'apache-hbase-at-github.com',
+                                passwordVariable: 'GITHUB_PASSWORD',
+                                usernameVariable: 'GITHUB_USER'
+                              )]) {
+                                sh label: 'test-patch', script: '''
+                                    hostname -a ; pwd ; ls -la
+                                    printenv 2>&1 | sort
+                                    echo "[INFO] Launching Yetus via ${YETUS_DRIVER}"
+                                    "${YETUS_DRIVER}"
+                                '''
+                            }
+                        }
+                    }
+                    post {
+                        always {
+                            junit testResults: "${WORKDIR_REL}/${SRC_REL}/**/target/**/TEST-*.xml", allowEmptyResults: true
+                            sh label: 'zip surefire reports', script: '''
+                                if [ -d "${PATCHDIR}/archiver" ]; then
+                                  count=$(find "${PATCHDIR}/archiver" -type f | wc -l)
+                                  if [[ 0 -ne ${count} ]]; then
+                                    echo "zipping ${count} archived files"
+                                    zip -q -m -r "${PATCHDIR}/test_logs.zip" "${PATCHDIR}/archiver"
+                                  else
+                                    echo "No archived files, skipping compressing."
+                                  fi
+                                else
+                                  echo "No archiver directory, skipping compressing."
+                                fi
+                            '''
+                            // Has to be relative to WORKSPACE.
+                            archiveArtifacts artifacts: "${WORKDIR_REL}/${PATCH_REL}/*", excludes: "${WORKDIR_REL}/${PATCH_REL}/precommit"
+                            archiveArtifacts artifacts: "${WORKDIR_REL}/${PATCH_REL}/**/*", excludes: "${WORKDIR_REL}/${PATCH_REL}/precommit/**/*"
+                            publishHTML target: [
+                              allowMissing: true,
+                              keepAll: true,
+                              alwaysLinkToLastBuild: true,
+                              // Has to be relative to WORKSPACE
+                              reportDir: "${WORKDIR_REL}/${PATCH_REL}",
+                              reportFiles: 'report.html',
+                              reportName: 'PR JDK8 Hadoop2 Check Report'
+                            ]
+                        }
+                        // Jenkins pipeline jobs fill slaves on PRs without this :(
+                        cleanup() {
+                            script {
+                                sh label: 'Cleanup workspace', script: '''
+                                    # See YETUS-764
+                                    if [ -f "${PATCHDIR}/pidfile.txt" ]; then
+                                      echo "test-patch process appears to still be running: killing"
+                                      kill `cat "${PATCHDIR}/pidfile.txt"` || true
+                                      sleep 10
+                                    fi
+                                    if [ -f "${PATCHDIR}/cidfile.txt" ]; then
+                                      echo "test-patch container appears to still be running: killing"
+                                      docker kill `cat "${PATCHDIR}/cidfile.txt"` || true
+                                    fi
+                                    # See HADOOP-13951
+                                    chmod -R u+rxw "${WORKSPACE}"
+                                '''
+                                dir ("${WORKDIR}") {
+                                    deleteDir()
+                                }
+                            }
+                        }
+                    }
+                }
+                stage ('yetus jdk11 hadoop3 checks') {
+                    agent {
+                        node {
+                            label 'Hadoop'
+                        }
+                    }
+                    environment {
+                        // customized per parallel stage
+                        PLUGINS = "${JDK_SPECIFIC_PLUGINS}"
+                        SET_JAVA_HOME = '/usr/lib/jvm/java-11'
+                        HADOOP_PROFILE = '3.0'
+                        WORKDIR_REL = "${WORKDIR_REL_JDK11_HADOOP3_CHECK}"
+                        // identical for all parallel stages
+                        WORKDIR = "${WORKSPACE}/${WORKDIR_REL}"
+                        YETUSDIR = "${WORKDIR}/${YETUS_REL}"
+                        SOURCEDIR = "${WORKDIR}/${SRC_REL}"
+                        PATCHDIR = "${WORKDIR}/${PATCH_REL}"
+                        BUILD_URL_ARTIFACTS = "artifact/${WORKDIR_REL}/${PATCH_REL}"
+                        DOCKERFILE = "${WORKDIR}/${DOCKERFILE_REL}"
+                        YETUS_DRIVER = "${WORKDIR}/${YETUS_DRIVER_REL}"
+                    }
+                    steps {
+                        dir("${SOURCEDIR}") {
+                            checkout scm
+                        }
+                        dir("${YETUSDIR}") {
+                            checkout([
+                              $class           : 'GitSCM',
+                              branches         : [[name: "${YETUS_VERSION}"]],
+                              userRemoteConfigs: [[url: 'https://github.com/apache/yetus.git']]]
+                            )
+                        }
+                        dir("${WORKDIR}") {
+                            withCredentials([
+                              usernamePassword(
+                                credentialsId: 'apache-hbase-at-github.com',
+                                passwordVariable: 'GITHUB_PASSWORD',
+                                usernameVariable: 'GITHUB_USER'
+                              )]) {
+                                sh label: 'test-patch', script: '''
+                                    hostname -a ; pwd ; ls -la
+                                    printenv 2>&1 | sort
+                                    echo "[INFO] Launching Yetus via ${YETUS_DRIVER}"
+                                    "${YETUS_DRIVER}"
+                                '''
+                            }
+                        }
+                    }
+                    post {
+                        always {
+                            junit testResults: "${WORKDIR_REL}/${SRC_REL}/**/target/**/TEST-*.xml", allowEmptyResults: true
+                            sh label: 'zip surefire reports', script: '''
+                                if [ -d "${PATCHDIR}/archiver" ]; then
+                                  count=$(find "${PATCHDIR}/archiver" -type f | wc -l)
+                                  if [[ 0 -ne ${count} ]]; then
+                                    echo "zipping ${count} archived files"
+                                    zip -q -m -r "${PATCHDIR}/test_logs.zip" "${PATCHDIR}/archiver"
+                                  else
+                                    echo "No archived files, skipping compressing."
+                                  fi
+                                else
+                                  echo "No archiver directory, skipping compressing."
+                                fi
+                            '''
+                            // Has to be relative to WORKSPACE.
+                            archiveArtifacts artifacts: "${WORKDIR_REL}/${PATCH_REL}/*", excludes: "${WORKDIR_REL}/${PATCH_REL}/precommit"
+                            archiveArtifacts artifacts: "${WORKDIR_REL}/${PATCH_REL}/**/*", excludes: "${WORKDIR_REL}/${PATCH_REL}/precommit/**/*"
+                            publishHTML target: [
+                              allowMissing: true,
+                              keepAll: true,
+                              alwaysLinkToLastBuild: true,
+                              // Has to be relative to WORKSPACE
+                              reportDir: "${WORKDIR_REL}/${PATCH_REL}",
+                              reportFiles: 'report.html',
+                              reportName: 'PR JDK11 Hadoop3 Check Report'
+                            ]
+                        }
+                        // Jenkins pipeline jobs fill slaves on PRs without this :(
+                        cleanup() {
+                            script {
+                                sh label: 'Cleanup workspace', script: '''
+                                    # See YETUS-764
+                                    if [ -f "${PATCHDIR}/pidfile.txt" ]; then
+                                      echo "test-patch process appears to still be running: killing"
+                                      kill `cat "${PATCHDIR}/pidfile.txt"` || true
+                                      sleep 10
+                                    fi
+                                    if [ -f "${PATCHDIR}/cidfile.txt" ]; then
+                                      echo "test-patch container appears to still be running: killing"
+                                      docker kill `cat "${PATCHDIR}/cidfile.txt"` || true
+                                    fi
+                                    # See HADOOP-13951
+                                    chmod -R u+rxw "${WORKSPACE}"
+                                '''
+                                dir ("${WORKDIR}") {
+                                    deleteDir()
+                                }
+                            }
+                        }
+                    }
                 }
             }
         }
-
     }
 
     post {
-        always {
-          script {
-            // Yetus output
-            archiveArtifacts "${env.PATCHDIR}/**"
-            // Publish the HTML report so that it can be looked at
-            // Has to be relative to WORKSPACE.
-            publishHTML (target: [
-                          allowMissing: true,
-                          keepAll: true,
-                          alwaysLinkToLastBuild: true,
-                          // Has to be relative to WORKSPACE
-                          reportDir: "${env.PATCHDIR}",
-                          reportFiles: 'report.html',
-                          reportName: 'Yetus Report'
-            ])
-            // Publish JUnit results
-            try {
-                junit "${env.SOURCEDIR}/**/target/surefire-reports/*.xml"
-            } catch(e) {
-                echo 'junit processing: ' + e.toString()
-            }
-          }
-        }
-
         // Jenkins pipeline jobs fill slaves on PRs without this :(
         cleanup() {
             script {
-                sh '''
-                    # See YETUS-764
-                    if [ -f "${WORKSPACE}/${PATCHDIR}/pidfile.txt" ]; then
-                      echo "test-patch process appears to still be running: killing"
-                      kill `cat "${WORKSPACE}/${PATCHDIR}/pidfile.txt"` || true
-                      sleep 10
-                    fi
-                    if [ -f "${WORKSPACE}/${PATCHDIR}/cidfile.txt" ]; then
-                      echo "test-patch container appears to still be running: killing"
-                      docker kill `cat "${WORKSPACE}/${PATCHDIR}/cidfile.txt"` || true
-                    fi
+                sh label: 'Cleanup workspace', script: '''
                     # See HADOOP-13951
                     chmod -R u+rxw "${WORKSPACE}"
                     '''
diff --git a/dev-support/docker/Dockerfile b/dev-support/docker/Dockerfile
index 6a3f377..6b136bc 100644
--- a/dev-support/docker/Dockerfile
+++ b/dev-support/docker/Dockerfile
@@ -14,47 +14,163 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-# Dockerfile for installing the necessary dependencies for building Hadoop.
-# See BUILDING.txt.
-
-FROM maven:3.5-jdk-8
-
-RUN apt-get -q update && apt-get -q install --no-install-recommends -y \
-       git \
-       bats \
-       findbugs \
-       libperl-critic-perl \
-       pylint \
-       python-dateutil \
-       rsync \
-       make \
-       gcc \
-       libc6-dev \
-       ruby \
-       ruby-dev \
-       wget \
-       && \
-    gem install --no-document rake rubocop ruby-lint
-
-ENV FINDBUGS_HOME /usr
-
-####
-# Install shellcheck
-###
-RUN mkdir -p /opt/shellcheck && \
-    curl -L -s -S \
-        https://storage.googleapis.com/shellcheck/shellcheck-stable.linux.x86_64.tar.xz \
-        -o /opt/shellcheck.tar.xz && \
-    tar xJf /opt/shellcheck.tar.xz --strip-components 1 -C /opt/shellcheck && \
-    ln -s /opt/shellcheck/shellcheck /usr/bin/shellcheck && \
-    rm -f /opt/shellcheck.tar.xz
+#
+# Dockerfile used as the build and test environment, amenable to Yetus.
+#
+# Built in multiple stages so as to avoid re-downloading large binaries when
+# tweaking unrelated aspects of the image.
 
-###
-# Avoid out of memory errors in builds
-###
-ENV MAVEN_OPTS -Xmx3g
+# start with a minimal image into which we can download remote tarballs
+FROM ubuntu:18.04 AS BASE_IMAGE
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+
+# hadolint ignore=DL3009
+RUN DEBIAN_FRONTEND=noninteractive apt-get -qq update && \
+  DEBIAN_FRONTEND=noninteractive apt-get -qq install --no-install-recommends -y \
+    ca-certificates=20180409 \
+    curl=7.58.0-2ubuntu3.8 \
+    locales=2.27-3ubuntu1
+
+RUN locale-gen en_US.UTF-8
+ENV LANG=en_US.UTF-8 LANGUAGE=en_US:en LC_ALL=en_US.UTF-8
+
+##
+# download sundry dependencies
+#
+
+FROM BASE_IMAGE AS FINDBUGS_DOWNLOAD_IMAGE
+# TODO: replace with Spotbugs HBASE-23077, HBASE-22383
+ENV FINDBUGS_VERSION '3.0.1'
+ENV FINDBUGS_URL "https://downloads.sourceforge.net/project/findbugs/findbugs/${FINDBUGS_VERSION}/findbugs-${FINDBUGS_VERSION}.tar.gz"
+ENV FINDBUGS_SHA256 'e80e0da0c213a27504ef3188ef25f107651700ffc66433eac6a7454bbe336419'
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+RUN curl --location --fail --silent --show-error --output /tmp/findbugs.tar.gz "${FINDBUGS_URL}" && \
+  echo "${FINDBUGS_SHA256} */tmp/findbugs.tar.gz" | sha256sum -c -
+
+FROM BASE_IMAGE AS HADOLINT_DOWNLOAD_IMAGE
+ENV HADOLINT_VERSION '1.17.5'
+ENV HADOLINT_URL "https://github.com/hadolint/hadolint/releases/download/v${HADOLINT_VERSION}/hadolint-Linux-x86_64"
+ENV HADOLINT_SHA256 '20dd38bc0602040f19268adc14c3d1aae11af27b463af43f3122076baf827a35'
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+RUN curl --location --fail --silent --show-error --output /tmp/hadolint "${HADOLINT_URL}" && \
+  echo "${HADOLINT_SHA256} */tmp/hadolint" | sha256sum -c -
+
+FROM BASE_IMAGE AS MAVEN_DOWNLOAD_IMAGE
+ENV MAVEN_VERSION='3.5.4'
+ENV MAVEN_URL "https://archive.apache.org/dist/maven/maven-3/${MAVEN_VERSION}/binaries/apache-maven-${MAVEN_VERSION}-bin.tar.gz"
+ENV MAVEN_SHA256 'ce50b1c91364cb77efe3776f756a6d92b76d9038b0a0782f7d53acf1e997a14d'
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+RUN curl --location --fail --silent --show-error --output /tmp/maven.tar.gz "${MAVEN_URL}" && \
+  echo "${MAVEN_SHA256} */tmp/maven.tar.gz" | sha256sum -c -
+
+FROM BASE_IMAGE AS OPENJDK8_DOWNLOAD_IMAGE
+ENV OPENJDK8_URL 'https://github.com/AdoptOpenJDK/openjdk8-binaries/releases/download/jdk8u232-b09/OpenJDK8U-jdk_x64_linux_hotspot_8u232b09.tar.gz'
+ENV OPENJDK8_SHA256 '7b7884f2eb2ba2d47f4c0bf3bb1a2a95b73a3a7734bd47ebf9798483a7bcc423'
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+RUN curl --location --fail --silent --show-error --output /tmp/adoptopenjdk8.tar.gz "${OPENJDK8_URL}" && \
+  echo "${OPENJDK8_SHA256} */tmp/adoptopenjdk8.tar.gz" | sha256sum -c -
+
+FROM BASE_IMAGE AS OPENJDK11_DOWNLOAD_IMAGE
+ENV OPENJDK11_URL 'https://github.com/AdoptOpenJDK/openjdk11-binaries/releases/download/jdk-11.0.6%2B10/OpenJDK11U-jdk_x64_linux_hotspot_11.0.6_10.tar.gz'
+ENV OPENJDK11_SHA256 '330d19a2eaa07ed02757d7a785a77bab49f5ee710ea03b4ee2fa220ddd0feffc'
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+RUN curl --location --fail --silent --show-error --output /tmp/adoptopenjdk11.tar.gz "${OPENJDK11_URL}" && \
+  echo "${OPENJDK11_SHA256} */tmp/adoptopenjdk11.tar.gz" | sha256sum -c -
+
+##
+# build the final image
+#
+
+FROM BASE_IMAGE
+SHELL ["/bin/bash", "-o", "pipefail", "-c"]
+
+##
+# install dependencies from system packages.
+# be careful not to install any system packages (i.e., findbugs) that will
+# pull in the default-jre.
+#
+
+# bring the base image into conformance with the expectations imposed by
+# Yetus and our personality file of what a build environment looks like.
+RUN DEBIAN_FRONTEND=noninteractive apt-get -qq install --no-install-recommends -y \
+  bash=4.4.18-2ubuntu1.2 \
+  build-essential=12.4ubuntu1 \
+  curl=7.58.0-2ubuntu3.8 \
+  diffutils=1:3.6-1 \
+  git=1:2.17.1-1ubuntu0.5 \
+  rsync=3.1.2-2.1ubuntu1 \
+  tar=1.29b-2ubuntu0.1 \
+  wget=1.19.4-1ubuntu2.2
+
+# install the dependencies required in order to enable the sundry precommit
+# checks/features provided by Yetus plugins.
+RUN DEBIAN_FRONTEND=noninteractive apt-get -qq install --no-install-recommends -y \
+  bats=0.4.0-1.1 \
+  libperl-critic-perl=1.130-1 \
+  python3=3.6.7-1~18.04 \
+  python3-pip=9.0.1-2.3~ubuntu1.18.04.1 \
+  python3-setuptools=39.0.1-2 \
+  ruby=1:2.5.1 \
+  ruby-dev=1:2.5.1 \
+  shellcheck=0.4.6-1 \
+  && \
+  apt-get clean && \
+  rm -rf /var/lib/apt/lists/*
+
+RUN python3 -mpip install --upgrade pip && \
+  python3 -mpip install pylint==2.4.4
+
+RUN gem install --no-document \
+  rake:13.0.1 \
+  rubocop:0.80.0 \
+  ruby-lint:2.3.1
+
+# hadolint ignore=DL3010
+COPY --from=FINDBUGS_DOWNLOAD_IMAGE /tmp/findbugs.tar.gz /tmp/findbugs.tar.gz
+RUN tar xzf /tmp/findbugs.tar.gz -C /opt && \
+  ln -s "/opt/$(dirname "$(tar -tf /tmp/findbugs.tar.gz | head -n1)")" /opt/findbugs && \
+  rm /tmp/findbugs.tar.gz
+
+COPY --from=HADOLINT_DOWNLOAD_IMAGE /tmp/hadolint /tmp/hadolint
+RUN mv /tmp/hadolint /usr/local/bin && \
+  chmod a+x /usr/local/bin/hadolint
+
+# hadolint ignore=DL3010
+COPY --from=MAVEN_DOWNLOAD_IMAGE /tmp/maven.tar.gz /tmp/maven.tar.gz
+RUN tar xzf /tmp/maven.tar.gz -C /opt && \
+  ln -s "/opt/$(dirname "$(tar -tf /tmp/maven.tar.gz | head -n1)")" /opt/maven && \
+  rm /tmp/maven.tar.gz
+
+##
+# ensure JVMs are available under `/usr/lib/jvm` and prefix each installation
+# as `java-` so as to conform with Yetus's assumptions.
+#
+
+# hadolint ignore=DL3010
+COPY --from=OPENJDK8_DOWNLOAD_IMAGE /tmp/adoptopenjdk8.tar.gz /tmp/adoptopenjdk8.tar.gz
+RUN mkdir -p /usr/lib/jvm && \
+  tar xzf /tmp/adoptopenjdk8.tar.gz -C /usr/lib/jvm && \
+  ln -s "/usr/lib/jvm/$(basename "$(tar -tf /tmp/adoptopenjdk8.tar.gz | head -n1)")" /usr/lib/jvm/java-8-adoptopenjdk && \
+  ln -s /usr/lib/jvm/java-8-adoptopenjdk /usr/lib/jvm/java-8 && \
+  rm /tmp/adoptopenjdk8.tar.gz
+
+# hadolint ignore=DL3010
+COPY --from=OPENJDK11_DOWNLOAD_IMAGE /tmp/adoptopenjdk11.tar.gz /tmp/adoptopenjdk11.tar.gz
+RUN mkdir -p /usr/lib/jvm && \
+  tar xzf /tmp/adoptopenjdk11.tar.gz -C /usr/lib/jvm && \
+  ln -s "/usr/lib/jvm/$(basename "$(tar -tf /tmp/adoptopenjdk11.tar.gz | head -n1)")" /usr/lib/jvm/java-11-adoptopenjdk && \
+  ln -s /usr/lib/jvm/java-11-adoptopenjdk /usr/lib/jvm/java-11 && \
+  rm /tmp/adoptopenjdk11.tar.gz
+
+# configure default environment for Yetus. Yetus in dockermode seems to require
+# these values to be specified here; the various --foo-path flags do not
+# propigate as expected, while these are honored.
+# TODO (nd): is this really true? investigate and file a ticket.
+ENV FINDBUGS_HOME '/opt/findbugs'
+ENV MAVEN_HOME '/opt/maven'
+ENV MAVEN_OPTS '-Xms6G -Xmx6G'
 
-CMD /bin/bash
+CMD ["/bin/bash"]
 
 ###
 # Everything past this point is either not needed for testing or breaks Yetus.
diff --git a/dev-support/hbase-personality.sh b/dev-support/hbase-personality.sh
index 100b847..0e1ae99 100755
--- a/dev-support/hbase-personality.sh
+++ b/dev-support/hbase-personality.sh
@@ -79,9 +79,6 @@ function personality_globals
 
   # TODO use PATCH_BRANCH to select jdk versions to use.
 
-  # Override the maven options
-  MAVEN_OPTS="${MAVEN_OPTS:-"-Xms6G -Xmx6G"}"
-
   # Yetus 0.7.0 enforces limits. Default proclimit is 1000.
   # Up it. See HBASE-19902 for how we arrived at this number.
   #shellcheck disable=SC2034
diff --git a/dev-support/jenkins_precommit_github_yetus.sh b/dev-support/jenkins_precommit_github_yetus.sh
new file mode 100755
index 0000000..f88df0a
--- /dev/null
+++ b/dev-support/jenkins_precommit_github_yetus.sh
@@ -0,0 +1,144 @@
+#!/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
+
+# place ourselves in the directory containing the hbase and yetus checkouts
+cd "$(dirname "$0")/../.."
+echo "executing from $(pwd)"
+
+if [[ "true" = "${DEBUG}" ]]; then
+  set -x
+  printenv 2>&1 | sort
+fi
+
+declare -i missing_env=0
+declare -a required_envs=(
+  # these ENV variables define the required API with Jenkinsfile_GitHub
+  "ARCHIVE_PATTERN_LIST"
+  "BUILD_URL_ARTIFACTS"
+  "DOCKERFILE"
+  "GITHUB_PASSWORD"
+  "GITHUB_USER"
+  "PATCHDIR"
+  "PLUGINS"
+  "SET_JAVA_HOME"
+  "SOURCEDIR"
+  "TESTS_FILTER"
+  "YETUSDIR"
+)
+# Validate params
+for required_env in "${required_envs[@]}"; do
+  if [ -z "${!required_env}" ]; then
+    echo "[ERROR] Required environment variable '${required_env}' is not set."
+    missing_env=${missing_env}+1
+  fi
+done
+
+if [ ${missing_env} -gt 0 ]; then
+  echo "[ERROR] Please set the required environment variables before invoking. If this error is " \
+       "on Jenkins, then please file a JIRA about the error."
+  exit 1
+fi
+
+# TODO (HBASE-23900): cannot assume test-patch runs directly from sources
+TESTPATCHBIN="${YETUSDIR}/precommit/src/main/shell/test-patch.sh"
+
+# this must be clean for every run
+rm -rf "${PATCHDIR}"
+mkdir -p "${PATCHDIR}"
+
+# Checking on H* machine nonsense
+mkdir "${PATCHDIR}/machine"
+"${SOURCEDIR}/dev-support/gather_machine_environment.sh" "${PATCHDIR}/machine"
+
+# If CHANGE_URL is set (e.g., Github Branch Source plugin), process it.
+# Otherwise exit, because we don't want HBase to do a
+# full build.  We wouldn't normally do this check for smaller
+# projects. :)
+if [[ -z "${CHANGE_URL}" ]]; then
+  echo "Full build skipped" > "${PATCHDIR}/report.html"
+  exit 0
+fi
+# enable debug output for yetus
+if [[ "true" = "${DEBUG}" ]]; then
+  YETUS_ARGS+=("--debug")
+fi
+# If we're doing docker, make sure we don't accidentally pollute the image with a host java path
+if [ -n "${JAVA_HOME}" ]; then
+  unset JAVA_HOME
+fi
+YETUS_ARGS+=("--ignore-unknown-options=true")
+YETUS_ARGS+=("--patch-dir=${PATCHDIR}")
+# where the source is located
+YETUS_ARGS+=("--basedir=${SOURCEDIR}")
+# our project defaults come from a personality file
+# which will get loaded automatically by setting the project name
+YETUS_ARGS+=("--project=hbase")
+# lots of different output formats
+YETUS_ARGS+=("--brief-report-file=${PATCHDIR}/brief.txt")
+YETUS_ARGS+=("--console-report-file=${PATCHDIR}/console.txt")
+YETUS_ARGS+=("--html-report-file=${PATCHDIR}/report.html")
+# enable writing back to Github
+YETUS_ARGS+=("--github-password=${GITHUB_PASSWORD}")
+YETUS_ARGS+=("--github-user=${GITHUB_USER}")
+# auto-kill any surefire stragglers during unit test runs
+YETUS_ARGS+=("--reapermode=kill")
+# set relatively high limits for ASF machines
+# changing these to higher values may cause problems
+# with other jobs on systemd-enabled machines
+YETUS_ARGS+=("--proclimit=10000")
+YETUS_ARGS+=("--dockermemlimit=20g")
+# -1 findbugs issues that show up prior to the patch being applied
+YETUS_ARGS+=("--findbugs-strict-precheck")
+# rsync these files back into the archive dir
+YETUS_ARGS+=("--archive-list=${ARCHIVE_PATTERN_LIST}")
+# URL for user-side presentation in reports and such to our artifacts
+YETUS_ARGS+=("--build-url-artifacts=${BUILD_URL_ARTIFACTS}")
+# plugins to enable
+YETUS_ARGS+=("--plugins=${PLUGINS}")
+# run in docker mode and specifically point to our
+# Dockerfile since we don't want to use the auto-pulled version.
+YETUS_ARGS+=("--docker")
+YETUS_ARGS+=("--dockerfile=${DOCKERFILE}")
+YETUS_ARGS+=("--mvn-custom-repos")
+YETUS_ARGS+=("--java-home=${SET_JAVA_HOME}")
+YETUS_ARGS+=("--whitespace-eol-ignore-list=.*/generated/.*")
+YETUS_ARGS+=("--whitespace-tabs-ignore-list=.*/generated/.*")
+YETUS_ARGS+=("--tests-filter=${TESTS_FILTER}")
+YETUS_ARGS+=("--personality=${SOURCEDIR}/dev-support/hbase-personality.sh")
+YETUS_ARGS+=("--quick-hadoopcheck")
+YETUS_ARGS+=("--skip-errorprone")
+# effectively treat dev-support as a custom maven module
+YETUS_ARGS+=("--skip-dirs=dev-support")
+# For testing with specific hadoop version. Activates corresponding profile in maven runs.
+if [[ -n "${HADOOP_PROFILE}" ]]; then
+  YETUS_ARGS+=("--hadoop-profile=${HADOOP_PROFILE}")
+fi
+if [[ -n "${EXCLUDE_TESTS_URL}" ]]; then
+  YETUS_ARGS+=("--exclude-tests-url=${EXCLUDE_TESTS_URL}")
+fi
+# help keep the ASF boxes clean
+YETUS_ARGS+=("--sentinel")
+# use emoji vote so it is easier to find the broken line
+YETUS_ARGS+=("--github-use-emoji-vote")
+
+echo "Launching yetus with command line:"
+echo "${TESTPATCHBIN} ${YETUS_ARGS[*]}"
+
+/usr/bin/env bash "${TESTPATCHBIN}" "${YETUS_ARGS[@]}"
diff --git a/pom.xml b/pom.xml
index fe07edf..0943da1 100755
--- a/pom.xml
+++ b/pom.xml
@@ -4158,4 +4158,5 @@
       <url>file:///tmp</url>
     </site>
   </distributionManagement>
+  <!-- precommit rebuild the project -->
 </project>