You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by st...@apache.org on 2019/03/07 00:02:58 UTC

[hbase] branch branch-2.0 updated (c6f1dc1 -> 8e2800f)

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

stack pushed a change to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/hbase.git.


    from c6f1dc1  HBASE-21999 [DEBUG] Exit if git returns empty revision! ADDENDUM2
     new 1da3021  HBASE-21935 Replace make_rc.sh with customized spark/dev/create-release
     new 8e2800f  SE-22006 Fix branch-2.1 findbugs warning; causes nightly show as failed.

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


Summary of changes:
 dev-support/create-release/do-release-docker.sh    | 164 ++++++++++
 dev-support/create-release/do-release.sh           |  77 +++++
 dev-support/create-release/hbase-rm/Dockerfile     |  48 +++
 dev-support/create-release/release-build.sh        | 340 +++++++++++++++++++++
 dev-support/create-release/release-tag.sh          | 101 ++++++
 dev-support/create-release/release-util.sh         | 276 +++++++++++++++++
 dev-support/create-release/vote.tmpl               |  33 ++
 .../hadoop/hbase/regionserver/HRegionServer.java   |   5 +-
 8 files changed, 1041 insertions(+), 3 deletions(-)
 create mode 100755 dev-support/create-release/do-release-docker.sh
 create mode 100755 dev-support/create-release/do-release.sh
 create mode 100644 dev-support/create-release/hbase-rm/Dockerfile
 create mode 100755 dev-support/create-release/release-build.sh
 create mode 100755 dev-support/create-release/release-tag.sh
 create mode 100755 dev-support/create-release/release-util.sh
 create mode 100644 dev-support/create-release/vote.tmpl


[hbase] 01/02: HBASE-21935 Replace make_rc.sh with customized spark/dev/create-release

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

stack pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 1da3021e6b731b8e7a335ad2d0f3a931aead70cc
Author: stack <st...@apache.org>
AuthorDate: Wed Feb 27 22:11:03 2019 -0800

    HBASE-21935 Replace make_rc.sh with customized spark/dev/create-release
    
    These scripts came originally from spark [1]. They were then
    modified to suit hbase context. Supercedes the old
    ../make_rc.sh script because what is here is more comprehensive
    doing more steps of the RM process as well as running in a
    container so the RM build environment can be a constant.
    
    It:
     * Tags release
     * Updates RELEASENOTES.md and CHANGES.md.
     * Sets version to the release version
     * Sets version to next SNAPSHOT version.
     * Builds, signs, and hashes all artifacts.
     * Generates the API report.
     * Pushes release tgzs to the dev dir in a apache dist.
     * Pushes to repository.apache.org staging.
     * Generates a vote email with filled-in fields.
    
    The entry point is the do-release-docker.sh script. Pass -h to
    see available options. For example, running below will do all
    steps above using the 'rm' dir under Downloads as workspace:
    
     $ ./do-release-docker.sh  -d ~/Downloads/rm
    
    1. https://github.com/apache/spark/tree/master/dev/create-release
    
    Signed-off-by: Peter Somogyi <ps...@cloudera.com>
    Signed-off-by: Duo Zhang <zh...@apache.org>
---
 dev-support/create-release/do-release-docker.sh | 164 ++++++++++++
 dev-support/create-release/do-release.sh        |  77 ++++++
 dev-support/create-release/hbase-rm/Dockerfile  |  48 ++++
 dev-support/create-release/release-build.sh     | 340 ++++++++++++++++++++++++
 dev-support/create-release/release-tag.sh       | 101 +++++++
 dev-support/create-release/release-util.sh      | 276 +++++++++++++++++++
 dev-support/create-release/vote.tmpl            |  33 +++
 7 files changed, 1039 insertions(+)

