[SPARK-18639] Build only a single pip package
## What changes were proposed in this pull request? We current build 5 separate pip binary tar balls, doubling the release script runtime. It'd be better to build one, especially for use cases that are just using Spark locally. In the long run, it would make more sense to have Hadoop support be pluggable. ## How was this patch tested? N/A - this is a release build script that doesn't have any automated test coverage. We will know if it goes wrong when we prepare releases. Author: Reynold Xin <rxin@databricks.com> Closes #16072 from rxin/SPARK-18639.
This commit is contained in:
parent
086b0c8f67
commit
37e52f8793
|
@ -150,6 +150,7 @@ if [[ "$1" == "package" ]]; then
|
|||
NAME=$1
|
||||
FLAGS=$2
|
||||
ZINC_PORT=$3
|
||||
BUILD_PIP_PACKAGE=$4
|
||||
cp -r spark spark-$SPARK_VERSION-bin-$NAME
|
||||
|
||||
cd spark-$SPARK_VERSION-bin-$NAME
|
||||
|
@ -170,24 +171,32 @@ if [[ "$1" == "package" ]]; then
|
|||
# Get maven home set by MVN
|
||||
MVN_HOME=`$MVN -version 2>&1 | grep 'Maven home' | awk '{print $NF}'`
|
||||
|
||||
echo "Creating distribution"
|
||||
./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --pip $FLAGS \
|
||||
-DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log
|
||||
cd ..
|
||||
|
||||
echo "Copying and signing python distribution"
|
||||
PYTHON_DIST_NAME=pyspark-$PYSPARK_VERSION.tar.gz
|
||||
cp spark-$SPARK_VERSION-bin-$NAME/python/dist/$PYTHON_DIST_NAME .
|
||||
if [ -z "$BUILD_PIP_PACKAGE" ]; then
|
||||
echo "Creating distribution without PIP package"
|
||||
./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz $FLAGS \
|
||||
-DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log
|
||||
cd ..
|
||||
else
|
||||
echo "Creating distribution with PIP package"
|
||||
./dev/make-distribution.sh --name $NAME --mvn $MVN_HOME/bin/mvn --tgz --pip $FLAGS \
|
||||
-DzincPort=$ZINC_PORT 2>&1 > ../binary-release-$NAME.log
|
||||
cd ..
|
||||
|
||||
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour \
|
||||
--output $PYTHON_DIST_NAME.asc \
|
||||
--detach-sig $PYTHON_DIST_NAME
|
||||
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
|
||||
MD5 $PYTHON_DIST_NAME > \
|
||||
$PYTHON_DIST_NAME.md5
|
||||
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
|
||||
SHA512 $PYTHON_DIST_NAME > \
|
||||
$PYTHON_DIST_NAME.sha
|
||||
echo "Copying and signing python distribution"
|
||||
PYTHON_DIST_NAME=pyspark-$PYSPARK_VERSION.tar.gz
|
||||
cp spark-$SPARK_VERSION-bin-$NAME/python/dist/$PYTHON_DIST_NAME .
|
||||
|
||||
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour \
|
||||
--output $PYTHON_DIST_NAME.asc \
|
||||
--detach-sig $PYTHON_DIST_NAME
|
||||
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
|
||||
MD5 $PYTHON_DIST_NAME > \
|
||||
$PYTHON_DIST_NAME.md5
|
||||
echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --print-md \
|
||||
SHA512 $PYTHON_DIST_NAME > \
|
||||
$PYTHON_DIST_NAME.sha
|
||||
fi
|
||||
|
||||
echo "Copying and signing regular binary distribution"
|
||||
cp spark-$SPARK_VERSION-bin-$NAME/spark-$SPARK_VERSION-bin-$NAME.tgz .
|
||||
|
@ -211,7 +220,7 @@ if [[ "$1" == "package" ]]; then
|
|||
make_binary_release "hadoop2.3" "-Phadoop-2.3 $FLAGS" "3033" &
|
||||
make_binary_release "hadoop2.4" "-Phadoop-2.4 $FLAGS" "3034" &
|
||||
make_binary_release "hadoop2.6" "-Phadoop-2.6 $FLAGS" "3035" &
|
||||
make_binary_release "hadoop2.7" "-Phadoop-2.7 $FLAGS" "3036" &
|
||||
make_binary_release "hadoop2.7" "-Phadoop-2.7 $FLAGS" "3036" "withpip" &
|
||||
make_binary_release "hadoop2.4-without-hive" "-Psparkr -Phadoop-2.4 -Pyarn -Pmesos" "3037" &
|
||||
make_binary_release "without-hadoop" "-Psparkr -Phadoop-provided -Pyarn -Pmesos" "3038" &
|
||||
wait
|
||||
|
|
Loading…
Reference in a new issue