You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by florianschmidt1994 <gi...@git.apache.org> on 2018/05/22 12:28:42 UTC

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

GitHub user florianschmidt1994 opened a pull request:

    https://github.com/apache/flink/pull/6053

    [FLINK-9257][E2E Tests] Fix wrong "All tests pass" message

    ## What is the purpose of the change
    Fix a wrongly printed "All tests PASS" message when they actually don't"
    
    Previously the behaviour was like this:
    >During the cleanup hook (trap cleanup EXIT in common.sh) it will be checked whether there are non-empty out files or log files with certain exceptions. If a tests fails with non-zero exit code, but does not have any exceptions or .out files, this will still print "All tests PASS" to stdout, even though they don't
    
    With this PR the whole test-runner is restructured so that
    1. The check for non-empty .out files, errors and exceptions in logs is triggered from the `run_test` method
    2. The error message after each test is dependant on both the exit code of the test script as well as the result from checking the log files
    3. cleanup is now triggered by the test runner, not by the individual tests anymore
    4. tests that signaled their failure by modifying `PASS`  now do so by exiting with non-zero exit code
    5. `check_result_hash` exits with 1 instead of modifying `PASS`
    
    Additionally this PR
    1. Reformats the output a little compared to previous tests
    ```
    Flink dist directory: /Users/florianschmidt/dev/flink/flink-dist/target/flink-1.6-SNAPSHOT-bin/flink-1.6-SNAPSHOT
    TEST_DATA_DIR: /Users/florianschmidt/dev/flink/flink-end-to-end-tests/temp-test-directory-00N
    Flink dist directory: /Users/florianschmidt/dev/flink/flink-dist/target/flink-1.6-SNAPSHOT-bin/flink-1.6-SNAPSHOT
    TEST_DATA_DIR: /Users/florianschmidt/dev/flink/flink-end-to-end-tests/temp-test-directory-00N
    flink-end-to-end-test directory: /Users/florianschmidt/dev/flink/flink-end-to-end-tests
    Flink distribution directory: /Users/florianschmidt/dev/flink/flink-dist/target/flink-1.6-SNAPSHOT-bin/flink-1.6-SNAPSHOT
    
    ==============================================================================
    Running 'Streaming Python Wordcount end-to-end test'
    ==============================================================================
    Flink dist directory: /Users/florianschmidt/dev/flink/flink-dist/target/flink-1.6-SNAPSHOT-bin/flink-1.6-SNAPSHOT
    TEST_DATA_DIR: /Users/florianschmidt/dev/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-00N
    Starting cluster.
    Starting standalonesession daemon on host Florians-MBP.fritz.box.
    Starting taskexecutor daemon on host Florians-MBP.fritz.box.
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Dispatcher REST endpoint is up.
    Starting execution of program
    Program execution finished
    Job with JobID 436dfd1f2a81ab4f818fc7fb9c395f0c has finished.
    Job Runtime: 7512 ms
    pass StreamingPythonWordCount
    Stopping taskexecutor daemon (pid: 9877) on host Florians-MBP.fritz.box.
    Stopping standalonesession daemon (pid: 9585) on host Florians-MBP.fritz.box.
    No zookeeper daemon to stop on host Florians-MBP.fritz.box.
    
    [PASS] 'Streaming Python Wordcount end-to-end test' passed after 0 minutes and 22 seconds! Test exited with exit code 0.
    
    
    ==============================================================================
    Running 'Wordcount end-to-end test'
    ==============================================================================
    Flink dist directory: /Users/florianschmidt/dev/flink/flink-dist/target/flink-1.6-SNAPSHOT-bin/flink-1.6-SNAPSHOT
    TEST_DATA_DIR: /Users/florianschmidt/dev/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-24N
    ```
    
    ## Verifying this change
    * I ran the test scripts manually and checked that they still behave as expected
    * I used the following script as a sample e2e-test to trigger different failure / success behaviours
    ```
    #!/usr/bin/env bash
    source "$(dirname "$0")"/common.sh
    
    # each of those can be used to cause a test to fail
    
    # echo "This should cause the test to fail" > $FLINK_DIR/log/test.out
    # check_result_hash "asf" "$FLINK_DIR/log/"
    # exit 1
    
    function test_cleanup {
        echo "Something"
    
        # Uncomment to see test fail in cleanup
        # exit 2
    }
    
    trap test_cleanup EXIT
    ```
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): no
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: no
      - The serializers: no
      - The runtime per-record code paths (performance sensitive): no
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: no
      - The S3 file system connector: no
    
    ## Documentation
    
      - Does this pull request introduce a new feature? no
      - If yes, how is the feature documented? not applicable


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/florianschmidt1994/flink flink-9257-fix-all-tests-pass-message

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/6053.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #6053
    
