You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/11/05 10:07:24 UTC

[GitHub] dawidwys closed pull request #6965: [FLINK-10368][e2e] Hardened kerberized yarn e2e test

dawidwys closed pull request #6965: [FLINK-10368][e2e] Hardened kerberized yarn e2e test
URL: https://github.com/apache/flink/pull/6965
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh
index 7b5e50ba439..5b98b96e51d 100755
--- a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh
+++ b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/bootstrap.sh
@@ -124,6 +124,7 @@ elif [ "$1" == "master" ]; then
     hdfs dfs -chown hadoop-user:hadoop-user /user/hadoop-user
 
     kdestroy
+    echo "Finished master initialization"
 
     while true; do sleep 1000; done
 elif [ "$1" == "worker" ]; then
diff --git a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml
index 9b17acc1656..c7736a69489 100644
--- a/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml
+++ b/flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/config/yarn-site.xml
@@ -21,6 +21,11 @@ under the License.
         <value>mapreduce_shuffle</value>
     </property>
 
+	<property>
+		<name>yarn.nodemanager.vmem-pmem-ratio</name>
+		<value>3</value>
+	</property>
+
     <!-- this is ignored by the default scheduler but we have to set it because Flink would
     complain if we didn't have it -->
     <property>
@@ -33,12 +38,12 @@ under the License.
     are scheduled on one NM, which wouldn't provoke a previously fixed Kerberos keytab bug. -->
     <property>
         <name>yarn.nodemanager.resource.memory-mb</name>
-        <value>4100</value>
+        <value>2500</value>
     </property>
 
     <property>
         <name>yarn.scheduler.minimum-allocation-mb</name>
-        <value>2000</value>
+        <value>1000</value>
     </property>
 
     <property>
diff --git a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh
index c9ef15d3dc5..5f2dea2ea6a 100755
--- a/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh
+++ b/flink-end-to-end-tests/test-scripts/test_yarn_kerberos_docker.sh
@@ -24,7 +24,8 @@ FLINK_TARBALL_DIR=$TEST_DATA_DIR
 FLINK_TARBALL=flink.tar.gz
 FLINK_DIRNAME=$(basename $FLINK_DIR)
 
-MAX_RETRY_SECONDS=800
+MAX_RETRY_SECONDS=120
+CLUSTER_SETUP_RETRIES=3
 
 echo "Flink Tarball directory $FLINK_TARBALL_DIR"
 echo "Flink tarball filename $FLINK_TARBALL"
@@ -33,20 +34,6 @@ echo "End-to-end directory $END_TO_END_DIR"
 docker --version
 docker-compose --version
 
