[ 
https://issues.apache.org/jira/browse/FLINK-10632?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16668974#comment-16668974
 ] 

ASF GitHub Bot commented on FLINK-10632:
----------------------------------------

tillrohrmann commented on a change in pull request #6943: [FLINK-10632][e2e] 
Running general purpose testing job with failure in per-job mode
URL: https://github.com/apache/flink/pull/6943#discussion_r229353255
 
 

 ##########
 File path: 
flink-end-to-end-tests/test-scripts/test_ha_per_job_cluster_datastream.sh
 ##########
 @@ -0,0 +1,160 @@
+#!/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 "$(dirname "$0")"/common.sh
+source "$(dirname "$0")"/common_ha.sh
+
+TEST_PROGRAM_JAR_NAME=DataStreamAllroundTestProgram.jar
+TEST_PROGRAM_JAR=${END_TO_END_DIR}/flink-datastream-allround-test/target/${TEST_PROGRAM_JAR_NAME}
+FLINK_LIB_DIR=${FLINK_DIR}/lib
+JOB_ID="00000000000000000000000000000000"
+
+function ha_cleanup() {
+  # don't call ourselves again for another signal interruption
+  trap "exit -1" INT
+  # don't call ourselves again for normal exit
+  trap "" EXIT
+
+  stop_watchdogs
+  kill_all 'StandaloneJobClusterEntryPoint'
+  shutdown_all
+  rm ${FLINK_LIB_DIR}/${TEST_PROGRAM_JAR_NAME}
+}
+
+trap ha_cleanup INT
+trap ha_cleanup EXIT
+
+function run_job() {
+    local PARALLELISM=$1
+    local BACKEND=$2
+    local ASYNC=$3
+    local INCREM=$4
+
+    local CHECKPOINT_DIR="${TEST_DATA_DIR}/checkpoints/"
+
+    ${FLINK_DIR}/bin/standalone-job.sh start \
+        --job-classname 
org.apache.flink.streaming.tests.DataStreamAllroundTestProgram \
+        -p ${PARALLELISM} \
+        --environment.parallelism ${PARALLELISM} \
+        --test.semantics exactly-once \
+        --test.simulate_failure true \
+        --test.simulate_failure.num_records 200 \
+        --test.simulate_failure.num_checkpoints 1 \
+        --test.simulate_failure.max_failures 20 \
+        --state_backend ${BACKEND} \
+        --state_backend.checkpoint_directory "file://${CHECKPOINT_DIR}" \
+        --state_backend.file.async ${ASYNC} \
+        --state_backend.rocks.incremental ${INCREM} \
+        --sequence_generator_source.sleep_time 15 \
+        --sequence_generator_source.sleep_after_elements 1
+}
+
+function verify_logs_per_job() {
+    local OUTPUT=$FLINK_DIR/log/*.out
+    local JM_FAILURES=$1
+    local EXIT_CODE=0
+    local VERIFY_CHECKPOINTS=$2
+
+    # verify that we have no alerts
+    if ! [ `cat ${OUTPUT} | wc -l` -eq 0 ]; then
+        echo "FAILURE: Alerts found at the general purpose job."
+        EXIT_CODE=1
+    fi
+
+    # checks that all apart from the first JM recover the failed jobgraph.
+    if ! [ `grep -r --include '*standalonejob*.log' 'Found 0 checkpoints in 
ZooKeeper' "${FLINK_DIR}/log/" | cut -d ":" -f 1 | uniq | wc -l` -eq 1 ]; then
+        echo "FAILURE: A JM did not take over, but started new job."
+        EXIT_CODE=1
+    fi
+
+    if ! [ `grep -r --include '*standalonejob*.log' -P 'Found \d+ checkpoints 
in ZooKeeper' "${FLINK_DIR}/log/" | cut -d ":" -f 1 | uniq | wc -l` -eq 
$((JM_FAILURES + 1)) ]; then
+        echo "FAILURE: A JM did not take over."
+        EXIT_CODE=1
+    fi
+
+    if [ "$VERIFY_CHECKPOINTS" = true ]; then
+    # search the logs for JMs that log completed checkpoints
+        if ! [ `grep -r --include '*standalonejob*.log' 'Completed checkpoint' 
"${FLINK_DIR}/log/" | cut -d ":" -f 1 | uniq | wc -l` -eq $((JM_FAILURES + 1)) 
]; then
+            echo "FAILURE: A JM did not execute the job."
+            EXIT_CODE=1
+        fi
+    fi
+
+    if [[ $EXIT_CODE != 0 ]]; then
+        echo "One or more tests FAILED."
+        exit $EXIT_CODE
+    fi
+}
+
+function run_ha_test() {
+    local PARALLELISM=$1
+    local BACKEND=$2
+    local ASYNC=$3
+    local INCREM=$4
+
+    local JM_KILLS=3
+
+
+    CLEARED=0
+
+    # add job jar to cluster classpath
+    cp ${TEST_PROGRAM_JAR} ${FLINK_LIB_DIR}
+
+    # start the cluster on HA mode
+    create_ha_config
+    start_local_zk
+
+    echo "Running on HA mode: parallelism=${PARALLELISM}, backend=${BACKEND}, 
asyncSnapshots=${ASYNC}, and incremSnapshots=${INCREM}."
+
+    # submit a job in detached mode and let it run
+    run_job ${PARALLELISM} ${BACKEND} ${ASYNC} ${INCREM}
+
+    start_taskmanagers 1
+
+    wait_job_running ${JOB_ID}
+
+    # start the watchdog that keeps the number of JMs stable
+    start_ha_jm_watchdog 1 "StandaloneJobClusterEntryPoint" run_job 
${PARALLELISM} ${BACKEND} ${ASYNC} ${INCREM}
+
+    sleep 5
+
+    # start the watchdog that keeps the number of TMs stable
+    start_ha_tm_watchdog ${JOB_ID} 1
+
+    # let the job run for a while to take some checkpoints
+    sleep 20
+
+    for (( c=0; c<${JM_KILLS}; c++ )); do
+        # kill the JM and wait for watchdog to
+        # create a new one which will take over
+        kill_single 'StandaloneJobClusterEntryPoint'
+        # let the job start and take some checkpoints
+        sleep 60
 
 Review comment:
   Ideally we would wait until the next checkpoint has been successfully taken.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


> Run general purpose test job with failures in per-job mode
> ----------------------------------------------------------
>
>                 Key: FLINK-10632
>                 URL: https://issues.apache.org/jira/browse/FLINK-10632
>             Project: Flink
>          Issue Type: Sub-task
>          Components: E2E Tests
>    Affects Versions: 1.7.0
>            Reporter: Till Rohrmann
>            Assignee: Dawid Wysakowicz
>            Priority: Major
>              Labels: pull-request-available
>             Fix For: 1.7.0
>
>
> Similar to FLINK-8973, we should add an end-to-end which runs the general 
> datastream job with failures on a per-job cluster with HA enabled (either 
> directly the {{StandaloneJobClusterEntrypoint}} or a docker image based on 
> this entrypoint).
> We should kill the TMs as well as the cluster entrypoint and verify that the 
> job recovers.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to