----
commit 2ca08c6ed0a4faea7e4094004d8d73aaa9140ac3
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T09:36:55Z

    Move error/exception/out checking to functions

commit 54c23a7858efad8d2b7dfda932ad5f50080bf95a
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T11:19:23Z

    Use changed test runner for nightly tests

commit 9cac304e4852b1c89ccf750446bda667768be2bb
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T11:22:03Z

    Use new test runner for pre-commit tests

commit c5108fee328e4b33558c46a6f52e03a2aa7a6dbc
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T11:35:17Z

    Use script exit codes where PASS="" has been used

commit ee2cfc489209baa8963adbdc05c55bf6b8f41aae
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T11:44:29Z

    Exit immediately on check_result_hash failure

commit acbbae63e11fceac74e1e691e967731235268a8c
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T11:45:02Z

    Add missing initialization for test_has_errors

commit 43a6244e34dcbdfa9727b49b0946b0f055510cf9
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T11:59:55Z

    Restructure test runner for nicer output

commit d4785eb23bf772cded6859f0545812a1be1e0991
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T12:03:38Z

    Remove sample test and unnecessary EXIT_CODE

commit 8f404f9f6dedbf58ab96fa515569ea50b83d79f7
Author: Florian Schmidt <fl...@...>
Date:   2018-05-22T12:11:41Z

    Add all tests passed message

