Skip to content

Commit

Permalink
[SPARK-32245][INFRA] Run Spark tests in Github Actions
Browse files Browse the repository at this point in the history
### What changes were proposed in this pull request?

This PR aims to run the Spark tests in Github Actions.

To briefly explain the main idea:

- Reuse `dev/run-tests.py` with SBT build
- Reuse the modules in `dev/sparktestsupport/modules.py` to test each module
- Pass the modules to test into `dev/run-tests.py` directly via `TEST_ONLY_MODULES` environment variable. For example, `pyspark-sql,core,sql,hive`.
- `dev/run-tests.py` _does not_ take the dependent modules into account but solely the specified modules to test.

Another thing to note might be `SlowHiveTest` annotation. Running the tests in Hive modules takes too much so the slow tests are extracted and it runs as a separate job. It was extracted from the actual elapsed time in Jenkins:

![Screen Shot 2020-07-09 at 7 48 13 PM](https://user-images.githubusercontent.com/6477701/87050238-f6098e80-c238-11ea-9c4a-ab505af61381.png)

So, Hive tests are separated into to jobs. One is slow test cases, and the other one is the other test cases.

_Note that_ the current GitHub Actions build virtually copies what the default PR builder on Jenkins does (without other profiles such as JDK 11, Hadoop 2, etc.). The only exception is Kinesis https://github.com/apache/spark/pull/29057/files#diff-04eb107ee163a50b61281ca08f4e4c7bR23

### Why are the changes needed?

Last week and onwards, the Jenkins machines became very unstable for many reasons:
  - Apparently, the machines became extremely slow. Almost all tests can't pass.
  - One machine (worker 4) started to have the corrupt `.m2` which fails the build.
  - Documentation build fails time to time for an unknown reason in Jenkins machine specifically. This is disabled for now at #29017.
  - Almost all PRs are basically blocked by this instability currently.

The advantages of using Github Actions:
  - To avoid depending on few persons who can access to the cluster.
  - To reduce the elapsed time in the build - we could split the tests (e.g., SQL, ML, CORE), and run them in parallel so the total build time will significantly reduce.
  - To control the environment more flexibly.
  - Other contributors can test and propose to fix Github Actions configurations so we can distribute this build management cost.

Note that:
- The current build in Jenkins takes _more than 7 hours_. With Github actions it takes _less than 2 hours_
- We can now control the environments especially for Python easily.
- The test and build look more stable than the Jenkins'.

### Does this PR introduce _any_ user-facing change?

No, dev-only change.

### How was this patch tested?

Tested at HyukjinKwon#4

Closes #29057 from HyukjinKwon/migrate-to-github-actions.

Authored-by: HyukjinKwon <gurwls223@apache.org>
Signed-off-by: Dongjoon Hyun <dongjoon@apache.org>
  • Loading branch information
HyukjinKwon authored and dongjoon-hyun committed Jul 11, 2020
1 parent 10a65ee commit b84ed41
Show file tree
Hide file tree
Showing 20 changed files with 400 additions and 173 deletions.
283 changes: 177 additions & 106 deletions .github/workflows/master.yml
Original file line number Diff line number Diff line change
@@ -1,156 +1,227 @@
name: master

on:
push:
branches:
- master
pull_request:
branches:
- master

jobs:
# TODO(SPARK-32248): Recover JDK 11 builds
# Build: build Spark and run the tests for specified modules.
build:

name: "Build modules: ${{ matrix.modules }} ${{ matrix.comment }} (JDK ${{ matrix.java }}, ${{ matrix.hadoop }}, ${{ matrix.hive }})"
runs-on: ubuntu-latest
strategy:
fail-fast: false
matrix:
java: [ '1.8', '11' ]
hadoop: [ 'hadoop-2.7', 'hadoop-3.2' ]
hive: [ 'hive-1.2', 'hive-2.3' ]
exclude:
- java: '11'
hive: 'hive-1.2'
- hadoop: 'hadoop-3.2'
hive: 'hive-1.2'
name: Build Spark - JDK${{ matrix.java }}/${{ matrix.hadoop }}/${{ matrix.hive }}

java:
- 1.8
hadoop:
- hadoop3.2
hive:
- hive2.3
# TODO(SPARK-32246): We don't test 'streaming-kinesis-asl' for now.
# Kinesis tests depends on external Amazon kinesis service.
# Note that the modules below are from sparktestsupport/modules.py.
modules:
- |-
core, unsafe, kvstore, avro,
network_common, network_shuffle, repl, launcher
examples, sketch, graphx
- |-
catalyst, hive-thriftserver
- |-
streaming, sql-kafka-0-10, streaming-kafka-0-10,
mllib-local, mllib,
yarn, mesos, kubernetes, hadoop-cloud, spark-ganglia-lgpl
- |-
pyspark-sql, pyspark-mllib, pyspark-resource
- |-
pyspark-core, pyspark-streaming, pyspark-ml
- |-
sparkr
# Here, we split Hive and SQL tests into some of slow ones and the rest of them.
included-tags: [""]
excluded-tags: [""]
comment: [""]
include:
# Hive tests
- modules: hive
java: 1.8
hadoop: hadoop3.2
hive: hive2.3
included-tags: org.apache.spark.tags.SlowHiveTest
comment: "- slow tests"
- modules: hive
java: 1.8
hadoop: hadoop3.2
hive: hive2.3
excluded-tags: org.apache.spark.tags.SlowHiveTest
comment: "- other tests"
# SQL tests
- modules: sql
java: 1.8
hadoop: hadoop3.2
hive: hive2.3
included-tags: org.apache.spark.tags.ExtendedSQLTest
comment: "- slow tests"
- modules: sql
java: 1.8
hadoop: hadoop3.2
hive: hive2.3
excluded-tags: org.apache.spark.tags.ExtendedSQLTest
comment: "- other tests"
env:
TEST_ONLY_MODULES: ${{ matrix.modules }}
TEST_ONLY_EXCLUDED_TAGS: ${{ matrix.excluded-tags }}
TEST_ONLY_INCLUDED_TAGS: ${{ matrix.included-tags }}
HADOOP_PROFILE: ${{ matrix.hadoop }}
HIVE_PROFILE: ${{ matrix.hive }}
# GitHub Actions' default miniconda to use in pip packaging test.
CONDA_PREFIX: /usr/share/miniconda
steps:
- uses: actions/checkout@master
# We split caches because GitHub Action Cache has a 400MB-size limit.
- uses: actions/cache@v1
- name: Checkout Spark repository
uses: actions/checkout@v2
# Cache local repositories. Note that GitHub Actions cache has a 2G limit.
- name: Cache Scala, SBT, Maven and Zinc
uses: actions/cache@v1
with:
path: build
key: build-${{ hashFiles('**/pom.xml') }}
restore-keys: |
build-
- uses: actions/cache@v1
with:
path: ~/.m2/repository/com
key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-com-${{ hashFiles('**/pom.xml') }}
restore-keys: |
${{ matrix.java }}-${{ matrix.hadoop }}-maven-com-
- uses: actions/cache@v1
- name: Cache Maven local repository
uses: actions/cache@v2
with:
path: ~/.m2/repository/org
key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-org-${{ hashFiles('**/pom.xml') }}
restore-keys: |
${{ matrix.java }}-${{ matrix.hadoop }}-maven-org-
- uses: actions/cache@v1
with:
path: ~/.m2/repository/net
key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-net-${{ hashFiles('**/pom.xml') }}
path: ~/.m2/repository
key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-${{ hashFiles('**/pom.xml') }}
restore-keys: |
${{ matrix.java }}-${{ matrix.hadoop }}-maven-net-
- uses: actions/cache@v1
${{ matrix.java }}-${{ matrix.hadoop }}-maven-
- name: Cache Ivy local repository
uses: actions/cache@v2
with:
path: ~/.m2/repository/io
key: ${{ matrix.java }}-${{ matrix.hadoop }}-maven-io-${{ hashFiles('**/pom.xml') }}
path: ~/.ivy2/cache
key: ${{ matrix.java }}-${{ matrix.hadoop }}-ivy-${{ hashFiles('**/pom.xml') }}-${{ hashFiles('**/plugins.sbt') }}
restore-keys: |
${{ matrix.java }}-${{ matrix.hadoop }}-maven-io-
- name: Set up JDK ${{ matrix.java }}
${{ matrix.java }}-${{ matrix.hadoop }}-ivy-
- name: Install JDK ${{ matrix.java }}
uses: actions/setup-java@v1
with:
java-version: ${{ matrix.java }}
- name: Build with Maven
run: |
export MAVEN_OPTS="-Xmx2g -XX:ReservedCodeCacheSize=1g -Dorg.slf4j.simpleLogger.defaultLogLevel=WARN"
export MAVEN_CLI_OPTS="--no-transfer-progress"
mkdir -p ~/.m2
./build/mvn $MAVEN_CLI_OPTS -DskipTests -Pyarn -Pmesos -Pkubernetes -Phive -P${{ matrix.hive }} -Phive-thriftserver -P${{ matrix.hadoop }} -Phadoop-cloud -Djava.version=${{ matrix.java }} install
rm -rf ~/.m2/repository/org/apache/spark
lint:
runs-on: ubuntu-latest
name: Linters (Java/Scala/Python), licenses, dependencies
steps:
- uses: actions/checkout@master
- uses: actions/setup-java@v1
# PySpark
- name: Install PyPy3
# SQL component also has Python related tests, for example, IntegratedUDFTestUtils.
# Note that order of Python installations here matters because default python3 is
# overridden by pypy3.
uses: actions/setup-python@v2
if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-'))
with:
java-version: '11'
- uses: actions/setup-python@v1
python-version: pypy3
architecture: x64
- name: Install Python 2.7
uses: actions/setup-python@v2
if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-'))
with:
python-version: '3.x'
architecture: 'x64'
- name: Scala
run: ./dev/lint-scala
- name: Java
run: ./dev/lint-java
- name: Python
run: |
pip install flake8 sphinx numpy
./dev/lint-python
- name: License
run: ./dev/check-license
- name: Dependencies
run: ./dev/test-dependencies.sh

lintr:
runs-on: ubuntu-latest
name: Linter (R)
steps:
- uses: actions/checkout@master
- uses: actions/setup-java@v1
python-version: 2.7
architecture: x64
- name: Install Python 3.6
uses: actions/setup-python@v2
if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-'))
with:
java-version: '11'
- uses: r-lib/actions/setup-r@v1
python-version: 3.6
architecture: x64
- name: Install Python packages
if: contains(matrix.modules, 'pyspark') || (contains(matrix.modules, 'sql') && !contains(matrix.modules, 'sql-'))
# PyArrow is not supported in PyPy yet, see ARROW-2651.
# TODO(SPARK-32247): scipy installation with PyPy fails for an unknown reason.
run: |
python3 -m pip install numpy pyarrow pandas scipy
python3 -m pip list
python2 -m pip install numpy pyarrow pandas scipy
python2 -m pip list
pypy3 -m pip install numpy pandas
pypy3 -m pip list
# SparkR
- name: Install R 3.6
uses: r-lib/actions/setup-r@v1
if: contains(matrix.modules, 'sparkr')
with:
r-version: '3.6.2'
- name: Install lib
r-version: 3.6
- name: Install R packages
if: contains(matrix.modules, 'sparkr')
run: |
sudo apt-get install -y libcurl4-openssl-dev
- name: install R packages
sudo Rscript -e "install.packages(c('knitr', 'rmarkdown', 'testthat', 'devtools', 'e1071', 'survival', 'arrow', 'roxygen2'), repos='https://cloud.r-project.org/')"
# Show installed packages in R.
sudo Rscript -e 'pkg_list <- as.data.frame(installed.packages()[, c(1,3:4)]); pkg_list[is.na(pkg_list$Priority), 1:2, drop = FALSE]'
# Run the tests.
- name: "Run tests: ${{ matrix.modules }}"
run: |
sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')"
sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')"
- name: package and install SparkR
run: ./R/install-dev.sh
- name: lint-r
run: ./dev/lint-r
# Hive tests become flaky when running in parallel as it's too intensive.
if [[ "$TEST_ONLY_MODULES" == "hive" ]]; then export SERIAL_SBT_TESTS=1; fi
mkdir -p ~/.m2
./dev/run-tests --parallelism 2
rm -rf ~/.m2/repository/org/apache/spark
docs:
# Static analysis, and documentation build
lint:
name: Linters, licenses, dependencies and documentation generation
runs-on: ubuntu-latest
name: Generate documents
steps:
- uses: actions/checkout@master
- uses: actions/cache@v1
- name: Checkout Spark repository
uses: actions/checkout@v2
- name: Cache Maven local repository
uses: actions/cache@v2
with:
path: ~/.m2/repository
key: docs-maven-repo-${{ hashFiles('**/pom.xml') }}
restore-keys: |
docs-maven-repo-
- uses: actions/setup-java@v1
docs-maven-
- name: Install JDK 1.8
uses: actions/setup-java@v1
with:
java-version: '1.8'
- uses: actions/setup-python@v1
java-version: 1.8
- name: Install Python 3.6
uses: actions/setup-python@v2
with:
python-version: '3.x'
architecture: 'x64'
- uses: actions/setup-ruby@v1
python-version: 3.6
architecture: x64
- name: Install Python linter dependencies
run: |
pip3 install flake8 sphinx numpy
- name: Install R 3.6
uses: r-lib/actions/setup-r@v1
with:
ruby-version: '2.7'
- uses: r-lib/actions/setup-r@v1
r-version: 3.6
- name: Install R linter dependencies and SparkR
run: |
sudo apt-get install -y libcurl4-openssl-dev
sudo Rscript -e "install.packages(c('devtools'), repos='https://cloud.r-project.org/')"
sudo Rscript -e "devtools::install_github('jimhester/lintr@v2.0.0')"
./R/install-dev.sh
- name: Install Ruby 2.7 for documentation generation
uses: actions/setup-ruby@v1
with:
r-version: '3.6.2'
- name: Install lib and pandoc
ruby-version: 2.7
- name: Install dependencies for documentation generation
run: |
sudo apt-get install -y libcurl4-openssl-dev pandoc
- name: Install packages
run: |
pip install sphinx mkdocs numpy
gem install jekyll jekyll-redirect-from rouge
sudo Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='https://cloud.r-project.org/')"
- name: Run jekyll build
sudo Rscript -e "install.packages(c('devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2'), repos='https://cloud.r-project.org/')"
- name: Scala linter
run: ./dev/lint-scala
- name: Java linter
run: ./dev/lint-java
- name: Python linter
run: ./dev/lint-python
- name: R linter
run: ./dev/lint-r
- name: License test
run: ./dev/check-license
- name: Dependencies test
run: ./dev/test-dependencies.sh
- name: Run documentation build
run: |
cd docs
jekyll build
30 changes: 30 additions & 0 deletions common/tags/src/test/java/org/apache/spark/tags/SlowHiveTest.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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.
*/

package org.apache.spark.tags;

import org.scalatest.TagAnnotation;

import java.lang.annotation.ElementType;
import java.lang.annotation.Retention;
import java.lang.annotation.RetentionPolicy;
import java.lang.annotation.Target;

@TagAnnotation
@Retention(RetentionPolicy.RUNTIME)
@Target({ElementType.METHOD, ElementType.TYPE})
public @interface SlowHiveTest { }
Original file line number Diff line number Diff line change
Expand Up @@ -685,7 +685,8 @@ class MasterSuite extends SparkFunSuite
}
}

test("SPARK-27510: Master should avoid dead loop while launching executor failed in Worker") {
// TODO(SPARK-32250): Enable the test back. It is flaky in GitHub Actions.
ignore("SPARK-27510: Master should avoid dead loop while launching executor failed in Worker") {
val master = makeAliveMaster()
var worker: MockExecutorLaunchFailWorker = null
try {
Expand Down
Loading

0 comments on commit b84ed41

Please sign in to comment.