diff --git a/dev-support/create-release/do-release-docker.sh b/dev-support/create-release/do-release-docker.sh
new file mode 100755
index 0000000..fcadf96
--- /dev/null
+++ b/dev-support/create-release/do-release-docker.sh
@@ -0,0 +1,164 @@
+#!/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.
+#
+
+#
+# Creates a HBase release candidate. The script will update versions, tag the branch,
+# build HBase binary packages and documentation, and upload maven artifacts to a staging
+# repository. There is also a dry run mode where only local builds are performed, and
+# nothing is uploaded to the ASF repos.
+#
+# Run with "-h" for options. For example, running below will do all
+# steps above using the 'rm' dir under Downloads as workspace:
+#
+# $ ./do-release-docker.sh  -d ~/Downloads/rm
+#
+# The scripts in this directory came originally from spark [1]. They were then
+# modified to suite the hbase context. These scripts supercedes the old
+# ../make_rc.sh script for making release candidates because what is here is more
+# comprehensive doing more steps of the RM process as well as running in a
+# container so the RM build environment can be a constant.
+#
+# It:
+#  * Tags release
+#  * Sets version to the release version
+#  * Sets version to next SNAPSHOT version.
+#  * Builds, signs, and hashes all artifacts.
+#  * Pushes release tgzs to the dev dir in a apache dist.
+#  * Pushes to repository.apache.org staging.
+#
+# The entry point is here, in the do-release-docker.sh script.
+#
+# 1. https://github.com/apache/spark/tree/master/dev/create-release
+#
+set -e
+SELF=$(cd $(dirname $0) && pwd)
+. "$SELF/release-util.sh"
+
+function usage {
+  local NAME=$(basename $0)
+  cat <<EOF
+Usage: $NAME [options]
+
+This script runs the release scripts inside a docker image. The image is hardcoded to be called
+"hbase-rm" and will be re-generated (as needed) on every invocation of this script.
+
+Options are:
+
+  -d [path]   : required: working directory (output will be written to an "output" directory in
+                the working directory).
+  -n          : dry run mode. Performs checks and local builds, but does not upload anything.
+  -t [tag]    : tag for the hbase-rm docker image to use for building (default: "latest").
+  -j [path]   : path to local JDK installation to use for building. By default the script will
+                use openjdk8 installed in the docker image.
+  -s [step]   : runs a single step of the process; valid steps are: tag, build, publish. if
+                none specified, runs tag, then build, and then publish.
+EOF
+}
+
+WORKDIR=
+IMGTAG=latest
+JAVA=
+RELEASE_STEP=
+while getopts "d:hj:ns:t:" opt; do
+  case $opt in
+    d) WORKDIR="$OPTARG" ;;
+    n) DRY_RUN=1 ;;
+    t) IMGTAG="$OPTARG" ;;
+    j) JAVA="$OPTARG" ;;
+    s) RELEASE_STEP="$OPTARG" ;;
+    h) usage ;;
+    ?) error "Invalid option. Run with -h for help." ;;
+  esac
+done
+
+if [ -z "$WORKDIR" ] || [ ! -d "$WORKDIR" ]; then
+  error "Work directory (-d) must be defined and exist. Run with -h for help."
+fi
+
+if [ -d "$WORKDIR/output" ]; then
+  read -p "Output directory already exists. Overwrite and continue? [y/n] " ANSWER
+  if [ "$ANSWER" != "y" ]; then
+    error "Exiting."
+  fi
+fi
+
+cd "$WORKDIR"
+rm -rf "$WORKDIR/output"
+mkdir "$WORKDIR/output"
+
+get_release_info
+
+# Place all RM scripts and necessary data in a local directory that must be defined in the command
+# line. This directory is mounted into the image. Its WORKDIR, the arg passed with -d.
+for f in "$SELF"/*; do
+  if [ -f "$f" ]; then
+    cp "$f" "$WORKDIR"
+  fi
+done
+
+GPG_KEY_FILE="$WORKDIR/gpg.key"
+fcreate_secure "$GPG_KEY_FILE"
+$GPG --export-secret-key --armor "$GPG_KEY" > "$GPG_KEY_FILE"
+
+run_silent "Building hbase-rm image with tag $IMGTAG..." "docker-build.log" \
+  docker build -t "hbase-rm:$IMGTAG" --build-arg UID=$UID "$SELF/hbase-rm"
+
+# Write the release information to a file with environment variables to be used when running the
+# image.
+ENVFILE="$WORKDIR/env.list"
+fcreate_secure "$ENVFILE"
+
+function cleanup {
+  rm -f "$ENVFILE"
+  rm -f "$GPG_KEY_FILE"
+}
+
+trap cleanup EXIT
+
+cat > $ENVFILE <<EOF
+DRY_RUN=$DRY_RUN
+SKIP_TAG=$SKIP_TAG
+RUNNING_IN_DOCKER=1
+GIT_BRANCH=$GIT_BRANCH
+NEXT_VERSION=$NEXT_VERSION
+RELEASE_VERSION=$RELEASE_VERSION
+RELEASE_TAG=$RELEASE_TAG
+GIT_REF=$GIT_REF
+HBASE_PACKAGE_VERSION=$HBASE_PACKAGE_VERSION
+ASF_USERNAME=$ASF_USERNAME
+GIT_NAME=$GIT_NAME
+GIT_EMAIL=$GIT_EMAIL
+GPG_KEY=$GPG_KEY
+ASF_PASSWORD=$ASF_PASSWORD
+GPG_PASSPHRASE=$GPG_PASSPHRASE
+RELEASE_STEP=$RELEASE_STEP
+EOF
+
+JAVA_VOL=
+if [ -n "$JAVA" ]; then
+  echo "JAVA_HOME=/opt/hbase-java" >> $ENVFILE
+  JAVA_VOL="--volume $JAVA:/opt/hbase-java"
+fi
+
+echo "Building $RELEASE_TAG; output will be at $WORKDIR/output"
+docker run -ti \
+  --env-file "$ENVFILE" \
+  --volume "$WORKDIR:/opt/hbase-rm" \
+  $JAVA_VOL \
+  "hbase-rm:$IMGTAG"
diff --git a/dev-support/create-release/do-release.sh b/dev-support/create-release/do-release.sh
new file mode 100755
index 0000000..dffb4fb
--- /dev/null
+++ b/dev-support/create-release/do-release.sh
@@ -0,0 +1,77 @@
+#!/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.
+#
+
+# Called by do-release-docker.sh. Can be run standalone but needs some love
+# for it to work smooth.
+SELF=$(cd $(dirname $0) && pwd)
+. "$SELF/release-util.sh"
+
+while getopts "bn" opt; do
+  case $opt in
+    b) GIT_BRANCH=$OPTARG ;;
+    n) DRY_RUN=1 ;;
+    ?) error "Invalid option: $OPTARG" ;;
+  esac
+done
+
+if [ "$RUNNING_IN_DOCKER" = "1" ]; then
+  # Inside docker, need to import the GPG key stored in the current directory.
+  echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --import "$SELF/gpg.key"
+
+  # We may need to adjust the path since JAVA_HOME may be overridden by the driver script.
+  if [ -n "$JAVA_HOME" ]; then
+    export PATH="$JAVA_HOME/bin:$PATH"
+  else
+    # JAVA_HOME for the openjdk package.
+    export JAVA_HOME=/usr
+  fi
+else
+  # Outside docker, need to ask for information about the release.
+  get_release_info
+fi
+export GPG_TTY=$(tty)
+
+function should_build {
+  local WHAT=$1
+  [ -z "$RELEASE_STEP" ] || [ "$WHAT" = "$RELEASE_STEP" ]
+}
+
+if should_build "tag" && [ $SKIP_TAG = 0 ]; then
+  run_silent "Creating release tag $RELEASE_TAG..." "tag.log" \
+    "$SELF/release-tag.sh"
+  echo "It may take some time for the tag to be synchronized to github."
+  echo "Press enter when you've verified that the new tag ($RELEASE_TAG) is available."
+  read
+else
+  echo "Skipping tag creation for $RELEASE_TAG."
+fi
+
+if should_build "build"; then
+  run_silent "Building HBase..." "build.log" \
+    "$SELF/release-build.sh" build
+else
+  echo "Skipping build step."
+fi
+
+if should_build "publish"; then
+  run_silent "Publishing release" "publish.log" \
+    "$SELF/release-build.sh" publish-release
+else
+  echo "Skipping publish step."
+fi
diff --git a/dev-support/create-release/hbase-rm/Dockerfile b/dev-support/create-release/hbase-rm/Dockerfile
new file mode 100644
index 0000000..ffa83b7
--- /dev/null
+++ b/dev-support/create-release/hbase-rm/Dockerfile
@@ -0,0 +1,48 @@
+#
+# 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.
+#
+
+# Image for building HBase releases. Based on Ubuntu 16.04.
+#
+# Includes:
+# * Java 8
+
+FROM ubuntu:16.04
+
+# These arguments are just for reuse and not really meant to be customized.
+ARG APT_INSTALL="apt-get install --no-install-recommends -y"
+
+# Install extra needed repos and refresh.
+#
+# This is all in a single "RUN" command so that if anything changes, "apt update" is run to fetch
+# the most current package versions (instead of potentially using old versions cached by docker).
+RUN apt-get clean && \
+  apt-get update && \
+  # Install openjdk 8.
+  $APT_INSTALL openjdk-8-jdk && \
+  update-alternatives --set java /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java && \
+  # Install build / source control tools
+  $APT_INSTALL curl python-pip wget git maven subversion lsof \
+    libcurl4-openssl-dev libxml2-dev && \
+  pip install python-dateutil
+
+WORKDIR /opt/hbase-rm/output
+
+ARG UID
+RUN useradd -m -s /bin/bash -p hbase-rm -u $UID hbase-rm
+USER hbase-rm:hbase-rm
+
+ENTRYPOINT [ "/opt/hbase-rm/do-release.sh" ]
diff --git a/dev-support/create-release/release-build.sh b/dev-support/create-release/release-build.sh
new file mode 100755
index 0000000..88bb2a5
--- /dev/null
+++ b/dev-support/create-release/release-build.sh
@@ -0,0 +1,340 @@
+#!/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
+
+trap cleanup EXIT
+
+SELF=$(cd $(dirname $0) && pwd)
+. "$SELF/release-util.sh"
+
+function exit_with_usage {
+  cat << EOF
+usage: release-build.sh <package|docs|publish-snapshot|publish-release>
+Creates build deliverables from an HBase commit.
+
+Top level targets are
+  build: Create binary packages and commit them to dist.apache.org/repos/dist/dev/hbase/
+  publish-snapshot: Publish snapshot release to Apache snapshots
+  publish-release: Publish a release to Apache release repo
+
+All other inputs are environment variables
+
+GIT_REF - Release tag or commit to build from
+HBASE_PACKAGE_VERSION - Release identifier in top level package directory (e.g. 2.1.2RC1)
+HBASE_VERSION - (optional) Version of HBase being built (e.g. 2.1.2)
+
+ASF_USERNAME - Username of ASF committer account
+ASF_PASSWORD - Password of ASF committer account
+
+GPG_KEY - GPG key used to sign release artifacts
+GPG_PASSPHRASE - Passphrase for GPG key
+EOF
+  exit 1
+}
+
+set -e
+
+function cleanup {
+  rm ${tmp_settings} &> /dev/null
+}
+
+if [ $# -eq 0 ]; then
+  exit_with_usage
+fi
+
+if [[ $@ == *"help"* ]]; then
+  exit_with_usage
+fi
+
+if [[ -z "$ASF_PASSWORD" ]]; then
+  echo 'The environment variable ASF_PASSWORD is not set. Enter the password.'
+  echo
+  stty -echo && printf "ASF password: " && read ASF_PASSWORD && printf '\n' && stty echo
+fi
+
+if [[ -z "$GPG_PASSPHRASE" ]]; then
+  echo 'The environment variable GPG_PASSPHRASE is not set. Enter the passphrase to'
+  echo 'unlock the GPG signing key that will be used to sign the release!'
+  echo
+  stty -echo && printf "GPG passphrase: " && read GPG_PASSPHRASE && printf '\n' && stty echo
+  export GPG_PASSPHRASE
+fi
+export GPG_TTY=$(tty)
+
+for env in ASF_USERNAME GPG_PASSPHRASE GPG_KEY; do
+  if [ -z "${!env}" ]; then
+    echo "ERROR: $env must be set to run this script"
+    exit_with_usage
+  fi
+done
+
+export LC_ALL=C.UTF-8
+export LANG=C.UTF-8
+
+# Commit ref to checkout when building
+GIT_REF=${GIT_REF:-master}
+
+RELEASE_STAGING_LOCATION="https://dist.apache.org/repos/dist/dev/hbase"
+
+GPG="gpg -u $GPG_KEY --no-tty --batch"
+NEXUS_ROOT=https://repository.apache.org/service/local/staging
+NEXUS_PROFILE=8e226b97c0c82 # Profile for HBase staging uploads via INFRA-17900 Need nexus "staging profile id" for the hbase project
+BASE_DIR=$(pwd)
+
+init_java
+init_mvn
+init_python
+
+rm -rf hbase
+git clone "$ASF_REPO"
+cd hbase
+git checkout $GIT_REF
+git_hash=`git rev-parse --short HEAD`
+echo "Checked out HBase git hash $git_hash"
+
+if [ -z "$HBASE_VERSION" ]; then
+  # Run $MVN in a separate command so that 'set -e' does the right thing.
+  TMP=$(mktemp)
+  $MVN help:evaluate -Dexpression=project.version > $TMP
+  HBASE_VERSION=$(cat $TMP | grep -v INFO | grep -v WARNING | grep -v Download)
+  rm $TMP
+fi
+
+# Depending on the version being built, certain extra profiles need to be activated.
+BASE_PROFILES="-Papache-release -Prelease"
+
+# Profiles for publishing snapshots and release to Maven Central
+PUBLISH_PROFILES="$BASE_PROFILES"
+
+if [[ ! $HBASE_VERSION < "2.0." ]]; then
+  if [[ $JAVA_VERSION < "1.8." ]]; then
+    echo "Java version $JAVA_VERSION is less than required 1.8 for 2.0+"
+    echo "Please set JAVA_HOME correctly."
+    exit 1
+  fi
+else
+  if ! [[ $JAVA_VERSION =~ 1\.7\..* ]]; then
+    if [ -z "$JAVA_7_HOME" ]; then
+      echo "Java version $JAVA_VERSION is higher than required 1.7 for pre-2.0"
+      echo "Please set JAVA_HOME correctly."
+      exit 1
+    else
+      export JAVA_HOME="$JAVA_7_HOME"
+    fi
+  fi
+fi
+
+# This is a band-aid fix to avoid the failure of Maven nightly snapshot in some Jenkins
+# machines by explicitly calling /usr/sbin/lsof. Please see SPARK-22377 and the discussion
+# in its pull request.
+LSOF=lsof
+if ! hash $LSOF 2>/dev/null; then
+  LSOF=/usr/sbin/lsof
+fi
+
+if [ -z "$HBASE_PACKAGE_VERSION" ]; then
+  HBASE_PACKAGE_VERSION="${HBASE_VERSION}-$(date +%Y_%m_%d_%H_%M)-${git_hash}"
+fi
+
+DEST_DIR_NAME="$HBASE_PACKAGE_VERSION"
+
+git clean -d -f -x
+cd ..
+tmp_repo=`pwd`/$(mktemp -d hbase-repo-XXXXX)
+# Reexamine. Not sure this working. Pass as arg? That don't seem to work either!
+tmp_settings="/tmp/tmp-settings.xml"
+echo "<settings><servers>" > $tmp_settings
+echo "<server><id>apache.snapshots.https</id><username>$ASF_USERNAME</username>" >> $tmp_settings
+echo "<password>$ASF_PASSWORD</password></server>" >> $tmp_settings
+echo "<server><id>apache-release</id><username>$ASF_USERNAME</username>" >> $tmp_settings
+echo "<password>$ASF_PASSWORD</password></server>" >> $tmp_settings
+echo "</servers>" >> $tmp_settings
+echo "<profiles><profile><id>apache-release</id><properties>" >> $tmp_settings
+echo "<gpg.keyname>$GPG_KEY</gpg.keyname>" >> $tmp_settings
+echo "<gpg.passphrase>$GPG_PASSPHRASE</gpg.passphrase>" >> $tmp_settings
+echo "</properties></profile></profiles>" >> $tmp_settings
+echo "</settings>" >> $tmp_settings
+export tmp_settings
+
+if [[ "$1" == "build" ]]; then
+  # Source and binary tarballs
+  echo "Packaging release source tarballs"
+  # Tar up the src and sign and hash it.
+  rm -rf hbase-$HBASE_VERSION-src*
+  ls
+  cd hbase
+  git archive --format=tar.gz --output=../hbase-$HBASE_VERSION-src.tar.gz --prefix=hbase-"${HBASE_VERSION}/" "${GIT_REF}"
+  cd ..
+  echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour --output hbase-$HBASE_VERSION-src.tar.gz.asc \
+    --detach-sig hbase-$HBASE_VERSION-src.tar.gz
+  echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
+    SHA512 hbase-$HBASE_VERSION-src.tar.gz > hbase-$HBASE_VERSION-src.tar.gz.sha512
+
+  # Updated for each binary build; spark did many. HBase does one bin only.
+  make_binary_release() {
+    FLAGS="$MVN_EXTRA_OPTS -B $BASE_RELEASE_PROFILES $2"
+
+    echo "`date -u +'%Y-%m-%dT%H:%M:%SZ'` Building binary dist"
+    cp -R hbase hbase-$HBASE_VERSION-bin
+    cd hbase-$HBASE_VERSION-bin
+
+    echo "`date -u +'%Y-%m-%dT%H:%M:%SZ'` Creating distribution: ($FLAGS)"
+
+    # Get maven home set by MVN
+    MVN_HOME=`$MVN -version 2>&1 | grep 'Maven home' | awk '{print $NF}'`
+
+    MAVEN_OPTS="${MAVEN_OPTS}" ${MVN} clean install -DskipTests \
+        -Dmaven.repo.local=${tmp_repo}
+    echo "`date -u +'%Y-%m-%dT%H:%M:%SZ'` Building all (site, etc., with release profiile)"
+    MAVEN_OPTS="${MAVEN_OPTS}" ${MVN} -Darguments="-Dgpg.passphrase=$GPG_PASSPHRASE" --settings $tmp_settings install -DskipTests \
+        -Dcheckstyle.skip=true site assembly:single \
+        -Papache-release -Prelease \
+        -Dmaven.repo.local=${tmp_repo}
+
+    echo "`date -u +'%Y-%m-%dT%H:%M:%SZ'` Copying and signing regular binary distribution"
+    cp ./hbase-assembly/target/hbase-$HBASE_VERSION-bin.tar.gz ..
+    cd ..
+    echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour \
+      --output hbase-$HBASE_VERSION-bin.tar.gz.asc \
+      --detach-sig hbase-$HBASE_VERSION-bin.tar.gz
+    echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
+      SHA512 hbase-$HBASE_VERSION-bin.tar.gz > \
+      hbase-$HBASE_VERSION-bin.tar.gz.sha512
+  }
+
+  rm -rf hbase-$HBASE_VERSION-bin*/
+  make_binary_release
+
+  if ! is_dry_run; then
+    svn co --depth=empty $RELEASE_STAGING_LOCATION svn-hbase
+    rm -rf "svn-hbase/${DEST_DIR_NAME}"
+    mkdir -p "svn-hbase/${DEST_DIR_NAME}"
+
+    echo "Copying release tarballs"
+    cp hbase-*.tar.* "svn-hbase/${DEST_DIR_NAME}/"
+    cp hbase-$HBASE_VERSION-bin/CHANGES.md "svn-hbase/${DEST_DIR_NAME}/"
+    cp hbase-$HBASE_VERSION-bin/RELEASENOTES.md "svn-hbase/${DEST_DIR_NAME}/"
+    generate_api_report ./hbase ${PREV_REL_TAG} ${RELEASE_TAG}
+    cp api*.html "svn-hbase/${DEST_DIR_NAME}/"
+
+    svn add "svn-hbase/${DEST_DIR_NAME}"
+
+    cd svn-hbase
+    svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache HBase $HBASE_PACKAGE_VERSION" --no-auth-cache
+    cd ..
+    rm -rf svn-hbase
+  fi
+
+  exit 0
+fi
+
+if [[ "$1" == "publish-snapshot" ]]; then
+  cd hbase
+  # Publish HBase to Maven release repo
+  echo "Deploying HBase SNAPSHOT at '$GIT_REF' ($git_hash)"
+  echo "Publish version is $HBASE_VERSION"
+  if [[ ! $HBASE_VERSION == *"SNAPSHOT"* ]]; then
+    echo "ERROR: Snapshots must have a version containing SNAPSHOT"
+    echo "ERROR: You gave version '$HBASE_VERSION'"
+    exit 1
+  fi
+  # Coerce the requested version
+  $MVN versions:set -DnewVersion=$HBASE_VERSION
+  $MVN --settings $tmp_settings -DskipTests $PUBLISH_PROFILES deploy
+  cd ..
+  exit 0
+fi
+
+if [[ "$1" == "publish-release" ]]; then
+  cd hbase
+  # Publish HBase to Maven release repo
+  echo "Publishing HBase checkout at '$GIT_REF' ($git_hash)"
+  echo "Publish version is $HBASE_VERSION"
+  # Coerce the requested version
+  $MVN versions:set -DnewVersion=$HBASE_VERSION
+  MAVEN_OPTS="${MAVEN_OPTS}" $MVN --settings $tmp_settings -Dmaven.repo.local=$tmp_repo \
+      -DskipTests $PUBLISH_PROFILES clean install
+  pushd $tmp_repo/org/apache/hbase
+  # Remove any extra files generated during install
+  # Do find in hbase* because thirdparty is at same level!
+  find hbase* -type f | grep -v \.jar | grep -v \.pom | xargs rm
+
+  # Using Nexus API documented here:
+  # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API
+  if ! is_dry_run; then
+    echo "Creating Nexus staging repository"
+    repo_request="<promoteRequest><data><description>Apache HBase $HBASE_VERSION (commit $git_hash)</description></data></promoteRequest>"
+    out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
+      -H "Content-Type:application/xml" -v \
+      $NEXUS_ROOT/profiles/$NEXUS_PROFILE/start)
+    staged_repo_id=$(echo $out | sed -e "s/.*\(orgapachehbase-[0-9]\{4\}\).*/\1/")
+    echo "Created Nexus staging repository: $staged_repo_id"
+  fi
+
+  # this must have .asc, and .sha1 - it really doesn't like anything else there
+  for file in $(find hbase* -type f)
+  do
+    if [[ "$file" == *.asc ]]; then
+      continue
+    fi
+    if [ ! -f $file.asc ]; then
+      echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --output $file.asc \
+        --detach-sig --armour $file;
+    fi
+    if [ $(command -v md5)  ]; then
+      # Available on OS X; -q to keep only hash
+      md5 -q $file > $file.md5
+    else
+      # Available on Linux; cut to keep only hash
+      md5sum $file | cut -f1 -d' ' > $file.md5
+    fi
+    sha1sum $file | cut -f1 -d' ' > $file.sha1
+  done
+
+  if ! is_dry_run; then
+    nexus_upload=$NEXUS_ROOT/deployByRepositoryId/$staged_repo_id
+    echo "Uplading files to $nexus_upload"
+    for file in $(find hbase* -type f)
+    do
+      # strip leading ./
+      file_short=$(echo $file | sed -e "s/\.\///")
+      dest_url="$nexus_upload/org/apache/hbase/$file_short"
+      echo "  Uploading $file to $dest_url"
+      curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url
+    done
+
+    echo "Closing nexus staging repository"
+    repo_request="<promoteRequest><data><stagedRepositoryId>$staged_repo_id</stagedRepositoryId><description>Apache HBase $HBASE_VERSION (commit $git_hash)</description></data></promoteRequest>"
+    out=$(curl -X POST -d "$repo_request" -u $ASF_USERNAME:$ASF_PASSWORD \
+      -H "Content-Type:application/xml" -v \
+      $NEXUS_ROOT/profiles/$NEXUS_PROFILE/finish)
+    echo "Closed Nexus staging repository: $staged_repo_id"
+  fi
+
+  popd
+  rm -rf $tmp_repo
+  cd ..
+  # Dump out email to send.
+  eval "echo \"$(<vote.tmpl)\"" |tee vote.txt
+  exit 0
+fi
+
+cd ..
+rm -rf hbase
+echo "ERROR: expects to be called with 'install', 'publish-release' or 'publish-snapshot'"
diff --git a/dev-support/create-release/release-tag.sh b/dev-support/create-release/release-tag.sh
new file mode 100755
index 0000000..82d9d6a
--- /dev/null
+++ b/dev-support/create-release/release-tag.sh
@@ -0,0 +1,101 @@
+#!/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.
+#
+
+# Tags release. Updates releasenotes and changes.
+SELF=$(cd $(dirname $0) && pwd)
+. "$SELF/release-util.sh"
+
+function exit_with_usage {
+  local NAME=$(basename $0)
+  cat << EOF
+usage: $NAME
+Tags an HBase release on a particular branch.
+
+Inputs are specified with the following environment variables:
+ASF_USERNAME - Apache Username
+ASF_PASSWORD - Apache Password
+GIT_NAME - Name to use with git
+GIT_EMAIL - E-mail address to use with git
+GIT_BRANCH - Git branch on which to make release
+RELEASE_VERSION - Version used in pom files for release
+RELEASE_TAG - Name of release tag
+NEXT_VERSION - Development version after release
+EOF
+  exit 1
+}
+
+set -e
+set -o pipefail
+
+if [[ $@ == *"help"* ]]; then
+  exit_with_usage
+fi
+
+if [[ -z "$ASF_PASSWORD" ]]; then
+  echo 'The environment variable ASF_PASSWORD is not set. Enter the password.'
+  echo
+  stty -echo && printf "ASF password: " && read ASF_PASSWORD && printf '\n' && stty echo
+fi
+
+for env in ASF_USERNAME ASF_PASSWORD RELEASE_VERSION RELEASE_TAG NEXT_VERSION GIT_EMAIL GIT_NAME GIT_BRANCH; do
+  if [ -z "${!env}" ]; then
+    echo "$env must be set to run this script"
+    exit 1
+  fi
+done
+
+init_java
+init_mvn
+
+ASF_HBASE_REPO="gitbox.apache.org/repos/asf/hbase.git"
+
+rm -rf hbase
+git clone "https://$ASF_USERNAME:$ASF_PASSWORD@$ASF_HBASE_REPO" -b $GIT_BRANCH
+update_releasenotes `pwd`/hbase $RELEASE_VERSION
+
+cd hbase
+
+git config user.name "$GIT_NAME"
+git config user.email $GIT_EMAIL
+
+# Create release version
+$MVN versions:set -DnewVersion=$RELEASE_VERSION | grep -v "no value" # silence logs
+git add RELEASENOTES.md CHANGES.md
+
+git commit -a -m "Preparing HBase release $RELEASE_TAG; tagging and updates to CHANGES.md and RELEASENOTES.md"
+echo "Creating tag $RELEASE_TAG at the head of $GIT_BRANCH"
+git tag $RELEASE_TAG
+
+# Create next version
+$MVN versions:set -DnewVersion=$NEXT_VERSION | grep -v "no value" # silence logs
+
+git commit -a -m "Preparing development version $NEXT_VERSION"
+
+if ! is_dry_run; then
+  # Push changes
+  git push origin $RELEASE_TAG
+  git push origin HEAD:$GIT_BRANCH
+  cd ..
+  rm -rf hbase
+else
+  cd ..
+  mv hbase hbase.tag
+  echo "Clone with version changes and tag available as hbase.tag in the output directory."
+fi
+
diff --git a/dev-support/create-release/release-util.sh b/dev-support/create-release/release-util.sh
new file mode 100755
index 0000000..0035768
--- /dev/null
+++ b/dev-support/create-release/release-util.sh
@@ -0,0 +1,276 @@
+#!/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.
+#
+DRY_RUN=${DRY_RUN:-0}
+GPG="gpg --no-tty --batch"
+ASF_REPO="https://gitbox.apache.org/repos/asf/hbase.git"
+
+ASF_REPO_WEBUI="https://gitbox.apache.org/repos/asf?p=hbase.git"
+ASF_GITHUB_REPO="https://github.com/apache/hbase"
+YETUS_VERSION=0.9.0
+
+function error {
+  echo "$*"
+  exit 1
+}
+
+function read_config {
+  local PROMPT="$1"
+  local DEFAULT="$2"
+  local REPLY=
+
+  read -p "$PROMPT [$DEFAULT]: " REPLY
+  local RETVAL="${REPLY:-$DEFAULT}"
+  if [ -z "$RETVAL" ]; then
+    error "$PROMPT is must be provided."
+  fi
+  echo "$RETVAL"
+}
+
+function parse_version {
+  grep -e '<version>.*</version>' | \
+    head -n 2 | tail -n 1 | cut -d'>' -f2 | cut -d '<' -f1
+}
+
+function run_silent {
+  local BANNER="$1"
+  local LOG_FILE="$2"
+  shift 2
+
+  echo "========================"
+  echo "= $BANNER"
+  echo "Command: $@"
+  echo "Log file: $LOG_FILE"
+
+  "$@" 1>"$LOG_FILE" 2>&1
+
+  local EC=$?
+  if [ $EC != 0 ]; then
+    echo "Command FAILED. Check full logs for details."
+    tail "$LOG_FILE"
+    exit $EC
+  fi
+}
+
+function fcreate_secure {
+  local FPATH="$1"
+  rm -f "$FPATH"
+  touch "$FPATH"
+  chmod 600 "$FPATH"
+}
+
+function check_for_tag {
+  curl -s --head --fail "$ASF_GITHUB_REPO/releases/tag/$1" > /dev/null
+}
+
+# Get all branches that begin with 'branch-', the hbase convention for
+# release branches, sort them and then pop off the most recent.
+function get_release_info {
+  if [ -z "$GIT_BRANCH" ]; then
+    # If no branch is specified, find out the latest branch from the repo.
+    GIT_BRANCH=$(git ls-remote --heads "$ASF_REPO" |
+      grep refs/heads/branch- |
+      awk '{print $2}' |
+      sort -r |
+      head -n 1 |
+      cut -d/ -f3)
+  fi
+
+  export GIT_BRANCH=$(read_config "Branch" "$GIT_BRANCH")
+
+  # Find the current version for the branch.
+  local VERSION=$(curl -s "$ASF_REPO_WEBUI;a=blob_plain;f=pom.xml;hb=refs/heads/$GIT_BRANCH" |
+    parse_version)
+  echo "Current branch version is $VERSION."
+
+  if [[ ! $VERSION =~ .*-SNAPSHOT ]]; then
+    error "Not a SNAPSHOT version: $VERSION"
+  fi
+
+  NEXT_VERSION="$VERSION"
+  RELEASE_VERSION="${VERSION/-SNAPSHOT/}"
+  SHORT_VERSION=$(echo "$VERSION" | cut -d . -f 1-2)
+  local REV=$(echo "$VERSION" | cut -d . -f 3)
+
+  # Find out what RC is being prepared.
+  # - If the current version is "x.y.0", then this is RC0 of the "x.y.0" release.
+  # - If not, need to check whether the previous version has been already released or not.
+  #   - If it has, then we're building RC0 of the current version.
+  #   - If it has not, we're building the next RC of the previous version.
+  local RC_COUNT
+  if [ $REV != 0 ]; then
+    local PREV_REL_REV=$((REV - 1))
+    # Export this finding.
+    export PREV_REL_TAG="rel/${SHORT_VERSION}.${PREV_REL_REV}"
+    if check_for_tag "$PREV_REL_TAG"; then
+      RC_COUNT=0
+      REV=$((REV + 1))
+      NEXT_VERSION="${SHORT_VERSION}.${REV}-SNAPSHOT"
+    else
+      RELEASE_VERSION="${SHORT_VERSION}.${PREV_REL_REV}"
+      RC_COUNT=$(git ls-remote --tags "$ASF_REPO" "${RELEASE_VERSION}RC*" | wc -l)
+      # This makes a 'number' of it.
+      RC_COUNT=$((RC_COUNT))
+    fi
+  else
+    REV=$((REV + 1))
+    NEXT_VERSION="${SHORT_VERSION}.${REV}-SNAPSHOT"
+    RC_COUNT=0
+  fi
+
+  export NEXT_VERSION
+  export RELEASE_VERSION=$(read_config "Release" "$RELEASE_VERSION")
+
+  RC_COUNT=$(read_config "RC #" "$RC_COUNT")
+
+  # Check if the RC already exists, and if re-creating the RC, skip tag creation.
+  RELEASE_TAG="${RELEASE_VERSION}RC${RC_COUNT}"
+  SKIP_TAG=0
+  if check_for_tag "$RELEASE_TAG"; then
+    read -p "$RELEASE_TAG already exists. Continue anyway [y/n]? " ANSWER
+    if [ "$ANSWER" != "y" ]; then
+      error "Exiting."
+    fi
+    SKIP_TAG=1
+  fi
+
+
+  export RELEASE_TAG
+
+  GIT_REF="$RELEASE_TAG"
+  if is_dry_run; then
+    echo "This is a dry run. Please confirm the ref that will be built for testing."
+    GIT_REF=$(read_config "Ref" "$GIT_REF")
+  fi
+  export GIT_REF
+  export HBASE_PACKAGE_VERSION="$RELEASE_TAG"
+
+  # Gather some user information.
+  export ASF_USERNAME=$(read_config "ASF user" "$LOGNAME")
+
+  GIT_NAME=$(git config user.name || echo "")
+  export GIT_NAME=$(read_config "Full name" "$GIT_NAME")
+
+  export GIT_EMAIL="$ASF_USERNAME@apache.org"
+  export GPG_KEY=$(read_config "GPG key" "$GIT_EMAIL")
+
+  cat <<EOF
+================
+Release details:
+BRANCH:       $GIT_BRANCH
+VERSION:      $RELEASE_VERSION
+TAG:          $RELEASE_TAG
+NEXT:         $NEXT_VERSION
+PREVIOUS TAG: $PREV_REL_TAG
+
+ASF USER:     $ASF_USERNAME
+GPG KEY:      $GPG_KEY
+FULL NAME:    $GIT_NAME
+E-MAIL:       $GIT_EMAIL
+================
+EOF
+
+  read -p "Is this info correct [y/n]? " ANSWER
+  if [ "$ANSWER" != "y" ]; then
+    echo "Exiting."
+    exit 1
+  fi
+
+  if ! is_dry_run; then
+    if [ -z "$ASF_PASSWORD" ]; then
+      stty -echo && printf "ASF password: " && read ASF_PASSWORD && printf '\n' && stty echo
+    fi
+  else
+    ASF_PASSWORD="***INVALID***"
+  fi
+
+  if [ -z "$GPG_PASSPHRASE" ]; then
+    stty -echo && printf "GPG passphrase: " && read GPG_PASSPHRASE && printf '\n' && stty echo
+    export GPG_TTY=$(tty)
+  fi
+
+  export ASF_PASSWORD
+  export GPG_PASSPHRASE
+}
+
+function is_dry_run {
+  [[ $DRY_RUN = 1 ]]
+}
+
+# Initializes JAVA_VERSION to the version of the JVM in use.
+function init_java {
+  if [ -z "$JAVA_HOME" ]; then
+    error "JAVA_HOME is not set."
+  fi
+  JAVA_VERSION=$("${JAVA_HOME}"/bin/javac -version 2>&1 | cut -d " " -f 2)
+  export JAVA_VERSION
+}
+
+function init_python {
+  if ! [ -x "$(command -v python2)"  ]; then
+    echo 'Error: python2 needed by yetus. Install or add link? E.g: sudo ln -sf /usr/bin/python2.7 /usr/local/bin/python2' >&2
+    exit 1
+  fi
+}
+
+# Set MVN
+function init_mvn {
+  if [ -n "$MAVEN_HOME" ]; then
+      MVN=${MAVEN_HOME}/bin/mvn
+  elif [ $(type -P mvn) ]; then
+      MVN=mvn
+  else
+    error "MAVEN_HOME is not set nor is mvn on the current path."
+  fi
+  export MVN
+}
+
+# Writes report into cwd!
+function generate_api_report {
+  local hbase=$1
+  local previous_release_tag=$2
+  local release_tag=$3
+  # Generate api report.
+  ${hbase}/dev-support/checkcompatibility.py --annotation \
+    org.apache.yetus.audience.InterfaceAudience.Public  \
+    $previous_release_tag $release_tag
+  local previous_version=$(echo ${previous_release_tag} | sed -e 's/rel\///')
+  cp ${hbase}/target/compat-check/report.html "./api_compare_${release_tag}_to_${previous_version}.html"
+}
+
+# Update the CHANGES.md
+# DOES NOT DO COMMITS! Caller should do that.
+# yetus requires python2 to be on the path.
+function update_releasenotes {
+  local hbase=$1
+  local release_version=$2
+  local yetus="apache-yetus-${YETUS_VERSION}"
+  wget -qO- "https://www.apache.org/dyn/mirrors/mirrors.cgi?action=download&filename=/yetus/${YETUS_VERSION}/${yetus}-bin.tar.gz" | tar xvz -C .
+  cd ./${yetus}
+  ./bin/releasedocmaker -p HBASE --fileversions -v ${release_version} -l --sortorder=newer --skip-credits
+  # The above generates RELEASENOTES.X.X.X.md and CHANGELOG.X.X.X.md.
+  # To insert into hbase CHANGES.me...need to cut the top off the
+  # CHANGELOG.X.X.X.md file removing license and first line and then
+  # insert it after the license comment closing; i.e. '-->'.
+  sed -i -e '/## Release/,$!d' CHANGELOG.${release_version}.md
+  sed -i -e "/->/r CHANGELOG.${release_version}.md" ${hbase}/CHANGES.md
+  # Similar for RELEASENOTES but slightly different.
+  sed -i -e '/Release Notes/,$!d' RELEASENOTES.${release_version}.md
+  sed -i -e "/->/r RELEASENOTES.${release_version}.md" ${hbase}/RELEASENOTES.md
+  cd ..
+}
diff --git a/dev-support/create-release/vote.tmpl b/dev-support/create-release/vote.tmpl
new file mode 100644
index 0000000..dfa8176
--- /dev/null
+++ b/dev-support/create-release/vote.tmpl
@@ -0,0 +1,33 @@
+Please vote on this release candidate (RC) for Apache HBase ${RELEASE_VERSION}.
+
+The VOTE will remain open for at least 72 hours.
+
+[ ] +1 Release this package as Apache HBase ${RELEASE_VERSION}
+[ ] -1 Do not release this package because ...
+
+The tag to be voted on is ${RELEASE_TAG}:
+
+ https://github.com/apache/hbase/tree/${RELEASE_TAG}
+
+The release files, including signatures, digests, etc. can be found at:
+
+ https://dist.apache.org/repos/dist/dev/hbase/${RELEASE_TAG}/
+
+Maven artifacts are available in a staging repository at:
+
+ https://repository.apache.org/content/repositories/orgapachehbase-${staged_repo_id}/
+
+Artifacts were signed with the ${GPG_KEY} key which can be found in:
+
+ https://dist.apache.org/repos/dist/release/hbase/KEYS
+
+The list of bug fixes going into {version} can be found in included
+CHANGES.md and RELEASENOTES.md available here:
+
+ https://dist.apache.org/repos/dist/dev/hbase/${RELEASE_VERSION}/CHANGES.md
+ https://dist.apache.org/repos/dist/dev/hbase/${RELEASE_VERSION}/RELEASENOTES.md
+
+To learn more about Apache HBase, please see http://hbase.apache.org/
+
+Thanks,
+Your HBase Release Manager


[hbase] 02/02: SE-22006 Fix branch-2.1 findbugs warning; causes nightly show as failed.

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

stack pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/hbase.git

commit 8e2800f50401c37dcb921533cff62b40efd7e8d6
Author: stack <st...@apache.org>
AuthorDate: Wed Mar 6 15:59:06 2019 -0800

    SE-22006 Fix branch-2.1 findbugs warning; causes nightly show as failed.
---
 .../java/org/apache/hadoop/hbase/regionserver/HRegionServer.java     | 5 ++---
 1 file changed, 2 insertions(+), 3 deletions(-)

diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
index 5e44a19..b413b5b 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java
@@ -1569,9 +1569,8 @@ public class HRegionServer extends HasThread implements
           MemStoreLAB.POOL_INITIAL_SIZE_DEFAULT);
       int chunkSize = conf.getInt(MemStoreLAB.CHUNK_SIZE_KEY, MemStoreLAB.CHUNK_SIZE_DEFAULT);
       // init the chunkCreator
-      ChunkCreator chunkCreator =
-          ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
-      initialCountPercentage, this.hMemManager);
+      ChunkCreator.initialize(chunkSize, offheap, globalMemStoreSize, poolSizePercentage,
+        initialCountPercentage, this.hMemManager);
     }
   }