-mkdir -p $FLINK_TARBALL_DIR
-tar czf $FLINK_TARBALL_DIR/$FLINK_TARBALL -C $(dirname $FLINK_DIR) .
-
-echo "Building Hadoop Docker container"
-until docker build --build-arg HADOOP_VERSION=2.8.4 -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/Dockerfile -t flink/docker-hadoop-secure-cluster:latest $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/; do
-    # with all the downloading and ubuntu updating a lot of flakiness can happen, make sure
-    # we don't immediately fail
-    echo "Something went wrong while building the Docker image, retrying ..."
-    sleep 2
-done
-
-echo "Starting Hadoop cluster"
-docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml up -d
-
 # make sure we stop our cluster at the end
 function cluster_shutdown {
   # don't call ourselves again for another signal interruption
@@ -60,12 +47,71 @@ function cluster_shutdown {
 trap cluster_shutdown INT
 trap cluster_shutdown EXIT
 
-until docker cp $FLINK_TARBALL_DIR/$FLINK_TARBALL master:/home/hadoop-user/; do
-    # we're retrying this one because we don't know yet if the container is ready
-    echo "Uploading Flink tarball to docker master failed, retrying ..."
-    sleep 5
+function start_hadoop_cluster() {
+    echo "Starting Hadoop cluster"
+    docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml up -d
+
+    # wait for kerberos to be set up
+    start_time=$(date +%s)
+    until docker logs master 2>&1 | grep -q "Finished master initialization"; do
+        current_time=$(date +%s)
+        time_diff=$((current_time - start_time))
+
+        if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then
+            return 1
+        else
+            echo "Waiting for hadoop cluster to come up. We have been trying for $time_diff seconds, retrying ..."
+            sleep 10
+        fi
+    done
+
+    # perform health checks
+    if ! { [ $(docker inspect -f '{{.State.Running}}' master 2>&1) = 'true' ] &&
+           [ $(docker inspect -f '{{.State.Running}}' slave1 2>&1) = 'true' ] &&
+           [ $(docker inspect -f '{{.State.Running}}' slave2 2>&1) = 'true' ] &&
+           [ $(docker inspect -f '{{.State.Running}}' kdc 2>&1) = 'true' ]; };
+    then
+        return 1
+    fi
+
+    return 0
+}
+
+mkdir -p $FLINK_TARBALL_DIR
+tar czf $FLINK_TARBALL_DIR/$FLINK_TARBALL -C $(dirname $FLINK_DIR) .
+
+echo "Building Hadoop Docker container"
+until docker build --build-arg HADOOP_VERSION=2.8.4 \
+    -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/Dockerfile \
+    -t flink/docker-hadoop-secure-cluster:latest \
+    $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/;
+do
+    # with all the downloading and ubuntu updating a lot of flakiness can happen, make sure
+    # we don't immediately fail
+    echo "Something went wrong while building the Docker image, retrying ..."
+    sleep 2
+done
+
+CLUSTER_STARTED=1
+for (( i = 0; i < $CLUSTER_SETUP_RETRIES; i++ ))
+do
+    if start_hadoop_cluster; then
+       echo "Cluster started successfully."
+       CLUSTER_STARTED=0
+       break #continue test, cluster set up succeeded
+    fi
+
+    echo "ERROR: Could not start hadoop cluster. Retrying..."
+    docker-compose -f $END_TO_END_DIR/test-scripts/docker-hadoop-secure-cluster/docker-compose.yml down
 done
 
+if [[ ${CLUSTER_STARTED} -ne 0 ]]; then
+    echo "ERROR: Could not start hadoop cluster. Aborting..."
+    exit 1
+fi
+
+docker cp $FLINK_TARBALL_DIR/$FLINK_TARBALL master:/home/hadoop-user/
+
 # now, at least the container is ready
 docker exec -it master bash -c "tar xzf /home/hadoop-user/$FLINK_TARBALL --directory /home/hadoop-user/"
 
@@ -73,6 +119,7 @@ docker exec -it master bash -c "tar xzf /home/hadoop-user/$FLINK_TARBALL --direc
 docker exec -it master bash -c "echo \"security.kerberos.login.keytab: /home/hadoop-user/hadoop-user.keytab\" > /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml"
 docker exec -it master bash -c "echo \"security.kerberos.login.principal: hadoop-user\" >> /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml"
 docker exec -it master bash -c "echo \"slot.request.timeout: 60000\" >> /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml"
+docker exec -it master bash -c "echo \"containerized.heap-cutoff-min: 100\" >> /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml"
 
 echo "Flink config:"
 docker exec -it master bash -c "cat /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml"
@@ -84,33 +131,28 @@ OUTPUT_PATH=hdfs:///user/hadoop-user/wc-out-$RANDOM
 start_time=$(date +%s)
 # it's important to run this with higher parallelism, otherwise we might risk that
 # JM and TM are on the same YARN node and that we therefore don't test the keytab shipping
-until docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -yn 3 -ys 1 -ytm 2000 -yjm 2000 -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH"; do
-    current_time=$(date +%s)
-	time_diff=$((current_time - start_time))
-
-    if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then
-        echo "We tried running the job for $time_diff seconds, max is $MAX_RETRY_SECONDS seconds, aborting"
-        mkdir -p $TEST_DATA_DIR/logs
-        echo "Hadoop logs:"
-        docker cp master:/var/log/hadoop/* $TEST_DATA_DIR/logs/
-        for f in $TEST_DATA_DIR/logs/*; do
-            echo "$f:"
-            cat $f
-        done
-        echo "Docker logs:"
-        docker logs master
-        exit 1
-    else
-        echo "Running the Flink job failed, might be that the cluster is not ready yet. We have been trying for $time_diff seconds, retrying ..."
-        sleep 5
-    fi
-done
-
-docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user"
-docker exec -it master bash -c "hdfs dfs -ls $OUTPUT_PATH"
-OUTPUT=$(docker exec -it master bash -c "hdfs dfs -cat $OUTPUT_PATH/*")
-docker exec -it master bash -c "kdestroy"
-echo "$OUTPUT"
+if docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \
+   /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -yn 3 -ys 1 -ytm 1000 -yjm 1000 \
+   -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH";
+then
+    docker exec -it master bash -c "kinit -kt /home/hadoop-user/hadoop-user.keytab hadoop-user"
+    docker exec -it master bash -c "hdfs dfs -ls $OUTPUT_PATH"
+    OUTPUT=$(docker exec -it master bash -c "hdfs dfs -cat $OUTPUT_PATH/*")
+    docker exec -it master bash -c "kdestroy"
+    echo "$OUTPUT"
+else
+    echo "Running the job failed."
+    mkdir -p $TEST_DATA_DIR/logs
+    echo "Hadoop logs:"
+    docker cp master:/var/log/hadoop/* $TEST_DATA_DIR/logs/
+    for f in $TEST_DATA_DIR/logs/*; do
+        echo "$f:"
+        cat $f
+    done
+    echo "Docker logs:"
+    docker logs master
+    exit 1
+fi
 
 if [[ ! "$OUTPUT" =~ "consummation,1" ]]; then
     echo "Output does not contain (consummation, 1) as required"
@@ -139,7 +181,10 @@ fi
 echo "Running Job without configured keytab, the exception you see below is expected"
 docker exec -it master bash -c "echo \"\" > /home/hadoop-user/$FLINK_DIRNAME/conf/flink-conf.yaml"
 # verify that it doesn't work if we don't configure a keytab
-OUTPUT=$(docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && /home/hadoop-user/$FLINK_DIRNAME/bin/flink run -m yarn-cluster -yn 3 -ys 1 -ytm 1200 -yjm 800 -p 3 /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH")
+OUTPUT=$(docker exec -it master bash -c "export HADOOP_CLASSPATH=\`hadoop classpath\` && \
+    /home/hadoop-user/$FLINK_DIRNAME/bin/flink run \
+    -m yarn-cluster -yn 3 -ys 1 -ytm 1000 -yjm 1000 -p 3 \
+    /home/hadoop-user/$FLINK_DIRNAME/examples/streaming/WordCount.jar --output $OUTPUT_PATH")
 echo "$OUTPUT"
 
 if [[ ! "$OUTPUT" =~ "Hadoop security with Kerberos is enabled but the login user does not have Kerberos credentials" ]]; then


 

----------------------------------------------------------------
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:
users@infra.apache.org


With regards,
Apache Git Services