Skip to content

Commit

Permalink
[CI] Move flaky tests that fail very often to "quarantine" test group (
Browse files Browse the repository at this point in the history
…apache#10148)

### Motivation

There are a few tests that fail very often. This is blocking the merging of PRs currently. 

### Modifications

Move the problematic tests to "quarantine" test group. This test group will be run, but the test failures will be ignored.
- Configure "excludedGroups" property with "quarantine" default value
- Make modifications so that tests in the quarantine group can be run. It is required to make changes to "BeforeMethod/BeforeClass" annotations so that the method will get run for the quarantine group. `alwaysRun=true` should be used instead of listing individual groups in the Before* annotations.
- Add reporting for quarantined tests so that the PR reviewer can check if there's a sudden change in Quarantined test results
Test result will be visible directly in GitHub Actions UI, [example](https://github.com/lhotari/pulsar/actions/runs/726717497).
![image](https://user-images.githubusercontent.com/66864/113923302-1464e080-97f1-11eb-8719-a25f854b7f8e.png)
[Detailed quarantined test results](https://github.com/lhotari/pulsar/runs/2289519905?check_suite_focus=true#step:8:165):
![image](https://user-images.githubusercontent.com/66864/113923589-5f7ef380-97f1-11eb-9179-8fc0e960f9fa.png)

### Issue reports for quarantined tests

* apache#10150
* apache#9916
* apache#10151
* apache#10117
* apache#10152
* apache#6300
  • Loading branch information
lhotari authored Apr 13, 2021
1 parent 197bd93 commit a7933ac
Show file tree
Hide file tree
Showing 80 changed files with 444 additions and 241 deletions.
6 changes: 5 additions & 1 deletion .github/workflows/ci-unit-broker-broker-gp1.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@ jobs:
unit-tests:
name:
runs-on: ubuntu-latest
timeout-minutes: 45
timeout-minutes: 60

steps:
- name: checkout
Expand Down Expand Up @@ -84,6 +84,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh BROKER_GROUP_1

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
4 changes: 4 additions & 0 deletions .github/workflows/ci-unit-broker-broker-gp2.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh BROKER_GROUP_2

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
4 changes: 4 additions & 0 deletions .github/workflows/ci-unit-broker-client-api.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh BROKER_CLIENT_API

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
4 changes: 4 additions & 0 deletions .github/workflows/ci-unit-broker-client-impl.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh BROKER_CLIENT_IMPL

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
4 changes: 4 additions & 0 deletions .github/workflows/ci-unit-broker-other.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh BROKER_FLAKY

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
4 changes: 4 additions & 0 deletions .github/workflows/ci-unit-proxy.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -84,6 +84,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh PROXY

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
4 changes: 4 additions & 0 deletions .github/workflows/ci-unit.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -80,6 +80,10 @@ jobs:
if: ${{ steps.changes.outputs.all_count > steps.changes.outputs.docs_count }}
run: ./build/run_unit_group.sh OTHER

- name: print JVM thread dumps when cancelled
if: cancelled()
run: ./build/pulsar_ci_tool.sh print_thread_dumps

- name: package surefire artifacts
if: failure()
run: |
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@ public class TlsProducerConsumerBase extends ProducerConsumerBase {
protected final String TLS_SERVER_KEY_FILE_PATH = "./src/test/resources/authentication/tls/broker-key.pem";
private final String clusterName = "use";

@BeforeMethod
@BeforeMethod(alwaysRun = true)
@Override
protected void setup() throws Exception {
// TLS configuration for Broker
Expand Down
56 changes: 56 additions & 0 deletions build/pulsar_ci_tool.sh
Original file line number Diff line number Diff line change
@@ -0,0 +1,56 @@
#!/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.
#

# shell function library for Pulsar CI builds

# lists all available functions in this tool
function ci_list_functions() {
declare -F | awk '{print $NF}' | sort | egrep '^ci_' | sed 's/^ci_//'
}

# prints thread dumps for all running JVMs
# used in CI when a job gets cancelled because of a job timeout
function ci_print_thread_dumps() {
for java_pid in $(jps -q -J-XX:+PerfDisableSharedMem); do
echo "----------------------- pid $java_pid -----------------------"
cat /proc/$java_pid/cmdline | xargs -0 echo
jcmd $java_pid Thread.print -l
jcmd $java_pid GC.heap_info
done
return 0
}

if [ -z "$1" ]; then
echo "usage: $0 [ci_tool_function_name]"
echo "Available ci tool functions:"
ci_list_functions
exit 1
fi
ci_function_name="ci_$1"
shift

if [[ "$(LC_ALL=C type -t $ci_function_name)" == "function" ]]; then
eval "$ci_function_name" "$@"
else
echo "Invalid ci tool function"
echo "Available ci tool functions:"
ci_list_functions
exit 1
fi
92 changes: 79 additions & 13 deletions build/run_unit_group.sh
Original file line number Diff line number Diff line change
Expand Up @@ -19,11 +19,20 @@
#

set -e
set -x
set -o pipefail
set -o errexit

MVN_TEST_COMMAND='build/retry.sh mvn -B -ntp test'
# solution for printing output in "set -x" trace mode without tracing the echo calls
shopt -s expand_aliases
echo_and_restore_trace() {
builtin echo "$@"
[ $trace_enabled -eq 1 ] && set -x || true
}
alias echo='{ [[ $- =~ .*x.* ]] && trace_enabled=1 || trace_enabled=0; set +x; } 2> /dev/null; echo_and_restore_trace'

MVN_COMMAND='mvn -B -ntp'
MVN_COMMAND_WITH_RETRY="build/retry.sh ${MVN_COMMAND}"
MVN_TEST_COMMAND="${MVN_COMMAND_WITH_RETRY} test"

echo -n "Test Group : $TEST_GROUP"

Expand All @@ -44,42 +53,99 @@ function broker_client_impl() {
$MVN_TEST_COMMAND -pl pulsar-broker -Dgroups='broker-impl'
}

# prints summaries of failed tests to console
# by using the targer/surefire-reports files
# works only when testForkCount > 1 since that is when surefire will create reports for individual test classes
function print_testng_failures() {
(
{ set +x; } 2>/dev/null
local testng_failed_file="$1"
local report_prefix="${2:-Test failure in}"
local group_title="${3:-Detailed test failures}"
if [ -f "$testng_failed_file" ]; then
local testng_report_dir=$(dirname "$testng_failed_file")
local failed_count=0
for failed_test_class in $(cat "$testng_failed_file" | grep 'class name=' | perl -p -e 's/.*\"(.*?)\".*/$1/'); do
((failed_count += 1))
if [ $failed_count -eq 1 ]; then
echo "::endgroup::"
echo "::group::${group_title}"
fi
local test_report_file="${testng_report_dir}/${failed_test_class}.txt"
if [ -f "${test_report_file}" ]; then
local test_report="$(cat "${test_report_file}" | egrep "^Tests run: " | perl -p -se 's/^(Tests run: .*) <<< FAILURE! - in (.*)$/::warning::$report_prefix $2 - $1/' -- -report_prefix="${report_prefix}")"
echo "$test_report"
cat "${test_report_file}"
fi
done
fi
)
}

function broker_flaky() {
echo "::endgroup::"
echo "::group::Running quarantined tests"
$MVN_COMMAND test -pl pulsar-broker -Dgroups='quarantine' -DexcludedGroups='' -DfailIfNoTests=false \
-DtestForkCount=2 -Dexclude='**/Replicator*Test.java' ||
print_testng_failures pulsar-broker/target/surefire-reports/testng-failed.xml "Quarantined test failure in" "Quarantined test failures"
# run quarantined Replicator tests separately
$MVN_COMMAND test -pl pulsar-broker -Dgroups='quarantine' -DexcludedGroups='' -DfailIfNoTests=false \
-DtestForkCount=1 -DtestReuseFork=false -Dinclude='**/Replicator*Test.java' || \
print_testng_failures pulsar-broker/target/surefire-reports/testng-failed.xml "Quarantined test failure in" "Quarantined Replicator*Test failures"
echo "::endgroup::"
echo "::group::Running flaky tests"
$MVN_TEST_COMMAND -pl pulsar-broker -Dgroups='flaky' -DtestForkCount=1 -DtestReuseFork=false
echo "::endgroup::"
}

function proxy() {
echo "::endgroup::"
echo "::group::Running quarantined pulsar-proxy tests"
$MVN_COMMAND test -pl pulsar-proxy -Dgroups='quarantine' -DexcludedGroups='' -DfailIfNoTests=false ||
print_testng_failures pulsar-proxy/target/surefire-reports/testng-failed.xml "Quarantined test failure in" "Quarantined test failures"
echo "::endgroup::"
echo "::group::Running pulsar-proxy tests"
$MVN_TEST_COMMAND -pl pulsar-proxy
echo "::endgroup::"
}

function other() {
build/retry.sh mvn -B -ntp install -PbrokerSkipTest \
$MVN_COMMAND_WITH_RETRY clean install -PbrokerSkipTest \
-Dexclude='org/apache/pulsar/proxy/**/*.java,
**/ManagedLedgerTest.java,
**/TestPulsarKeyValueSchemaHandler.java,
**/PrimitiveSchemaTest.java,
BlobStoreManagedLedgerOffloaderTest.java'

$MVN_TEST_COMMAND -pl managed-ledger -Dinclude='**/ManagedLedgerTest.java,
**/OffloadersCacheTest.java' \
-DtestForkCount=1 \
-DtestReuseFork=true
**/OffloadersCacheTest.java'

$MVN_TEST_COMMAND -pl pulsar-sql/presto-pulsar-plugin -Dinclude='**/TestPulsarKeyValueSchemaHandler.java' \
-DtestForkCount=1
$MVN_TEST_COMMAND -pl pulsar-sql/presto-pulsar-plugin -Dinclude='**/TestPulsarKeyValueSchemaHandler.java'

$MVN_TEST_COMMAND -pl pulsar-client -Dinclude='**/PrimitiveSchemaTest.java' \
-DtestForkCount=1
$MVN_TEST_COMMAND -pl pulsar-client -Dinclude='**/PrimitiveSchemaTest.java'

$MVN_TEST_COMMAND -pl tiered-storage/jcloud -Dinclude='**/BlobStoreManagedLedgerOffloaderTest.java' \
-DtestForkCount=1
$MVN_TEST_COMMAND -pl tiered-storage/jcloud -Dinclude='**/BlobStoreManagedLedgerOffloaderTest.java'

echo "::endgroup::"
local modules_with_quarantined_tests=$(git grep -l '@Test.*"quarantine"' | grep '/src/test/java/' | \
awk -F '/src/test/java/' '{ print $1 }' | egrep -v 'pulsar-broker|pulsar-proxy' | sort | uniq | \
perl -0777 -p -e 's/\n(\S)/,$1/g')
if [ -n "${modules_with_quarantined_tests}" ]; then
echo "::group::Running quarantined tests outside of pulsar-broker & pulsar-proxy (if any)"
$MVN_COMMAND -pl "${modules_with_quarantined_tests}" test -Dgroups='quarantine' -DexcludedGroups='' \
-DfailIfNoTests=false || \
echo "::warning::There were test failures in the 'quarantine' test group."
echo "::endgroup::"
fi
}

# Test Groups -- end --

TEST_GROUP=$1

echo -n "Test Group : $TEST_GROUP"
echo "Test Group : $TEST_GROUP"

set -x

case $TEST_GROUP in

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,7 +40,7 @@ public abstract class TestRetrySupport {
private int failedSetupNumber = -1;
private int cleanedUpSetupNumber;

@BeforeMethod(groups = { "setup", "flaky", "extra" })
@BeforeMethod(alwaysRun = true)
public final void stateCheck(Method method) throws Exception {
// run cleanup and setup if the current setup number is the one where a failure happened
// this is to cleanup state before retrying
Expand All @@ -59,7 +59,7 @@ public final void stateCheck(Method method) throws Exception {
}
}

@AfterMethod(alwaysRun = true, groups = { "setup", "flaky", "extra" })
@AfterMethod(alwaysRun = true)
public final void failureCheck(ITestResult testResult, Method method) {
// track the setup number where the failure happened
if (!testResult.isSuccess()) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,9 +48,8 @@ public class EntryCacheManagerTest extends MockedBookKeeperTestCase {
ManagedLedgerImpl ml1;
ManagedLedgerImpl ml2;

@BeforeMethod
public void setup(Method method) throws Exception {
super.setUp(method);
@Override
protected void setUpTestCase() throws Exception {
OrderedScheduler executor = OrderedScheduler.newSchedulerBuilder().numThreads(1).build();

ml1 = mock(ManagedLedgerImpl.class);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,9 +52,8 @@ public class EntryCacheTest extends MockedBookKeeperTestCase {

private ManagedLedgerImpl ml;

@BeforeMethod
public void setUp(Method method) throws Exception {
super.setUp(method);
@Override
protected void setUpTestCase() throws Exception {
ml = mock(ManagedLedgerImpl.class);
when(ml.getName()).thenReturn("name");
when(ml.getExecutor()).thenReturn(executor);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ public BookKeeperClusterTestCase(int numBookies) {
this.numBookies = numBookies;
}

@BeforeMethod
@BeforeMethod(alwaysRun = true)
public void setUp() throws Exception {
// enable zookeeper `zookeeper.4lw.commands.whitelist`
System.setProperty("zookeeper.4lw.commands.whitelist", "*");
Expand Down Expand Up @@ -211,7 +211,7 @@ protected ServerConfiguration newServerConfiguration(String ledgerRootPath) thro
tmpDirs.add(f);
f.delete();
f.mkdir();

int port = 0;
return newServerConfiguration(port, zkUtil.getZooKeeperConnectString(), f, new File[] { f }, ledgerRootPath);
}
Expand Down
Loading

0 comments on commit a7933ac

Please sign in to comment.