----


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195079715
  
    --- Diff: flink-end-to-end-tests/test-scripts/test-runner-common.sh ---
    @@ -0,0 +1,76 @@
    +#!/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.
    +################################################################################
    +
    +source "${END_TO_END_DIR}"/test-scripts/common.sh
    +
    +#######################################
    +# Prints the given description, runs the given test and prints how long the execution took.
    +# Arguments:
    +#   $1: description of the test
    +#   $2: command to execute
    +#######################################
    +function run_test {
    +    description="$1"
    +    command="$2"
    +
    +    printf "\n==============================================================================\n"
    +    printf "Running '${description}'\n"
    +    printf "==============================================================================\n"
    +    start_timer
    +    ${command}
    +    exit_code="$?"
    +    time_elapsed=$(end_timer)
    +
    +    check_logs_for_errors
    +    check_logs_for_exceptions
    +    check_logs_for_non_empty_out_files
    +
    +    cleanup
    +
    +    if [[ ${exit_code} == 0 ]]; then
    +        if [[ ! "$PASS" ]]; then
    --- End diff --
    
    I thought we aren't using pass anymore?


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    I'll take a look tomorrow.


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    Oh you already rebased it in the mean-time. neat.


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r190125404
  
    --- Diff: flink-end-to-end-tests/test-scripts/test-runner-common.sh ---
    @@ -0,0 +1,76 @@
    +#!/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.
    +################################################################################
    +
    +source "$(pwd)"/test-scripts/common.sh
    +
    +#######################################
    +# Prints the given description, runs the given test and prints how long the execution took.
    +# Arguments:
    +#   $1: description of the test
    +#   $2: command to execute
    +#######################################
    +function run_test {
    +    description="$1"
    +    command="$2"
    +
    +    printf "\n==============================================================================\n"
    +    printf "Running '${description}'\n"
    +    printf "==============================================================================\n"
    +    start_timer
    +    ${command}
    +    exit_code="$?"
    +    time_elapsed=$(end_timer)
    +
    +    check_logs_for_errors
    +    check_logs_for_exceptions
    +    check_logs_for_non_empty_out_files
    --- End diff --
    
    Maybe we should move all these methods:
    ```
    start_timer
    end_timer
    check_logs_for_errors
    check_logs_for_exceptions
    check_logs_for_non_empty_out_files
    ```
    
    to `test-runner-common.sh` since that's the only place they are used anyways


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195079122
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -71,8 +69,14 @@ function run_local_recovery_test {
         local incremental=$4
         local kill_jvm=$5
     
    -    echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${kill_jvm}."
    -    TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    +    echo "Running local recovery test with configuration:
    +        parallelism: ${parallelism}
    +        max attempts: ${max_attempts}
    +        backend: ${backend}
    +        incremental checkpoints: ${incremental}
    +        kill JVM: ${kill_jvm}"
    +
    +    TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    --- End diff --
    
    well we already did export it there, but you removed it :P


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195079963
  
    --- Diff: flink-end-to-end-tests/test-scripts/test-runner-common.sh ---
    @@ -0,0 +1,76 @@
    +#!/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.
    +################################################################################
    +
    +source "${END_TO_END_DIR}"/test-scripts/common.sh
    +
    +#######################################
    +# Prints the given description, runs the given test and prints how long the execution took.
    +# Arguments:
    +#   $1: description of the test
    +#   $2: command to execute
    +#######################################
    +function run_test {
    +    description="$1"
    +    command="$2"
    +
    +    printf "\n==============================================================================\n"
    +    printf "Running '${description}'\n"
    +    printf "==============================================================================\n"
    +    start_timer
    +    ${command}
    +    exit_code="$?"
    +    time_elapsed=$(end_timer)
    +
    +    check_logs_for_errors
    +    check_logs_for_exceptions
    +    check_logs_for_non_empty_out_files
    +
    +    cleanup
    +
    +    if [[ ${exit_code} == 0 ]]; then
    --- End diff --
    
    Am I mistaken or didn't you already change this to `EXIT_CODE`? Did maybe something go wrong during the rebase?


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195076092
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -71,8 +69,14 @@ function run_local_recovery_test {
         local incremental=$4
         local kill_jvm=$5
     
    -    echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${kill_jvm}."
    -    TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    +    echo "Running local recovery test with configuration:
    +        parallelism: ${parallelism}
    +        max attempts: ${max_attempts}
    +        backend: ${backend}
    +        incremental checkpoints: ${incremental}
    +        kill JVM: ${kill_jvm}"
    +
    +    TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    --- End diff --
    
    Only works if we export END_TO_END_DIR in run_nightly/precommit_tests.sh, but I'll just add that there as well


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195036305
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -53,8 +53,6 @@ function cleanup_after_test {
         #
         kill ${watchdog_pid} 2> /dev/null
         wait ${watchdog_pid} 2> /dev/null
    -    #
    -    cleanup
    --- End diff --
    
    This should not be a concern anymore with the new changed where each configuration of `test_local_recovery_and_scheduling` is its own test-case


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r190156721
  
    --- Diff: flink-end-to-end-tests/run-nightly-tests.sh ---
    @@ -30,168 +30,50 @@ if [ -z "$FLINK_DIR" ] ; then
         exit 1
     fi
     
    -source "$(dirname "$0")"/test-scripts/common.sh
    +source "$(dirname "$0")"/test-scripts/test-runner-common.sh
     
     FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized
     
     echo "flink-end-to-end-test directory: $END_TO_END_DIR"
     echo "Flink distribution directory: $FLINK_DIR"
     
    -EXIT_CODE=0
    -
     # Template for adding a test:
     
    -# if [ $EXIT_CODE == 0 ]; then
    -#    run_test "<description>" "$END_TO_END_DIR/test-scripts/<script_name>"
    -#    EXIT_CODE=$?
    -# fi
    -
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file false false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v1.7.1) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v2.3.5) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v5.1.2) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Local recovery and sticky scheduling nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Quickstarts nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -# Exit code for Travis build success/failure
    -exit $EXIT_CODE
    +# run_test "<description>" "$END_TO_END_DIR/test-scripts/<script_name>"
    +
    +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file true false"
    +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file false false"
    +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true false"
    +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true true"
    +
    +run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true"
    +run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false"
    +run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true"
    +run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false"
    +run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true"
    +run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false"
    +run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks"
    +run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks"
    +run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks"
    +
    +run_test "Resuming Externalized Checkpoint (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true"
    +run_test "Resuming Externalized Checkpoint (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false"
    +run_test "Resuming Externalized Checkpoint (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks"
    +
    +run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true true"
    +run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false true"
    +run_test "Resuming Externalized Checkpoint after terminal failure (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks true"
    +
    +run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
    +run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh"
    +run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh"
    +run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh"
    --- End diff --
    
    +1 to having each execution as a separate test


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    Looks good to me, let's see what travis says.


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    Thanks @zentol and @tzulitai for the review. I addressed your concerns in the lastest couple of commits


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/flink/pull/6053


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195080229
  
    --- Diff: flink-end-to-end-tests/test-scripts/test-runner-common.sh ---
    @@ -0,0 +1,76 @@
    +#!/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.
    +################################################################################
    +
    +source "${END_TO_END_DIR}"/test-scripts/common.sh
    +
    +#######################################
    +# Prints the given description, runs the given test and prints how long the execution took.
    +# Arguments:
    +#   $1: description of the test
    +#   $2: command to execute
    +#######################################
    +function run_test {
    +    description="$1"
    +    command="$2"
    +
    +    printf "\n==============================================================================\n"
    +    printf "Running '${description}'\n"
    +    printf "==============================================================================\n"
    +    start_timer
    +    ${command}
    +    exit_code="$?"
    +    time_elapsed=$(end_timer)
    +
    +    check_logs_for_errors
    +    check_logs_for_exceptions
    +    check_logs_for_non_empty_out_files
    +
    +    cleanup
    +
    +    if [[ ${exit_code} == 0 ]]; then
    --- End diff --
    
    Oh damn. Something might have gone wrong here... 


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    @zentol Looks like travis likes it 🙂


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195060959
  
    --- Diff: flink-end-to-end-tests/run-nightly-tests.sh ---
    @@ -30,168 +30,57 @@ if [ -z "$FLINK_DIR" ] ; then
         exit 1
     fi
     
    -source "$(dirname "$0")"/test-scripts/common.sh
    +source "$(dirname "$0")"/test-scripts/test-runner-common.sh
    --- End diff --
    
    use `END_TO_END_DIR` instead


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r190124962
  
    --- Diff: flink-end-to-end-tests/run-pre-commit-tests.sh ---
    @@ -31,56 +31,23 @@ if [ -z "$FLINK_DIR" ] ; then
     fi
     
     source "$(dirname "$0")"/test-scripts/common.sh
    --- End diff --
    
    I think this can now be removed?


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    merging.


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r192139268
  
    --- Diff: flink-end-to-end-tests/test-scripts/test-runner-common.sh ---
    @@ -0,0 +1,76 @@
    +#!/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.
    +################################################################################
    +
    +source "$(pwd)"/test-scripts/common.sh
    +
    +#######################################
    +# Prints the given description, runs the given test and prints how long the execution took.
    +# Arguments:
    +#   $1: description of the test
    +#   $2: command to execute
    +#######################################
    +function run_test {
    +    description="$1"
    +    command="$2"
    +
    +    printf "\n==============================================================================\n"
    +    printf "Running '${description}'\n"
    +    printf "==============================================================================\n"
    +    start_timer
    +    ${command}
    +    exit_code="$?"
    +    time_elapsed=$(end_timer)
    +
    +    check_logs_for_errors
    +    check_logs_for_exceptions
    +    check_logs_for_non_empty_out_files
    --- End diff --
    
    We have been discussing about changing the semantics at some point to leave it up to each individual test case to check the logs for errors and drop it from the test runner, maybe even with a whitelist / blacklist approach of expected exceptions. If we want to go that way I'd say leave it in common.sh
    We could also say we're probably gonna stick with the current approach for a while, then I'd say let's move them to test-runner-common.sh


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195071307
  
    --- Diff: flink-end-to-end-tests/run-nightly-tests.sh ---
    @@ -25,180 +25,65 @@ if [ -z "$END_TO_END_DIR" ] ; then
         exit 1  # fail
     fi
     
    -export END_TO_END_DIR="${END_TO_END_DIR}"
    -
     if [ -z "$FLINK_DIR" ] ; then
         echo "You have to export the Flink distribution directory as FLINK_DIR"
         exit 1
     fi
     
    -source "$(dirname "$0")"/test-scripts/common.sh
    +source "$(dirname "$0")"/test-scripts/test-runner-common.sh
     
     FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized
     
     echo "flink-end-to-end-test directory: $END_TO_END_DIR"
     echo "Flink distribution directory: $FLINK_DIR"
     
    -EXIT_CODE=0
    -
     # Template for adding a test:
     
    -# if [ $EXIT_CODE == 0 ]; then
    -#    run_test "<description>" "$END_TO_END_DIR/test-scripts/<script_name>"
    -#    EXIT_CODE=$?
    -# fi
    -
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file false false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v1.7.1) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v2.3.5) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v5.1.2) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Local recovery and sticky scheduling nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Quickstarts Java nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh java"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Quickstarts Scala nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh scala"
    -  EXIT_CODE=$?
    -fi
    -
    -# Exit code for Travis build success/failure
    -exit $EXIT_CODE
    +# run_test "<description>" "$END_TO_END_DIR/test-scripts/<script_name>"
    +
    +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file true false"
    +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file false false"
    +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true false"
    +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true true"
    +
    +run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true"
    +run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false"
    +run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true"
    +run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false"
    +run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true"
    +run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false"
    +run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks"
    +run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks"
    +run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks"
    +
    +run_test "Resuming Externalized Checkpoint (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true"
    +run_test "Resuming Externalized Checkpoint (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false"
    +run_test "Resuming Externalized Checkpoint (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks"
    +
    +run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true true"
    +run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false true"
    +run_test "Resuming Externalized Checkpoint after terminal failure (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks true"
    +
    +run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
    +run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh"
    +run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh"
    +run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4"
    +
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false false"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 3 file false true"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false false"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true false"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks false true"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh 4 10 rocks true true"
    +
    +run_test "Quickstarts nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh"
    --- End diff --
    
    this shouldn't be here


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195081885
  
    --- Diff: flink-end-to-end-tests/test-scripts/test-runner-common.sh ---
    @@ -0,0 +1,76 @@
    +#!/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.
    +################################################################################
    +
    +source "${END_TO_END_DIR}"/test-scripts/common.sh
    +
    +#######################################
    +# Prints the given description, runs the given test and prints how long the execution took.
    +# Arguments:
    +#   $1: description of the test
    +#   $2: command to execute
    +#######################################
    +function run_test {
    +    description="$1"
    +    command="$2"
    +
    +    printf "\n==============================================================================\n"
    +    printf "Running '${description}'\n"
    +    printf "==============================================================================\n"
    +    start_timer
    +    ${command}
    +    exit_code="$?"
    +    time_elapsed=$(end_timer)
    +
    +    check_logs_for_errors
    +    check_logs_for_exceptions
    +    check_logs_for_non_empty_out_files
    +
    +    cleanup
    +
    +    if [[ ${exit_code} == 0 ]]; then
    --- End diff --
    
    Oh that is still in there b.c. I left the `check_logs_for_non_empty_out_files` etc. untouched, which again use the `PASS` thing to signal whether or not a test case should fail. 
    I'll go ahead and change this to our new convention as well.
    At least it's nothing went wrong during the rebase 😅


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r190126071
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -53,8 +53,6 @@ function cleanup_after_test {
         #
         kill ${watchdog_pid} 2> /dev/null
         wait ${watchdog_pid} 2> /dev/null
    -    #
    -    cleanup
    --- End diff --
    
    The `test_local_recovery_and_scheduling` test currently bundles several executions of the test (e.g. with different state backend configurations) in a single run of the test script. That's why it required this cleanup within the test itself.
    
    How would the change of this PR affect this?
    In general, should we also restructure e2e tests so that each execution configuration variant should be executed with the `test-runner-cleanup#run_test` method (instead of cleaning up itself in-between executions)?
    
    AFAIK, only the `test_local_recovery_and_scheduling` does this at the moment.


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    ```
    ==============================================================================
    Running 'Streaming Python Wordcount end-to-end test'
    ==============================================================================
    Flink dist directory: /home/travis/build/apache/flink/build-target
    TEST_DATA_DIR: /home/travis/build/apache/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-10412258255
    Starting cluster.
    Starting standalonesession daemon on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    Starting taskexecutor daemon on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Dispatcher REST endpoint is up.
    Starting execution of program
    Program execution finished
    Job with JobID 06184a085272dd12b3573b1bcb96badc has finished.
    Job Runtime: 6103 ms
    pass StreamingPythonWordCount
    Stopping taskexecutor daemon (pid: 31303) on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    Stopping standalonesession daemon (pid: 30988) on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    
    [PASS] 'Streaming Python Wordcount end-to-end test' passed after 0 minutes and 24 seconds! Test exited with exit code 0.
    
    
    ==============================================================================
    Running 'Wordcount end-to-end test'
    ==============================================================================
    Flink dist directory: /home/travis/build/apache/flink/build-target
    TEST_DATA_DIR: /home/travis/build/apache/flink/flink-end-to-end-tests/test-scripts/temp-test-directory-36174383269
    Starting cluster.
    Starting standalonesession daemon on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    Starting taskexecutor daemon on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Waiting for dispatcher REST endpoint to come up...
    Dispatcher REST endpoint is up.
    Starting execution of program
    Program execution finished
    Job with JobID 30256ad7ff23ea8543ddca76bacaaee5 has finished.
    Job Runtime: 1352 ms
    pass WordCount
    Stopping taskexecutor daemon (pid: 835) on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    Stopping standalonesession daemon (pid: 517) on host travis-job-363a754a-fe5f-4873-bbe8-fe4064b95bc8.
    
    [PASS] 'Wordcount end-to-end test' passed after 0 minutes and 11 seconds! Test exited with exit code 0.
    ```
    
    Travis logs excerpt looks good. Follow up commits looks good.
    +1, LGTM on my side.


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by tzulitai <gi...@git.apache.org>.
Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r190126277
  
    --- Diff: flink-end-to-end-tests/run-nightly-tests.sh ---
    @@ -30,168 +30,50 @@ if [ -z "$FLINK_DIR" ] ; then
         exit 1
     fi
     
    -source "$(dirname "$0")"/test-scripts/common.sh
    +source "$(dirname "$0")"/test-scripts/test-runner-common.sh
     
     FLINK_DIR="`( cd \"$FLINK_DIR\" && pwd )`" # absolutized and normalized
     
     echo "flink-end-to-end-test directory: $END_TO_END_DIR"
     echo "Flink distribution directory: $FLINK_DIR"
     
    -EXIT_CODE=0
    -
     # Template for adding a test:
     
    -# if [ $EXIT_CODE == 0 ]; then
    -#    run_test "<description>" "$END_TO_END_DIR/test-scripts/<script_name>"
    -#    EXIT_CODE=$?
    -# fi
    -
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file false false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks false"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Resuming Externalized Checkpoint after terminal failure (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks true"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v1.7.1) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 1 https://download.elastic.co/elasticsearch/elasticsearch/elasticsearch-1.7.1.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v2.3.5) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 2 https://download.elastic.co/elasticsearch/release/org/elasticsearch/distribution/tar/elasticsearch/2.3.5/elasticsearch-2.3.5.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test \
    -    "Elasticsearch (v5.1.2) sink end-to-end test" \
    -    "$END_TO_END_DIR/test-scripts/test_streaming_elasticsearch.sh 5 https://artifacts.elastic.co/downloads/elasticsearch/elasticsearch-5.1.2.tar.gz"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Local recovery and sticky scheduling nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -if [ $EXIT_CODE == 0 ]; then
    -  run_test "Quickstarts nightly end-to-end test" "$END_TO_END_DIR/test-scripts/test_quickstarts.sh"
    -  EXIT_CODE=$?
    -fi
    -
    -# Exit code for Travis build success/failure
    -exit $EXIT_CODE
    +# run_test "<description>" "$END_TO_END_DIR/test-scripts/<script_name>"
    +
    +run_test "Running HA (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file true false"
    +run_test "Running HA (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh file false false"
    +run_test "Running HA (rocks, non-incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true false"
    +run_test "Running HA (rocks, incremental) end-to-end test" "$END_TO_END_DIR/test-scripts/test_ha.sh rocks true true"
    +
    +run_test "Resuming Savepoint (file, async, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file true"
    +run_test "Resuming Savepoint (file, sync, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 file false"
    +run_test "Resuming Savepoint (file, async, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file true"
    +run_test "Resuming Savepoint (file, sync, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 file false"
    +run_test "Resuming Savepoint (file, async, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file true"
    +run_test "Resuming Savepoint (file, sync, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 file false"
    +run_test "Resuming Savepoint (rocks, no parallelism change) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 2 rocks"
    +run_test "Resuming Savepoint (rocks, scale up) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 2 4 rocks"
    +run_test "Resuming Savepoint (rocks, scale down) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_savepoint.sh 4 2 rocks"
    +
    +run_test "Resuming Externalized Checkpoint (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true"
    +run_test "Resuming Externalized Checkpoint (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false"
    +run_test "Resuming Externalized Checkpoint (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks"
    +
    +run_test "Resuming Externalized Checkpoint after terminal failure (file, async) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file true true"
    +run_test "Resuming Externalized Checkpoint after terminal failure (file, sync) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh file false true"
    +run_test "Resuming Externalized Checkpoint after terminal failure (rocks) end-to-end test" "$END_TO_END_DIR/test-scripts/test_resume_externalized_checkpoints.sh rocks true"
    +
    +run_test "DataSet allround end-to-end test" "$END_TO_END_DIR/test-scripts/test_batch_allround.sh"
    +run_test "Streaming SQL end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_sql.sh"
    +run_test "Streaming bucketing end-to-end test" "$END_TO_END_DIR/test-scripts/test_streaming_bucketing.sh"
    +run_test "Stateful stream job upgrade end-to-end test" "$END_TO_END_DIR/test-scripts/test_stateful_stream_job_upgrade.sh 2 4"
    +run_test "Local recovery and sticky scheduling end-to-end test" "$END_TO_END_DIR/test-scripts/test_local_recovery_and_scheduling.sh"
    --- End diff --
    
    This test currently performs log verifications and cleanups within a single execution of the test script, since it specifies multiple executions with different state backend configurations.
    
    Should we break this up, so that each configuration variant is explicitly executed by the `run_test` method (like what we currently do with the savepoint / externalized checkpoint tests)


---

[GitHub] flink pull request #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass"...

Posted by zentol <gi...@git.apache.org>.
Github user zentol commented on a diff in the pull request:

    https://github.com/apache/flink/pull/6053#discussion_r195071781
  
    --- Diff: flink-end-to-end-tests/test-scripts/test_local_recovery_and_scheduling.sh ---
    @@ -71,8 +69,14 @@ function run_local_recovery_test {
         local incremental=$4
         local kill_jvm=$5
     
    -    echo "Running local recovery test on ${backend} backend: incremental checkpoints = ${incremental}, kill JVM = ${kill_jvm}."
    -    TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    +    echo "Running local recovery test with configuration:
    +        parallelism: ${parallelism}
    +        max attempts: ${max_attempts}
    +        backend: ${backend}
    +        incremental checkpoints: ${incremental}
    +        kill JVM: ${kill_jvm}"
    +
    +    TEST_PROGRAM_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-local-recovery-and-allocation-test/target/StickyAllocationAndLocalRecoveryTestJob.jar
    --- End diff --
    
    use `END_TO_END_DIR` instead


---

[GitHub] flink issue #6053: [FLINK-9257][E2E Tests] Fix wrong "All tests pass" messag...

Posted by florianschmidt1994 <gi...@git.apache.org>.
Github user florianschmidt1994 commented on the issue:

    https://github.com/apache/flink/pull/6053
  
    @zentol Maybe you could have a look at this?


---