[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-24 Thread walterddr
Github user walterddr commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r190788353
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,196 @@
+#!/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
+
+start_cluster
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR="\"pact\": \"(Data Source)\""
+JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR="\"pact\": \"(Data Sink)\""
+JOB_LIST_REGEX_EXTRACTOR_BY_STATUS="([0-9,a-f]*) :"
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function extract_valid_pact_from_job_info_return() {
+PACT_MATCH=0
+if [[ $1 =~ $JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR ]];
+then
+PACT_MATCH=$PACT_MATCH
+else
+PACT_MATCH=-1
+fi
+if [[ $1 =~ $JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR ]];
+then
+PACT_MATCH=$PACT_MATCH
+else
+PACT_MATCH=-1
+fi
+echo ${PACT_MATCH}
+}
+
+function extract_valid_job_list_by_type_from_job_list_return() {
+JOB_LIST_MATCH=0
+JOB_LIST_REGEX_EXTRACTOR="$JOB_LIST_REGEX_EXTRACTOR_BY_STATUS $2 $3"
+if [[ $1 =~ $JOB_LIST_REGEX_EXTRACTOR ]];
+then
+JOB_LIST_MATCH=$JOB_LIST_MATCH
+else
+JOB_LIST_MATCH=-1
+fi
+echo ${JOB_LIST_MATCH}
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
--- End diff --

checking the clean up code seems like it doesn't explicitly call stopping 
all tm (similar to `start-cluster.sh`, I need to explicitly call `taskmanager 
start`). I should remove only the `stop-cluster` actually.


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-23 Thread walterddr
Github user walterddr commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r190433925
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
--- End diff --

good point. I will add the check 👍 


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-23 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r190146869
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,196 @@
+#!/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
+
+start_cluster
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR="\"pact\": \"(Data Source)\""
+JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR="\"pact\": \"(Data Sink)\""
+JOB_LIST_REGEX_EXTRACTOR_BY_STATUS="([0-9,a-f]*) :"
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function extract_valid_pact_from_job_info_return() {
+PACT_MATCH=0
+if [[ $1 =~ $JOB_INFO_PACT_DATA_SOURCE_REGEX_EXTRACTOR ]];
+then
+PACT_MATCH=$PACT_MATCH
+else
+PACT_MATCH=-1
+fi
+if [[ $1 =~ $JOB_INFO_PACT_DATA_SINK_REGEX_EXTRACTOR ]];
+then
+PACT_MATCH=$PACT_MATCH
+else
+PACT_MATCH=-1
+fi
+echo ${PACT_MATCH}
+}
+
+function extract_valid_job_list_by_type_from_job_list_return() {
+JOB_LIST_MATCH=0
+JOB_LIST_REGEX_EXTRACTOR="$JOB_LIST_REGEX_EXTRACTOR_BY_STATUS $2 $3"
+if [[ $1 =~ $JOB_LIST_REGEX_EXTRACTOR ]];
+then
+JOB_LIST_MATCH=$JOB_LIST_MATCH
+else
+JOB_LIST_MATCH=-1
+fi
+echo ${JOB_LIST_MATCH}
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
--- End diff --

I don't think we need to explicitly shutdown the cluster and TMs here; that 
is already part of the `cleanup` call


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-23 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r190145819
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
--- End diff --

Well, we can have a completely normal exit code from the `run` execution, 
but the `-p` option completely ignored if we change the CLI to simply not 
recognize the option.

This is an extreme case, though.


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-21 Thread walterddr
Github user walterddr commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r189673647
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
--- End diff --

Changed to use `-p 1`. I think the part that "parallelism should be taken 
by the CLI command" is verified by the exit code from the `run` execution. is 
that correct?


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-18 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r189185140
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
--- End diff --

Since this is a detached execution, we probably want to wait until this job 
completes before continuing?


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-18 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r189185253
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
+  -c org.apache.flink.examples.java.wordcount.WordCount \
+  $FLINK_DIR/examples/batch/WordCount.jar \
+  --input file:///$FLINK_DIR/README.txt \
+  --output file:///${TEST_DATA_DIR}/out/result"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test information APIs\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink info 
$FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink list"
--- End diff --

Should we verify the output of `list`?


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-18 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r189185689
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
+  -c org.apache.flink.examples.java.wordcount.WordCount \
+  $FLINK_DIR/examples/batch/WordCount.jar \
+  --input file:///$FLINK_DIR/README.txt \
+  --output file:///${TEST_DATA_DIR}/out/result"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test information APIs\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink info 
$FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink list"
+EXIT_CODE=$?
+fi
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink list -s"
+EXIT_CODE=$?
+fi
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink list -r"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test operation on running streaming jobs\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+RETURN=`$FLINK_DIR/bin/flink run -d \
+$PERIODIC_JOB_JAR --outputPath file:///${TEST_DATA_DIR}/out/result`
+echo "job submission returns: $RETURN"
+JOB_ID=`extract_job_id_from_job_submission_return "$RETURN"`
+eval "$FLINK_DIR/bin/flink cancel ${JOB_ID}"
+EXIT_CODE=$?
+fi
+
+printf 

[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-18 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r189185232
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
+  -c org.apache.flink.examples.java.wordcount.WordCount \
+  $FLINK_DIR/examples/batch/WordCount.jar \
+  --input file:///$FLINK_DIR/README.txt \
+  --output file:///${TEST_DATA_DIR}/out/result"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test information APIs\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink info 
$FLINK_DIR/examples/batch/WordCount.jar"
--- End diff --

Should we verify the output of `info`?


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-05-18 Thread tzulitai
Github user tzulitai commented on a diff in the pull request:

https://github.com/apache/flink/pull/5863#discussion_r189185047
  
--- Diff: flink-end-to-end-tests/test-scripts/test_cli_api.sh ---
@@ -0,0 +1,155 @@
+#!/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
+
+start_cluster
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+$FLINK_DIR/bin/taskmanager.sh start
+
+# Test for CLI commands.
+# verify only the return code the content correctness of the API results.

+PERIODIC_JOB_JAR=$TEST_INFRA_DIR/../../flink-end-to-end-tests/flink-api-test/target/PeriodicStreamingJob.jar
+JOB_ID_REGEX_EXTRACTOR=".*JobID ([0-9,a-f]*)"
+SAVE_POINT_REGEX_EXTRACTOR=".*Savepoint stored in (.*)\\."
+
+EXIT_CODE=0
+
+function extract_job_id_from_job_submission_return() {
+if [[ $1 =~ $JOB_ID_REGEX_EXTRACTOR ]];
+then
+JOB_ID="${BASH_REMATCH[1]}";
+else
+JOB_ID=""
+fi
+echo "$JOB_ID"
+}
+
+function extract_savepoint_path_from_savepoint_return() {
+if [[ $1 =~ $SAVE_POINT_REGEX_EXTRACTOR ]];
+then
+SAVEPOINT_PATH="${BASH_REMATCH[1]}";
+else
+SAVEPOINT_PATH=""
+fi
+echo "$SAVEPOINT_PATH"
+}
+
+function cleanup_cli_test() {
+  stop_cluster
+  $FLINK_DIR/bin/taskmanager.sh stop-all
+
+  cleanup
+}
+
+printf 
"\n==\n"
+printf "Test default job launch with non-detach mode\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run $FLINK_DIR/examples/batch/WordCount.jar"
+EXIT_CODE=$?
+fi
+
+printf 
"\n==\n"
+printf "Test run with complex parameter set\n"
+printf 
"==\n"
+if [ $EXIT_CODE == 0 ]; then
+eval "$FLINK_DIR/bin/flink run -m localhost:8081 -p 4 -q -d \
--- End diff --

There probably should be some verification that the job actually runs with 
DOP=4


---


[GitHub] flink pull request #5863: [FLINK-8985][e2etest] initial support for End-to-e...

2018-04-17 Thread walterddr
GitHub user walterddr opened a pull request:

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

[FLINK-8985][e2etest] initial support for End-to-end CLI test, excluding 
YARN test

## What is the purpose of the change

Adding end to end test for CLI APIs.

## Brief change log

Added test_cli_api.sh script to test combinations of CLI commands listed in 
the doc section of Flink. Including:
  - Start up command sets (run)
  - Operational command sets (list/info/cancel)
  - Savepoint command sets (savepoint)

## Verifying this change

This is a test

## Does this pull request potentially affect one of the following parts:

No

## Documentation

No

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

$ git pull https://github.com/walterddr/flink FLINK-8985

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

https://github.com/apache/flink/pull/5863.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 #5863


commit 5f36ee5d4dcbb60a29a413fd29cfaaa69f7e8a47
Author: Rong Rong 
Date:   2018-04-17T20:45:23Z

initial commit to support CLI test, excluding YARN test




---