From 44454b69ff87a19dd6aa20e22e569e3d0811109b Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 23 May 2018 13:15:49 -0700 Subject: [PATCH 01/11] [SPARK-24372][build] Add scripts to help with preparing releases. The "do-release.sh" script asks questions about the RC being prepared, trying to find out as much as possible automatically, and then executes the existing scripts with proper arguments to prepare the release. This script was used to prepare the 2.3.1 release candidates, so was tested in that context. The docker version runs that same script inside a docker image especially crafted for building Spark releases. That image is based on the work by Felix C. linked in the bug. At this point is has been only midly tested. I also added a template for the vote e-mail, with placeholders for things that need to be replaced, although there is no automation around that for the moment. It shouldn't be hard to hook up certain things like version and tags to this, or to figure out certain things like the repo URL from the output of the release scripts. --- dev/create-release/do-release-docker.sh | 100 ++++++++++++ dev/create-release/do-release.sh | 53 +++++++ dev/create-release/release-util.sh | 195 ++++++++++++++++++++++++ dev/create-release/spark-rm/Dockerfile | 89 +++++++++++ dev/create-release/vote.tmpl | 64 ++++++++ 5 files changed, 501 insertions(+) create mode 100755 dev/create-release/do-release-docker.sh create mode 100755 dev/create-release/do-release.sh create mode 100644 dev/create-release/release-util.sh create mode 100644 dev/create-release/spark-rm/Dockerfile create mode 100644 dev/create-release/vote.tmpl diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh new file mode 100755 index 0000000000000..c5bff21e44f98 --- /dev/null +++ b/dev/create-release/do-release-docker.sh @@ -0,0 +1,100 @@ +#!/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 +SELF=$(cd $(dirname $0) && pwd) +. "$SELF/release-util.sh" + +WORKDIR= +IMGTAG=latest +while getopts "d:n:t:" opt; do + case $opt in + d) WORKDIR="$OPTARG" ;; + n) DRY_RUN=1 ;; + t) IMGTAG="$OPTARG" ;; + ?) error "Invalid option: $OPTARG" ;; + esac +done + +if [ -z "$WORKDIR" ] || [ ! -d "$WORKDIR" ]; then + error "Work directory (-d) must be defined and exist." +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. +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 spark-rm image with tag $IMGTAG..." "docker-build.log" \ + docker build -t "spark-rm:$IMGTAG" --build-arg UID=$UID "$SELF/spark-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 <.*' | \ + head -n 2 | tail -n 1 | cut -d'>' -f2 | cut -d '<' -f1 +} + +function run_silent { + local BANNER="$1" + shift + local LOG_FILE="$1" + shift + + echo "==================================" + echo "= $BANNER" + echo "Command: $@" + echo "Log file: $LOG_FILE" + + if [ $DRY_RUN = 0 ]; then + "$@" 1>"$LOG_FILE" 2>&1 + else + echo "$@" + touch "$LOG_FILE" + fi + + local EC=$? + if [ $EC != 0 ]; then + echo "Command FAILED. Check full logs for details." + tail "$LOG_FILE" + return $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_REPO_WEBUI;a=commit;h=$1" >/dev/null +} + +function get_release_info { + if [ -z "$GIT_BRANCH" ]; then + # If no branch is specified, found out the latest branch from the repo. + GIT_BRANCH=$(git ls-remote --heads "$ASF_REPO" | + grep -v refs/heads/master | + 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 rc1 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 rc1 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)) + local PREV_REL_TAG="v${SHORT_VERSION}.${PREV_REL_REV}" + if check_for_tag "$PREV_REL_TAG"; then + RC_COUNT=1 + 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" "v${RELEASE_VERSION}-rc*" | wc -l) + RC_COUNT=$((RC_COUNT + 1)) + fi + else + REV=$((REV + 1)) + NEXT_VERSION="${SHORT_VERSION}-${REV}-SNAPSHOT" + RC_COUNT=1 + 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="v${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 + export GIT_REF="$RELEASE_TAG" + export SPARK_PACKAGE_VERSION="$RELEASE_TAG" + + # Gather some user information. + export ASF_USERNAME=$(read_config "ASF user" "$LOGNAME") + + GIT_NAME=$(git config user.name) + 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 <> /etc/apt/sources.list && \ + gpg --keyserver keyserver.ubuntu.com --recv-key E084DAB9 && \ + gpg -a --export E084DAB9 | apt-key add - && \ + apt-get clean && \ + rm -rf /var/lib/apt/lists/* && \ + apt-get clean && \ + apt-get update && \ + $APT_INSTALL software-properties-common && \ + apt-add-repository -y ppa:brightbox/ruby-ng && \ + apt-get update + +# Install openjdk 8. +RUN $APT_INSTALL openjdk-8-jdk && \ + update-alternatives --set java /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java + +# Install build / source control tools +RUN $APT_INSTALL curl wget git maven ivy subversion make gcc libffi-dev \ + pandoc pandoc-citeproc libssl-dev libcurl4-openssl-dev libxml2-dev && \ + ln -s -T /usr/share/java/ivy.jar /usr/share/ant/lib/ivy.jar && \ + curl -sL https://deb.nodesource.com/setup_4.x | bash && \ + $APT_INSTALL nodejs + +# Install needed python packages. Use pip for installing packages (for consistency). +ARG BASE_PIP_PKGS="setuptools wheel virtualenv" +ARG PIP_PKGS="pyopenssl pypandoc numpy pygments sphinx" + +RUN $APT_INSTALL libpython2.7-dev libpython3-dev python-pip python3-pip && \ + pip install $BASE_PIP_PKGS && \ + pip install $PIP_PKGS && \ + cd && \ + virtualenv -p python3 p35 && \ + . p35/bin/activate && \ + pip install $BASE_PIP_PKGS && \ + pip install $PIP_PKGS + +# Install R packages and dependencies used when building. +# R depends on pandoc*, libssl (which are installed above). +RUN $APT_INSTALL r-base r-base-dev && \ + $APT_INSTALL texlive-latex-base texlive texlive-fonts-extra texinfo qpdf && \ + Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='http://cran.us.r-project.org/')" && \ + Rscript -e "devtools::install_github('jimhester/lintr')" + +# Install tools needed to build the documentation. +RUN $APT_INSTALL ruby2.3 ruby2.3-dev && \ + gem install jekyll --no-rdoc --no-ri && \ + gem install jekyll-redirect-from && \ + gem install pygments.rb + +WORKDIR /opt/spark-rm/output + +ARG UID +RUN useradd -m -s /bin/bash -p spark-rm -u $UID spark-rm +USER spark-rm:spark-rm + +ENTRYPOINT [ "/opt/spark-rm/do-release.sh", "-n" ] diff --git a/dev/create-release/vote.tmpl b/dev/create-release/vote.tmpl new file mode 100644 index 0000000000000..44e8893c14351 --- /dev/null +++ b/dev/create-release/vote.tmpl @@ -0,0 +1,64 @@ +Please vote on releasing the following candidate as Apache Spark version {version}. + +The vote is open until {deadline} and passes if a majority of at least 3 +1 PMC votes are cast. + +[ ] +1 Release this package as Apache Spark {version} +[ ] -1 Do not release this package because ... + +To learn more about Apache Spark, please see http://spark.apache.org/ + +The tag to be voted on is {tag} (commit {tag_commit}): +https://github.com/apache/spark/tree/{tag} + +The release files, including signatures, digests, etc. can be found at: +https://dist.apache.org/repos/dist/dev/spark/{tag}-bin/ + +Signatures used for Spark RCs can be found in this file: +https://dist.apache.org/repos/dist/dev/spark/KEYS + +The staging repository for this release can be found at: +https://repository.apache.org/content/repositories/orgapachespark-{repo_id}/ + +The documentation corresponding to this release can be found at: +https://dist.apache.org/repos/dist/dev/spark/{tag}-docs/ + +The list of bug fixes going into {version} can be found at the following URL: +https://issues.apache.org/jira/projects/SPARK/versions/{jira_version_id} + +FAQ + +========================= +How can I help test this release? +========================= + +If you are a Spark user, you can help us test this release by taking +an existing Spark workload and running on this release candidate, then +reporting any regressions. + +If you're working in PySpark you can set up a virtual env and install +the current RC and see if anything important breaks, in the Java/Scala +you can add the staging repository to your projects resolvers and test +with the RC (make sure to clean up the artifact cache before/after so +you don't end up building with a out of date RC going forward). + +=========================================== +What should happen to JIRA tickets still targeting {version}? +=========================================== + +The current list of open tickets targeted at {version} can be found at: +{open_issues_link} + +Committers should look at those and triage. Extremely important bug +fixes, documentation, and API tweaks that impact compatibility should +be worked on immediately. Everything else please retarget to an +appropriate release. + +================== +But my bug isn't fixed? +================== + +In order to make timely releases, we will typically not hold the +release unless the bug in question is a regression from the previous +release. That being said, if there is something which is a regression +that has not been correctly targeted please ping me or a committer to +help target the issue. \ No newline at end of file From 1366f236aafec5014fb794f59e1f23252dd52621 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 11 Jun 2018 13:13:46 -0700 Subject: [PATCH 02/11] Add RAT exclude. --- dev/.rat-excludes | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/.rat-excludes b/dev/.rat-excludes index 9552d001a079c..23b24212b4d29 100644 --- a/dev/.rat-excludes +++ b/dev/.rat-excludes @@ -106,3 +106,4 @@ spark-warehouse structured-streaming/* kafka-source-initial-offset-version-2.1.0.bin kafka-source-initial-offset-future-version.bin +vote.tmpl From 04f6371a3aa0f03ba1c37b5b450bea69923388e9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 12 Jun 2018 14:39:00 -0700 Subject: [PATCH 03/11] Update vote count wording. --- dev/create-release/vote.tmpl | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/create-release/vote.tmpl b/dev/create-release/vote.tmpl index 44e8893c14351..2ce953c2f7ec4 100644 --- a/dev/create-release/vote.tmpl +++ b/dev/create-release/vote.tmpl @@ -1,6 +1,7 @@ Please vote on releasing the following candidate as Apache Spark version {version}. -The vote is open until {deadline} and passes if a majority of at least 3 +1 PMC votes are cast. +The vote is open until {deadline} and passes if a majority +1 PMC votes are cast, with +a minimum of 3 +1 votes. [ ] +1 Release this package as Apache Spark {version} [ ] -1 Do not release this package because ... From 48eda71029bb414f404312fbdecf7a35d883660c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 14 Jun 2018 14:08:57 -0700 Subject: [PATCH 04/11] Changes mostly targeted at using the script for 2.1 builds. - Allow mounting a local JDK (2.1 wants 1.7). - Tweak profiles and builds to 2.1 - Restore support for Scala 2.10 - set up JAVA_HOME in the docker container I also expanded "dry run" mode a bit to build one of the binary packages, and also the documentation. The build documentation is now also kept in the output directory along with the other artifacts. --- dev/create-release/do-release-docker.sh | 15 +++- dev/create-release/do-release.sh | 16 ++-- dev/create-release/release-build.sh | 108 +++++++++++++++--------- dev/create-release/release-util.sh | 56 ++++++++---- 4 files changed, 133 insertions(+), 62 deletions(-) diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index c5bff21e44f98..1f62339f964c2 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -23,11 +23,13 @@ SELF=$(cd $(dirname $0) && pwd) WORKDIR= IMGTAG=latest -while getopts "d:n:t:" opt; do +JAVA= +while getopts "d:j:nt:" opt; do case $opt in d) WORKDIR="$OPTARG" ;; n) DRY_RUN=1 ;; t) IMGTAG="$OPTARG" ;; + j) JAVA="$OPTARG" ;; ?) error "Invalid option: $OPTARG" ;; esac done @@ -93,8 +95,15 @@ ASF_PASSWORD=$ASF_PASSWORD GPG_PASSPHRASE=$GPG_PASSPHRASE EOF +JAVA_VOL= +if [ -n "$JAVA" ]; then + echo "JAVA_HOME=/opt/spark-java" >> $ENVFILE + JAVA_VOL="--volume $JAVA:/opt/spark-java" +fi + echo "Building $RELEASE_TAG; output will be at $WORKDIR/output" -docker run \ +docker run -ti \ --env-file "$ENVFILE" \ - --volume "$WORKDIR":/opt/spark-rm \ + --volume "$WORKDIR:/opt/spark-rm" \ + $JAVA_VOL \ "spark-rm:$IMGTAG" diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh index 85ff4cc6282e5..12c78da195ff2 100755 --- a/dev/create-release/do-release.sh +++ b/dev/create-release/do-release.sh @@ -28,18 +28,24 @@ while getopts "bn" opt; do esac done -set -e - 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 "$SELF/gpg.key" + 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 if [ $SKIP_TAG = 0 ]; then - run_silent "Creating release tag $RELEASE_TAG..." "tag.log" \ + maybe_run "Creating release tag $RELEASE_TAG..." "tag.log" \ "$SELF/release-tag.sh" else echo "Skipping tag creation for $RELEASE_TAG." @@ -49,5 +55,5 @@ run_silent "Building Spark..." "build.log" \ "$SELF/release-build.sh" package run_silent "Building documentation..." "docs.log" \ "$SELF/release-build.sh" docs -run_silent "Publishing release" "publish.log" \ +maybe_run "Publishing release" "publish.log" \ "$SELF/release-build.sh" publish-release diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 5faa3d3260a56..e6a1e733ba6f7 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -17,6 +17,9 @@ # limitations under the License. # +SELF=$(cd $(dirname $0) && pwd) +. "$SELF/release-util.sh" + function exit_with_usage { cat << EOF usage: release-build.sh @@ -89,17 +92,6 @@ BASE_DIR=$(pwd) MVN="build/mvn --force" -# Hive-specific profiles for some builds -HIVE_PROFILES="-Phive -Phive-thriftserver" -# Profiles for publishing snapshots and release to Maven Central -PUBLISH_PROFILES="-Pmesos -Pyarn -Pkubernetes -Pflume $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl" -# Profiles for building binary releases -BASE_RELEASE_PROFILES="-Pmesos -Pyarn -Pkubernetes -Pflume -Psparkr" -# Scala 2.11 only profiles for some builds -SCALA_2_11_PROFILES="-Pkafka-0-8" -# Scala 2.12 only profiles for some builds -SCALA_2_12_PROFILES="-Pscala-2.12" - rm -rf spark git clone https://git-wip-us.apache.org/repos/asf/spark.git cd spark @@ -112,6 +104,28 @@ if [ -z "$SPARK_VERSION" ]; then | grep -v INFO | grep -v WARNING | grep -v Download) fi +# Depending on the version being built, certain extra profiles need to be activated, and +# different versions of Scala are supported. +BASE_PROFILES="-Pmesos -Pyarn" +PUBLISH_SCALA_2_10=0 +SCALA_2_10_PROFILES="-Pscala-2.10" +SCALA_2_11_PROFILES= +SCALA_2_12_PROFILES="-Pscala-2.12" + +if [[ $SPARK_VERSION > "2.3" ]]; then + BASE_PROFILES="$BASE_PROFILES -Pkubernetes -Pflume" + SCALA_2_11_PROFILES="-Pkafka-0-8" +else + PUBLISH_SCALA_2_10=1 +fi + +# Hive-specific profiles for some builds +HIVE_PROFILES="-Phive -Phive-thriftserver" +# Profiles for publishing snapshots and release to Maven Central +PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-asl" +# Profiles for building binary releases +BASE_RELEASE_PROFILES="$BASE_PROFILES -Psparkr" + # Verify we have the right java version set if [ -z "$JAVA_HOME" ]; then echo "Please set JAVA_HOME." @@ -127,7 +141,7 @@ if [[ ! $SPARK_VERSION < "2.2." ]]; then exit 1 fi else - if [[ $java_version > "1.7." ]]; then + 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.2" echo "Please set JAVA_HOME correctly." @@ -174,8 +188,9 @@ if [[ "$1" == "package" ]]; then FLAGS=$2 ZINC_PORT=$3 BUILD_PACKAGE=$4 - cp -r spark spark-$SPARK_VERSION-bin-$NAME + echo "Building binary dist $NAME" + cp -r spark spark-$SPARK_VERSION-bin-$NAME cd spark-$SPARK_VERSION-bin-$NAME # TODO There should probably be a flag to make-distribution to allow 2.12 support @@ -250,25 +265,32 @@ if [[ "$1" == "package" ]]; then # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. make_binary_release "hadoop2.6" "-Phadoop-2.6 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3035" "withr" & - make_binary_release "hadoop2.7" "-Phadoop-2.7 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3036" "withpip" & - make_binary_release "without-hadoop" "-Phadoop-provided $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3038" & + + if ! is_dry_run; then + make_binary_release "hadoop2.7" "-Phadoop-2.7 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3036" "withpip" & + make_binary_release "without-hadoop" "-Phadoop-provided $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3038" & + fi + wait rm -rf spark-$SPARK_VERSION-bin-*/ - svn co --depth=empty $RELEASE_STAGING_LOCATION svn-spark - rm -rf "svn-spark/${DEST_DIR_NAME}-bin" - mkdir -p "svn-spark/${DEST_DIR_NAME}-bin" - - echo "Copying release tarballs" - cp spark-* "svn-spark/${DEST_DIR_NAME}-bin/" - cp pyspark-* "svn-spark/${DEST_DIR_NAME}-bin/" - cp SparkR_* "svn-spark/${DEST_DIR_NAME}-bin/" - svn add "svn-spark/${DEST_DIR_NAME}-bin" + if ! is_dry_run; then + svn co --depth=empty $RELEASE_STAGING_LOCATION svn-spark + rm -rf "svn-spark/${DEST_DIR_NAME}-bin" + mkdir -p "svn-spark/${DEST_DIR_NAME}-bin" + + echo "Copying release tarballs" + cp spark-* "svn-spark/${DEST_DIR_NAME}-bin/" + cp pyspark-* "svn-spark/${DEST_DIR_NAME}-bin/" + cp SparkR_* "svn-spark/${DEST_DIR_NAME}-bin/" + svn add "svn-spark/${DEST_DIR_NAME}-bin" + + cd svn-spark + svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION" + cd .. + rm -rf svn-spark + fi - cd svn-spark - svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION" - cd .. - rm -rf svn-spark exit 0 fi @@ -282,18 +304,22 @@ if [[ "$1" == "docs" ]]; then cd .. cd .. - svn co --depth=empty $RELEASE_STAGING_LOCATION svn-spark - rm -rf "svn-spark/${DEST_DIR_NAME}-docs" - mkdir -p "svn-spark/${DEST_DIR_NAME}-docs" + if ! is_dry_run; then + svn co --depth=empty $RELEASE_STAGING_LOCATION svn-spark + rm -rf "svn-spark/${DEST_DIR_NAME}-docs" + mkdir -p "svn-spark/${DEST_DIR_NAME}-docs" - echo "Copying release documentation" - cp -R "spark/docs/_site" "svn-spark/${DEST_DIR_NAME}-docs/" - svn add "svn-spark/${DEST_DIR_NAME}-docs" + echo "Copying release documentation" + cp -R "spark/docs/_site" "svn-spark/${DEST_DIR_NAME}-docs/" + svn add "svn-spark/${DEST_DIR_NAME}-docs" - cd svn-spark - svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION docs" - cd .. - rm -rf svn-spark + cd svn-spark + svn ci --username $ASF_USERNAME --password "$ASF_PASSWORD" -m"Apache Spark $SPARK_PACKAGE_VERSION docs" + cd .. + rm -rf svn-spark + fi + + mv "spark/docs/_site" docs/ exit 0 fi @@ -356,6 +382,12 @@ if [[ "$1" == "publish-release" ]]; then $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $SCALA_2_11_PROFILES $PUBLISH_PROFILES clean install + if [[ $PUBLISH_SCALA_2_10 = 1 ]]; then + ./dev/change-scala-version.sh 2.10 + $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -Dscala-2.10 \ + -DskipTests $PUBLISH_PROFILES $SCALA_2_10_PROFILES clean install + fi + #./dev/change-scala-version.sh 2.12 #$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo \ # -DskipTests $SCALA_2_12_PROFILES ยง$PUBLISH_PROFILES clean install diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index dbb3886f8a807..bba2278d26355 100644 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -47,27 +47,35 @@ function parse_version { function run_silent { local BANNER="$1" - shift - local LOG_FILE="$1" - shift + local LOG_FILE="$2" + shift 2 - echo "==================================" + echo "========================" echo "= $BANNER" echo "Command: $@" echo "Log file: $LOG_FILE" - if [ $DRY_RUN = 0 ]; then - "$@" 1>"$LOG_FILE" 2>&1 - else - echo "$@" - touch "$LOG_FILE" - fi + "$@" 1>"$LOG_FILE" 2>&1 local EC=$? if [ $EC != 0 ]; then echo "Command FAILED. Check full logs for details." tail "$LOG_FILE" - return $EC + exit $EC + fi +} + +function maybe_run { + if is_dry_run; then + local BANNER="$1" + local LOG_FILE="$2" + shift 2 + echo "======= DRY RUN =======" + echo "= $BANNER" + echo "Command: $@" + echo "Log file: $LOG_FILE" + else + run_silent "$@" fi } @@ -149,8 +157,15 @@ function get_release_info { SKIP_TAG=1 fi + export RELEASE_TAG - export GIT_REF="$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 SPARK_PACKAGE_VERSION="$RELEASE_TAG" # Gather some user information. @@ -183,13 +198,22 @@ EOF exit 1 fi - if [ -z "$ASF_PASSWORD" ]; then - stty -echo && printf "ASF password: " && read ASF_PASSWORD && printf '\n' && stty echo - export ASF_PASSWORD + 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_PASSPHRASE fi + + export ASF_PASSWORD + export GPG_PASSPHRASE +} + +function is_dry_run { + [[ $DRY_RUN = 1 ]] } From 7fe133afac729e8ebc2bba864ead03345ea0ed7c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Jun 2018 10:13:40 -0700 Subject: [PATCH 05/11] Small fixes. --- dev/create-release/do-release-docker.sh | 35 ++++++++++++++++++++++--- dev/create-release/release-util.sh | 4 +-- 2 files changed, 34 insertions(+), 5 deletions(-) diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index 1f62339f964c2..2a6d7026eb328 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -17,25 +17,54 @@ # limitations under the License. # +# +# Creates a Spark release candidate. The script will update versions, tag the branch, +# build Spark 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. +# + set -e SELF=$(cd $(dirname $0) && pwd) . "$SELF/release-util.sh" +function usage { + local NAME=$(basename $0) + cat < Date: Mon, 18 Jun 2018 13:22:23 -0700 Subject: [PATCH 06/11] Another version number fix. --- dev/create-release/release-util.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index af6de49decd20..29295cc6927cd 100644 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -129,7 +129,7 @@ function get_release_info { if check_for_tag "$PREV_REL_TAG"; then RC_COUNT=1 REV=$((REV + 1)) - NEXT_VERSION="${SHORT_VERSION}-${REV}-SNAPSHOT" + NEXT_VERSION="${SHORT_VERSION}.${REV}-SNAPSHOT" else RELEASE_VERSION="${SHORT_VERSION}.${PREV_REL_REV}" RC_COUNT=$(git ls-remote --tags "$ASF_REPO" "v${RELEASE_VERSION}-rc*" | wc -l) From 82ae00e61dfbe40e75dbcd3d224fcd5c3f5f74a7 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Jun 2018 13:39:43 -0700 Subject: [PATCH 07/11] Un-hardcode dry run mode with docker. --- dev/create-release/do-release.sh | 3 +++ dev/create-release/spark-rm/Dockerfile | 2 +- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh index 12c78da195ff2..33716eb4a41ea 100755 --- a/dev/create-release/do-release.sh +++ b/dev/create-release/do-release.sh @@ -47,6 +47,9 @@ fi if [ $SKIP_TAG = 0 ]; then maybe_run "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 diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 1cd923b43ea9a..ef308461830fc 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -86,4 +86,4 @@ ARG UID RUN useradd -m -s /bin/bash -p spark-rm -u $UID spark-rm USER spark-rm:spark-rm -ENTRYPOINT [ "/opt/spark-rm/do-release.sh", "-n" ] +ENTRYPOINT [ "/opt/spark-rm/do-release.sh" ] From 57713541abd926d5697da7ef90f843eaaf9ce67f Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Jun 2018 15:28:17 -0700 Subject: [PATCH 08/11] Some updates for Java 7. Seems like Maven Central updated something over the weekend, and Java 7 was having trouble connecting; needed some extra SSL-related options. I also made the build sequential, which makes it slower, but avoids some issues with parallel maven processes that I was hitting when building 2.1. --- dev/create-release/do-release-docker.sh | 1 + dev/create-release/release-build.sh | 38 ++++++++++++++++--------- 2 files changed, 25 insertions(+), 14 deletions(-) diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index 2a6d7026eb328..1c536768aa20d 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -87,6 +87,7 @@ for f in "$SELF"/*; do 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" diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index e6a1e733ba6f7..515a042b02f36 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -90,7 +90,15 @@ NEXUS_ROOT=https://repository.apache.org/service/local/staging NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads BASE_DIR=$(pwd) -MVN="build/mvn --force" +MVN="build/mvn -B" + +java_version=$("${JAVA_HOME}"/bin/javac -version 2>&1 | cut -d " " -f 2) +MVN_EXTRA_OPTS= +if [[ $java_version < "1.8." ]]; then + # Needed for maven central when using Java 7. + MVN_EXTRA_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" + MVN="$MVN $MVN_EXTRA_OPTS" +fi rm -rf spark git clone https://git-wip-us.apache.org/repos/asf/spark.git @@ -100,8 +108,11 @@ git_hash=`git rev-parse --short HEAD` echo "Checked out Spark git hash $git_hash" if [ -z "$SPARK_VERSION" ]; then - SPARK_VERSION=$($MVN help:evaluate -Dexpression=project.version \ - | grep -v INFO | grep -v WARNING | grep -v Download) + # Run $MVN in a separate command so that 'set -e' does the right thing. + TMP=$(mktemp) + $MVN help:evaluate -Dexpression=project.version > $TMP + SPARK_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, and @@ -132,8 +143,6 @@ if [ -z "$JAVA_HOME" ]; then exit 1 fi -java_version=$("${JAVA_HOME}"/bin/javac -version 2>&1 | cut -d " " -f 2) - if [[ ! $SPARK_VERSION < "2.2." ]]; then if [[ $java_version < "1.8." ]]; then echo "Java version $java_version is less than required 1.8 for 2.2+" @@ -259,19 +268,20 @@ if [[ "$1" == "package" ]]; then spark-$SPARK_VERSION-bin-$NAME.tgz.sha512 } - # TODO: Check exit codes of children here: - # http://stackoverflow.com/questions/1570262/shell-get-exit-code-of-background-process - # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. - make_binary_release "hadoop2.6" "-Phadoop-2.6 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3035" "withr" & - + if ! make_binary_release "hadoop2.6" "$MVN_EXTRA_OPTS -B -Phadoop-2.6 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3035" "withr"; then + error "Failed to build hadoop2.6 package. Check logs for details." + fi if ! is_dry_run; then - make_binary_release "hadoop2.7" "-Phadoop-2.7 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3036" "withpip" & - make_binary_release "without-hadoop" "-Phadoop-provided $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3038" & + if ! make_binary_release "hadoop2.7" "$MVN_EXTRA_OPTS -B -Phadoop-2.7 $HIVE_PROFILES $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3036" "withpip"; then + error "Failed to build hadoop2.7 package. Check logs for details." + fi + if ! make_binary_release "without-hadoop" "$MVN_EXTRA_OPTS -B -Phadoop-provided $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES" "3037"; then + error "Failed to build without-hadoop package. Check logs for details." + fi fi - wait rm -rf spark-$SPARK_VERSION-bin-*/ if ! is_dry_run; then @@ -384,7 +394,7 @@ if [[ "$1" == "publish-release" ]]; then if [[ $PUBLISH_SCALA_2_10 = 1 ]]; then ./dev/change-scala-version.sh 2.10 - $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -Dscala-2.10 \ + $MVN -DzincPort=$((ZINC_PORT + 1)) -Dmaven.repo.local=$tmp_repo -Dscala-2.10 \ -DskipTests $PUBLISH_PROFILES $SCALA_2_10_PROFILES clean install fi From a39933eef41370c07cf4ac70665943dfe0ae00a9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Jun 2018 18:44:42 -0700 Subject: [PATCH 09/11] More fixes. - Override SBT_OPTS for Java 7 - Allow executins individual steps in docker - Fix docker image so that changing what to install also forces an "apt update" to happen. --- dev/create-release/do-release-docker.sh | 6 +++- dev/create-release/do-release.sh | 33 ++++++++++++++---- dev/create-release/release-build.sh | 1 + dev/create-release/spark-rm/Dockerfile | 46 ++++++++++++------------- 4 files changed, 54 insertions(+), 32 deletions(-) diff --git a/dev/create-release/do-release-docker.sh b/dev/create-release/do-release-docker.sh index 1c536768aa20d..fa7b73cdb40ec 100755 --- a/dev/create-release/do-release-docker.sh +++ b/dev/create-release/do-release-docker.sh @@ -46,18 +46,21 @@ Options are: -t [tag] : tag for the spark-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, docs, publish EOF } WORKDIR= IMGTAG=latest JAVA= -while getopts "d:hj:nt:" opt; do +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 @@ -123,6 +126,7 @@ GIT_EMAIL=$GIT_EMAIL GPG_KEY=$GPG_KEY ASF_PASSWORD=$ASF_PASSWORD GPG_PASSPHRASE=$GPG_PASSPHRASE +RELEASE_STEP=$RELEASE_STEP EOF JAVA_VOL= diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh index 33716eb4a41ea..c1598777a7187 100755 --- a/dev/create-release/do-release.sh +++ b/dev/create-release/do-release.sh @@ -44,7 +44,12 @@ else get_release_info fi -if [ $SKIP_TAG = 0 ]; then +function should_build { + local WHAT=$1 + [ -n "$RELEASE_STEP" ] && [ "$WHAT" = "$RELEASE_STEP" ] +} + +if should_build "docs" && [ $SKIP_TAG = 0 ]; then maybe_run "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." @@ -54,9 +59,23 @@ else echo "Skipping tag creation for $RELEASE_TAG." fi -run_silent "Building Spark..." "build.log" \ - "$SELF/release-build.sh" package -run_silent "Building documentation..." "docs.log" \ - "$SELF/release-build.sh" docs -maybe_run "Publishing release" "publish.log" \ - "$SELF/release-build.sh" publish-release +if should_build "build"; then + run_silent "Building Spark..." "build.log" \ + "$SELF/release-build.sh" package +else + echo "Skipping build step." +fi + +if should_build "docs"; then + run_silent "Building documentation..." "docs.log" \ + "$SELF/release-build.sh" docs +else + echo "Skipping docs step." +fi + +if should_build "publish"; then + maybe_run "Publishing release" "publish.log" \ + "$SELF/release-build.sh" publish-release +else + echo "Skipping publish step." +fi diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 515a042b02f36..38407982bfbc2 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -96,6 +96,7 @@ java_version=$("${JAVA_HOME}"/bin/javac -version 2>&1 | cut -d " " -f 2) MVN_EXTRA_OPTS= if [[ $java_version < "1.8." ]]; then # Needed for maven central when using Java 7. + export SBT_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" MVN_EXTRA_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" MVN="$MVN $MVN_EXTRA_OPTS" fi diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index ef308461830fc..07ce320177f5a 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -29,9 +29,15 @@ 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" +ARG BASE_PIP_PKGS="setuptools wheel virtualenv" +ARG PIP_PKGS="pyopenssl pypandoc numpy pygments sphinx" + # Install extra needed repos and refresh. # - CRAN repo # - Ruby repo (for doc generation) +# +# 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 echo 'deb http://cran.cnr.Berkeley.edu/bin/linux/ubuntu xenial/' >> /etc/apt/sources.list && \ gpg --keyserver keyserver.ubuntu.com --recv-key E084DAB9 && \ gpg -a --export E084DAB9 | apt-key add - && \ @@ -41,41 +47,33 @@ RUN echo 'deb http://cran.cnr.Berkeley.edu/bin/linux/ubuntu xenial/' >> /etc/apt apt-get update && \ $APT_INSTALL software-properties-common && \ apt-add-repository -y ppa:brightbox/ruby-ng && \ - apt-get update - -# Install openjdk 8. -RUN $APT_INSTALL openjdk-8-jdk && \ - update-alternatives --set java /usr/lib/jvm/java-8-openjdk-amd64/jre/bin/java - -# Install build / source control tools -RUN $APT_INSTALL curl wget git maven ivy subversion make gcc libffi-dev \ + 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 wget git maven ivy subversion make gcc lsof libffi-dev \ pandoc pandoc-citeproc libssl-dev libcurl4-openssl-dev libxml2-dev && \ ln -s -T /usr/share/java/ivy.jar /usr/share/ant/lib/ivy.jar && \ curl -sL https://deb.nodesource.com/setup_4.x | bash && \ - $APT_INSTALL nodejs - -# Install needed python packages. Use pip for installing packages (for consistency). -ARG BASE_PIP_PKGS="setuptools wheel virtualenv" -ARG PIP_PKGS="pyopenssl pypandoc numpy pygments sphinx" - -RUN $APT_INSTALL libpython2.7-dev libpython3-dev python-pip python3-pip && \ + $APT_INSTALL nodejs && \ + # Install needed python packages. Use pip for installing packages (for consistency). + $APT_INSTALL libpython2.7-dev libpython3-dev python-pip python3-pip && \ pip install $BASE_PIP_PKGS && \ pip install $PIP_PKGS && \ cd && \ virtualenv -p python3 p35 && \ . p35/bin/activate && \ pip install $BASE_PIP_PKGS && \ - pip install $PIP_PKGS - -# Install R packages and dependencies used when building. -# R depends on pandoc*, libssl (which are installed above). -RUN $APT_INSTALL r-base r-base-dev && \ + pip install $PIP_PKGS && \ + # Install R packages and dependencies used when building. + # R depends on pandoc*, libssl (which are installed above). + $APT_INSTALL r-base r-base-dev && \ $APT_INSTALL texlive-latex-base texlive texlive-fonts-extra texinfo qpdf && \ Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='http://cran.us.r-project.org/')" && \ - Rscript -e "devtools::install_github('jimhester/lintr')" - -# Install tools needed to build the documentation. -RUN $APT_INSTALL ruby2.3 ruby2.3-dev && \ + Rscript -e "devtools::install_github('jimhester/lintr')" && \ + # Install tools needed to build the documentation. + $APT_INSTALL ruby2.3 ruby2.3-dev && \ gem install jekyll --no-rdoc --no-ri && \ gem install jekyll-redirect-from && \ gem install pygments.rb From b6d4c705b7aea7b55a1618892cba12bedb1cd6c5 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 18 Jun 2018 19:47:30 -0700 Subject: [PATCH 10/11] Even more fixes. - Some bugs I introduced in last commit. - Fix release-tag.sh with Java 1.7. - Expand dry run to tag and publish steps. --- dev/create-release/do-release.sh | 8 +-- dev/create-release/release-build.sh | 81 +++++++++++++---------------- dev/create-release/release-tag.sh | 26 ++++++--- dev/create-release/release-util.sh | 37 ++++++++----- 4 files changed, 82 insertions(+), 70 deletions(-) diff --git a/dev/create-release/do-release.sh b/dev/create-release/do-release.sh index c1598777a7187..f1d4f3ab5ddec 100755 --- a/dev/create-release/do-release.sh +++ b/dev/create-release/do-release.sh @@ -46,11 +46,11 @@ fi function should_build { local WHAT=$1 - [ -n "$RELEASE_STEP" ] && [ "$WHAT" = "$RELEASE_STEP" ] + [ -z "$RELEASE_STEP" ] || [ "$WHAT" = "$RELEASE_STEP" ] } -if should_build "docs" && [ $SKIP_TAG = 0 ]; then - maybe_run "Creating release tag $RELEASE_TAG..." "tag.log" \ +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." @@ -74,7 +74,7 @@ else fi if should_build "publish"; then - maybe_run "Publishing release" "publish.log" \ + run_silent "Publishing release" "publish.log" \ "$SELF/release-build.sh" publish-release else echo "Skipping publish step." diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 38407982bfbc2..24a62a8f4c7d3 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -90,19 +90,11 @@ NEXUS_ROOT=https://repository.apache.org/service/local/staging NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads BASE_DIR=$(pwd) -MVN="build/mvn -B" - -java_version=$("${JAVA_HOME}"/bin/javac -version 2>&1 | cut -d " " -f 2) -MVN_EXTRA_OPTS= -if [[ $java_version < "1.8." ]]; then - # Needed for maven central when using Java 7. - export SBT_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" - MVN_EXTRA_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" - MVN="$MVN $MVN_EXTRA_OPTS" -fi +init_java +init_maven_sbt rm -rf spark -git clone https://git-wip-us.apache.org/repos/asf/spark.git +git clone "$ASF_REPO" cd spark git checkout $GIT_REF git_hash=`git rev-parse --short HEAD` @@ -138,22 +130,16 @@ PUBLISH_PROFILES="$BASE_PROFILES $HIVE_PROFILES -Pspark-ganglia-lgpl -Pkinesis-a # Profiles for building binary releases BASE_RELEASE_PROFILES="$BASE_PROFILES -Psparkr" -# Verify we have the right java version set -if [ -z "$JAVA_HOME" ]; then - echo "Please set JAVA_HOME." - exit 1 -fi - if [[ ! $SPARK_VERSION < "2.2." ]]; then - if [[ $java_version < "1.8." ]]; then - echo "Java version $java_version is less than required 1.8 for 2.2+" + if [[ $JAVA_VERSION < "1.8." ]]; then + echo "Java version $JAVA_VERSION is less than required 1.8 for 2.2+" echo "Please set JAVA_HOME correctly." exit 1 fi else - if ! [[ $java_version =~ 1\.7\..* ]]; then + 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.2" + echo "Java version $JAVA_VERSION is higher than required 1.7 for pre-2.2" echo "Please set JAVA_HOME correctly." exit 1 else @@ -378,13 +364,15 @@ if [[ "$1" == "publish-release" ]]; then # Using Nexus API documented here: # https://support.sonatype.com/entries/39720203-Uploading-to-a-Staging-Repository-via-REST-API - echo "Creating Nexus staging repository" - repo_request="Apache Spark $SPARK_VERSION (commit $git_hash)" - 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/.*\(orgapachespark-[0-9]\{4\}\).*/\1/") - echo "Created Nexus staging repository: $staged_repo_id" + if ! is_dry_run; then + echo "Creating Nexus staging repository" + repo_request="Apache Spark $SPARK_VERSION (commit $git_hash)" + 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/.*\(orgapachespark-[0-9]\{4\}\).*/\1/") + echo "Created Nexus staging repository: $staged_repo_id" + fi tmp_repo=$(mktemp -d spark-repo-XXXXX) @@ -393,7 +381,7 @@ if [[ "$1" == "publish-release" ]]; then $MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo -DskipTests $SCALA_2_11_PROFILES $PUBLISH_PROFILES clean install - if [[ $PUBLISH_SCALA_2_10 = 1 ]]; then + if ! is_dry_run && [[ $PUBLISH_SCALA_2_10 = 1 ]]; then ./dev/change-scala-version.sh 2.10 $MVN -DzincPort=$((ZINC_PORT + 1)) -Dmaven.repo.local=$tmp_repo -Dscala-2.10 \ -DskipTests $PUBLISH_PROFILES $SCALA_2_10_PROFILES clean install @@ -429,23 +417,26 @@ if [[ "$1" == "publish-release" ]]; then sha1sum $file | cut -f1 -d' ' > $file.sha1 done - nexus_upload=$NEXUS_ROOT/deployByRepositoryId/$staged_repo_id - echo "Uplading files to $nexus_upload" - for file in $(find . -type f) - do - # strip leading ./ - file_short=$(echo $file | sed -e "s/\.\///") - dest_url="$nexus_upload/org/apache/spark/$file_short" - echo " Uploading $file_short" - curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url - done + if ! is_dry_run; then + nexus_upload=$NEXUS_ROOT/deployByRepositoryId/$staged_repo_id + echo "Uplading files to $nexus_upload" + for file in $(find . -type f) + do + # strip leading ./ + file_short=$(echo $file | sed -e "s/\.\///") + dest_url="$nexus_upload/org/apache/spark/$file_short" + echo " Uploading $file_short" + curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url + done + + echo "Closing nexus staging repository" + repo_request="$staged_repo_idApache Spark $SPARK_VERSION (commit $git_hash)" + 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 - echo "Closing nexus staging repository" - repo_request="$staged_repo_idApache Spark $SPARK_VERSION (commit $git_hash)" - 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" popd rm -rf $tmp_repo cd .. diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index a05716a5f66bb..628bc0504c9c8 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -17,6 +17,9 @@ # limitations under the License. # +SELF=$(cd $(dirname $0) && pwd) +. "$SELF/release-util.sh" + function exit_with_usage { cat << EOF usage: tag-release.sh @@ -36,6 +39,7 @@ EOF } set -e +set -o pipefail if [[ $@ == *"help"* ]]; then exit_with_usage @@ -54,8 +58,10 @@ for env in ASF_USERNAME ASF_PASSWORD RELEASE_VERSION RELEASE_TAG NEXT_VERSION GI fi done +init_java +init_maven_sbt + ASF_SPARK_REPO="git-wip-us.apache.org/repos/asf/spark.git" -MVN="build/mvn --force" rm -rf spark git clone "https://$ASF_USERNAME:$ASF_PASSWORD@$ASF_SPARK_REPO" -b $GIT_BRANCH @@ -94,9 +100,15 @@ sed -i".tmp7" 's/SPARK_VERSION_SHORT:.*$/SPARK_VERSION_SHORT: '"$R_NEXT_VERSION" git commit -a -m "Preparing development version $NEXT_VERSION" -# Push changes -git push origin $RELEASE_TAG -git push origin HEAD:$GIT_BRANCH - -cd .. -rm -rf spark +if ! is_dry_run; then + # Push changes + git push origin $RELEASE_TAG + git push origin HEAD:$GIT_BRANCH + + cd .. + rm -rf spark +else + cd .. + mv spark spark.tag + echo "Clone with version changes and tag available as spark.tag in the output directory." +fi diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index 29295cc6927cd..783684fd6a386 100644 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -65,20 +65,6 @@ function run_silent { fi } -function maybe_run { - if is_dry_run; then - local BANNER="$1" - local LOG_FILE="$2" - shift 2 - echo "======= DRY RUN =======" - echo "= $BANNER" - echo "Command: $@" - echo "Log file: $LOG_FILE" - else - run_silent "$@" - fi -} - function fcreate_secure { local FPATH="$1" rm -f "$FPATH" @@ -217,3 +203,26 @@ EOF 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 +} + +# Initializes MVN_EXTRA_OPTS and SBT_OPTS depending on the JAVA_VERSION in use. Requires init_java. +function init_maven_sbt { + MVN="build/mvn -B" + MVN_EXTRA_OPTS= + SBT_OPTS= + if [[ $JAVA_VERSION < "1.8." ]]; then + # Needed for maven central when using Java 7. + SBT_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" + MVN_EXTRA_OPTS="-Dhttps.protocols=TLSv1.1,TLSv1.2" + MVN="$MVN $MVN_EXTRA_OPTS" + fi + export MVN MVN_EXTRA_OPTS SBT_OPTS +} From 745342d499ef290d5ce5484bcb2349d2be57a2fb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 19 Jun 2018 11:46:09 -0700 Subject: [PATCH 11/11] Avoid silent failure if user.name is not set. --- dev/create-release/release-util.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index 783684fd6a386..7426b0d6ca08d 100644 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -157,7 +157,7 @@ function get_release_info { # Gather some user information. export ASF_USERNAME=$(read_config "ASF user" "$LOGNAME") - GIT_NAME=$(git config user.name) + GIT_NAME=$(git config user.name || echo "") export GIT_NAME=$(read_config "Full name" "$GIT_NAME") export GIT_EMAIL="$ASF_USERNAME@apache.org"