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 2020/01/30 15:15:28 UTC

[GitHub] [flink] rmetzger opened a new pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

rmetzger opened a new pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976
 
 
   
   ## What is the purpose of the change
   
   This pull request adds support for building and testing Flink on Azure Pipelines.
   There are two entry files (build definitions) in this pull request
   a) `azure-pipelines.yml` for the use with free Azure Pipelines accounts (executes all tests with default parameters)
   b) ` tools/azure-pipelines/build-apache-repo.yml` is a build definition specifically for the `apache/flink` repo. It uses a custom build machines through the `flink-ci/flink` repository. These machines have a lot of performance, thus providing faster build times.
   
   Both entry files rely on the same build job definitions in `tools/azure-pipelines/jobs-template.yml`.
   In addition to that, there are some utilities for running tests on Azure added in this PR. Travis test execution should be unaffected by this change.
   
   ## Verifying this change
   
   - [create a AZP account & setup a AZP pipelines for your Flink fork](https://cwiki.apache.org/confluence/display/FLINK/%5Bpreview%5D+Azure+Pipelines)
   - Push the changes in this PR to a branch in your GH repo
   - validate that the building for pushes is working
   - trigger another build, with "MODE=e2e" to see end to end test execution on AZP.
   
   Note: e2e test execution does currently fail, due to an issue with minikube.
   
   ## Does this pull request potentially affect one of the following parts:
   - this PR only affects the build system, potentially Travis.
   
   ## Documentation
   
   (work in progress) documentation is available here: https://cwiki.apache.org/confluence/display/FLINK/%5Bpreview%5D+Azure+Pipelines
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5225 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149168315 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   Hash:da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149168315) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5225) 
   * da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375848264
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
 
 Review comment:
   can this be shared across both .yml files?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378777994
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh
 ##########
 @@ -39,10 +39,31 @@ on_exit test_cleanup
 TEST_ES_JAR=${END_TO_END_DIR}/flink-elasticsearch${ELASTICSEARCH_VERSION}-test/target/Elasticsearch${ELASTICSEARCH_VERSION}SinkExample.jar
 
 # run the Flink job
-$FLINK_DIR/bin/flink run -p 1 $TEST_ES_JAR \
+JOB_ID=$($FLINK_DIR/bin/flink run -d -p 1 $TEST_ES_JAR \
   --numRecords 20 \
   --index index \
-  --type type
+  --type type | awk '{print $NF}' | tail -n 1)
 
+
+# wait for 10 seconds
+wait_job_submitted ${JOB_ID}
 
 Review comment:
   This call just waits for the job to be submitted (somehow). The second loop expects the job to be in state "RUNNING".
   It would fail if the job status was "CREATED", which I assume is a state that we might see.
   
   To be honest, I can also undo the changes to the elasticsearch script + the common.sh script. I just left them in in case the tests are failing in the future.
   
   With my changes, the tests will at least fail after a few seconds, instead of hanging indefinitely, and they will print the Flink logs to make debugging easier.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] AHeise commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378883513
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/common_s3_minio.sh
 ##########
 @@ -43,18 +56,18 @@ S3_TEST_DATA_WORDS_URI="s3://$IT_CASE_S3_BUCKET/words"
 #   S3_ENDPOINT
 ###################################
 function s3_start {
-  echo "Spawning minio for s3 tests"
+  echo "Spawning minio for s3 tests with DATA_DIR=$DATA_DIR"
   export MINIO_CONTAINER_ID=$(docker run -d \
-    -P \
-    --mount type=bind,source="$TEST_INFRA_DIR",target=/data \
 
 Review comment:
   LGTM. A few things are quite specific, but given that we will replace these tests with Java soonish, I'd not invest too much time in it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378356957
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/common_s3_minio.sh
 ##########
 @@ -43,18 +56,18 @@ S3_TEST_DATA_WORDS_URI="s3://$IT_CASE_S3_BUCKET/words"
 #   S3_ENDPOINT
 ###################################
 function s3_start {
-  echo "Spawning minio for s3 tests"
+  echo "Spawning minio for s3 tests with DATA_DIR=$DATA_DIR"
   export MINIO_CONTAINER_ID=$(docker run -d \
-    -P \
-    --mount type=bind,source="$TEST_INFRA_DIR",target=/data \
 
 Review comment:
   @AHeise can you sanity check these changes?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373029484
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -68,7 +69,7 @@ MVN_TEST_MODULES=$(get_test_modules_for_stage ${TEST})
 # Flink, which however should all be built locally. see FLINK-7230
 #
 MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn"
-MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -B -Pskip-webui-build $MVN_LOGGING_OPTIONS"
+MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Dmaven.wagon.http.pool=false -B -Pskip-webui-build $MVN_LOGGING_OPTIONS"
 
 Review comment:
   separate commit; how have you verified the effectiveness of this change?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373460583
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   https://issues.apache.org/jira/browse/FLINK-15834

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373030472
 
 

 ##########
 File path: flink-end-to-end-tests/run-nightly-tests.sh
 ##########
 @@ -88,8 +88,11 @@ run_test "Resuming Externalized Checkpoint after terminal failure (rocks, increm
 # Docker
 ################################################################################
 
-run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh"
-run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs"
+# Ignore these tests on Azure
 
 Review comment:
   why?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373455464
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -273,27 +280,27 @@ cd ../../
 case $TEST in
     (misc)
         if [ $EXIT_CODE == 0 ]; then
-            printf "\n\n==============================================================================\n"
-            printf "Running bash end-to-end tests\n"
-            printf "==============================================================================\n"
+            echo "\n\n==============================================================================\n"
+            echo "Running bash end-to-end tests\n"
+            echo "==============================================================================\n"
 
             FLINK_DIR=build-target flink-end-to-end-tests/run-pre-commit-tests.sh
 
             EXIT_CODE=$?
         else
-            printf "\n==============================================================================\n"
-            printf "Previous build failure detected, skipping bash end-to-end tests.\n"
-            printf "==============================================================================\n"
+            echo "\n==============================================================================\n"
+            echo "Previous build failure detected, skipping bash end-to-end tests.\n"
+            echo "==============================================================================\n"
         fi
         if [ $EXIT_CODE == 0 ]; then
-            printf "\n\n==============================================================================\n"
-            printf "Running java end-to-end tests\n"
 
 Review comment:
   I didn't realize at this point that you're calling the _travis_ watchdog from the azure scripts ;)
   
   Consider this comment resolved. Do note however that there are plenty of printf statements in the e2e scripts that either have to modified as well or we figure out why printf isn't working on azure.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373457638
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   ok, thx for the fast response.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378170728
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
 
 Review comment:
   As jobs.
   
   Stages are different triggers, such as "build on push / pr", "end 2 end tests", "nightly cron"
   Jobs (included from the jobs-template) are parameterized through the `environment` for different scala / hadoop / whatnot.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375834725
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
 
 Review comment:
   how do you intend to handle 1.10/1.9?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378364489
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   how come? Isn't there a reference somewhere else that some e2e tests use docker and hence the container must be setup outside?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375833309
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   ping to remove the jdk11 stage

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373380548
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -68,7 +69,7 @@ MVN_TEST_MODULES=$(get_test_modules_for_stage ${TEST})
 # Flink, which however should all be built locally. see FLINK-7230
 #
 MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn"
-MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -B -Pskip-webui-build $MVN_LOGGING_OPTIONS"
+MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Dmaven.wagon.http.pool=false -B -Pskip-webui-build $MVN_LOGGING_OPTIONS"
 
 Review comment:
   The amount of connection timeouts significantly reduced.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373029092
 
 

 ##########
 File path: tools/travis/setup_maven.sh
 ##########
 @@ -36,3 +37,4 @@ if [ -d "${HOME}/.m2/repository/" ]; then
   find ${HOME}/.m2/repository/ -name "*.jar" -exec sh -c 'if ! zip -T {} >/dev/null ; then echo "deleting invalid file: {}"; rm -f {} ; fi' \;
 fi
 
+echo "Installed Maven ${MAVEN_VERSION} to ${M2_HOME}"
 
 Review comment:
   separate hotfix commit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379291684
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   Running the pre commit on a separate machine doesn't make much sense, as it needs 25 minutes of compiling, and then runs for 2 minutes: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5164&view=logs&j=62470212-5e37-5318-626e-1835b0c31696
   
   I'm now trying to run the entire misc stage on an azure hosted instance.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379301818
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   FYI; you may want to look into decreasing the transfer.sh timeout, we blew 10 minutes in multiple profiles on that build you linked.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375878025
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
 
 Review comment:
   When the `CACHE_KEY` has a miss, it'll use the fallback key.
   What this means in practice is that we are downloading the cache, even if there were changes to the pom files.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379325288
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   I kicked off this hacky build last night, that's why some things are not correct.
   I now include the custom maven version setup step, and the cache: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5168&view=results
   Let's see how it affects the build time.
   
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379420605
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   I will look into the problems you've mentioned. This is a successful run of running the entire misc stage on an Azure instance: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5172&view=results
   The benefit of this approach would be, that the number of builds stays the same.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378768263
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   Thanks a lot. I'll rebase.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580324983
 
 
   Do we have to make any changes to CiBot?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378743130
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   But the pre-commit-tests are still being run in docker, no?
   
   azure-controller.sh#L191: `Some tests in the "run-pre-commit-tests.sh" collection launch Docker containers.`
   
   If running end-to-end tests in a docker container is troublesome, shouldn't the pre-commit-tests also be executed separately? As it stands we have to handle both setups.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375844789
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,193 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+
+cat << EOF > /tmp/az_settings.xml
 
 Review comment:
   document the reason

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378357685
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh
 ##########
 @@ -39,10 +39,31 @@ on_exit test_cleanup
 TEST_ES_JAR=${END_TO_END_DIR}/flink-elasticsearch${ELASTICSEARCH_VERSION}-test/target/Elasticsearch${ELASTICSEARCH_VERSION}SinkExample.jar
 
 # run the Flink job
-$FLINK_DIR/bin/flink run -p 1 $TEST_ES_JAR \
+JOB_ID=$($FLINK_DIR/bin/flink run -d -p 1 $TEST_ES_JAR \
   --numRecords 20 \
   --index index \
-  --type type
+  --type type | awk '{print $NF}' | tail -n 1)
 
+
+# wait for 10 seconds
+wait_job_submitted ${JOB_ID}
 
 Review comment:
   isn't the loop below automatically handling this case as well?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378772192
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
 
 Review comment:
   Yes

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378393191
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -166,7 +171,7 @@ print_stacktraces () {
 put_yarn_logs_to_artifacts() {
 	# Make sure to be in project root
 	cd $HERE/../
-	for file in `find ./flink-yarn-tests/target/flink-yarn-tests* -type f -name '*.log'`; do
+	for file in `find ./flink-yarn-tests/target -type f -name '*.log'`; do
 
 Review comment:
   I was debugging failing YARN tests as part of the migration. IIRC, some of the newer YARN tests log into a different directory. It doesn't hurt to search for log files to be included in the debugging file a bit more broadly.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375847722
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
 
 Review comment:
   For the future, are we modeling different scala/hadoop versions as jobs, or stages?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375842487
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
 
 Review comment:
   this is still WIP, correct? If so, let's exclude it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375836002
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
 
 Review comment:
   ```suggestion
   #  - end-to-end tests
   ```
   or
   ```suggestion
   #  - e2e tests
   ```
   We don't need another way to phrase it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373374860
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
 
 Review comment:
   This is not a repository reference. It refers to a utility docker image.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378165147
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
 
 Review comment:
   It is not wip anymore. End to end tests are working ✅ 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378394935
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   Yeah .. 
   The google mirror caused me some nice headache last night :( 
   ```
   2020-02-11T15:40:36.2013808Z [INFO] --- gmavenplus-plugin:1.8.1:execute (merge-categories) @ flink-end-to-end-tests ---
   2020-02-11T15:40:36.2032562Z [INFO] Downloading: https://maven-central.storage-download.googleapis.com/repos/central/data/org/codehaus/groovy/groovy-all/2.5.9/groovy-all-2.5.9.pom
   2020-02-11T15:40:36.4201087Z [WARNING] The POM for org.codehaus.groovy:groovy-all:pom:2.5.9 is missing, no dependency information available
   ```
   It seems that this file is really not available on the Google mirror. I guess we have to rely on the maven central mirrors :( 
   
   https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_apis/build/builds/5069/logs/14

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585750029
 
 
   I have updated this, the GitHub UI just doesn't show it next to your review comment:
   ![image](https://user-images.githubusercontent.com/89049/74439062-c7df7880-4e6b-11ea-8f4f-a7b0d4d61765.png)
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379423419
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   yeah but the build times for misc has increased by 17 minutes, increasing the total build duration if all jobs are running.
   I think in your previous version you also weren't running the java pre-commit-tests, which make a dedicated job more useful.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585145327
 
 
   Thanks a lot for your review.
   I have fixed the end to end test execution since the last time I responded to your comments.
   Once https://github.com/apache/flink/pull/11067 is merged, all tests should pass on Azure.
   
   Please let me know if you have further comments.
   Subsequent changes are in separate commits, for easier review.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378171401
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
 
 Review comment:
   No, I have not found a way. There are sadly some duplications between the two build definitions, but I've tried to keep the `azure-pipelines.yml` file as simple as possible (because that's the one most people will use and look at).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373029139
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -138,6 +144,7 @@ upload_artifacts_s3() {
 		artifacts upload --bucket $UPLOAD_BUCKET --key $UPLOAD_ACCESS_KEY --secret $UPLOAD_SECRET_KEY --target-paths $UPLOAD_TARGET_PATH $ARTIFACTS_FILE
 	fi
 
+
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375878754
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
 
 Review comment:
   Since we are using a fallback key, the new cache file will be based on a previous cache.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378363259
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
 
 Review comment:
   what is this `@1` thing?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378352103
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,95 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+# to be enabled with FLINK-15834
+#schedules:
+#- cron: "0 0 * * *"
+#  displayName: nightly build
+#  branches:
+#    include:
+#    - master
+#    - release-*
+#  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-2a765ab
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-16.04'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+  # to be enabled with FLINK-15834
 
 Review comment:
   remove now and add later in FLINK-15834

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585748453
 
 
   > I believe I have reacted to all your comments.
   
   You missed https://github.com/apache/flink/pull/10976#discussion_r378347602

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378162555
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
 
 Review comment:
   I clarified the language.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/149168315 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/149168315) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375833517
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
 
 Review comment:
   ```suggestion
     # Special stage for nightly builds:
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378347190
 
 

 ##########
 File path: tools/azure-pipelines/setup_kubernetes.sh
 ##########
 @@ -0,0 +1,26 @@
+# 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.
+
+echo "Replace moby by docker"
+docker version
+sudo apt-get remove -y moby-engine
+curl -fsSL https://download.docker.com/linux/ubuntu/gpg | sudo apt-key add -
+sudo add-apt-repository \
+   "deb [arch=amd64] https://download.docker.com/linux/ubuntu \
+   $(lsb_release -cs) \
+   stable"
+sudo apt-get update
+sudo apt-get install -y docker-ce docker-ce-cli containerd.io
 
 Review comment:
   Could we make this part of the custom docker image we're using anyway?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585145327
 
 
   Thanks a lot for your review.
   I have fixed the end to end test execution since the last time I responded to your comments.
   Once https://github.com/apache/flink/pull/11067 is merged, all tests should pass on Azure.
   
   Please let me know if you have further comments.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373387519
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   Yeah, no. This is some left over work in progress.
   What do you prefer: adding proper jdk11 support as part of this PR, or removing it now, and doing it in a separate PR? (I prefer the latter)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378161731
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
 
 Review comment:
   I will fix this as part of FLINK-15834

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373029004
 
 

 ##########
 File path: tools/travis/setup_maven.sh
 ##########
 @@ -21,6 +21,7 @@ MAVEN_VERSION="3.2.5"
 MAVEN_CACHE_DIR=${HOME}/maven_cache
 MAVEN_VERSIONED_DIR=${MAVEN_CACHE_DIR}/apache-maven-${MAVEN_VERSION}
 
+
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373558936
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -68,7 +69,7 @@ MVN_TEST_MODULES=$(get_test_modules_for_stage ${TEST})
 # Flink, which however should all be built locally. see FLINK-7230
 #
 MVN_LOGGING_OPTIONS="-Dlog.dir=${ARTIFACTS_DIR} -Dlog4j.configuration=file://$LOG4J_PROPERTIES -Dorg.slf4j.simpleLogger.log.org.apache.maven.cli.transfer.Slf4jMavenTransferListener=warn"
-MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -B -Pskip-webui-build $MVN_LOGGING_OPTIONS"
+MVN_COMMON_OPTIONS="-nsu -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dfast -Dmaven.wagon.http.pool=false -B -Pskip-webui-build $MVN_LOGGING_OPTIONS"
 
 Review comment:
   then let's a) also add this into the travis_controller.sh and move the change into a separate commit.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378741445
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   You can rebase on master or cherry-pick `a984b6a6ca60aa8adbc94397d9dfeee0ac8542fb`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378769414
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   Correct, the pre-commit tests run in docker (because they are executed on each push / pr). For now, I want all per push / pr stuff running on the custom build machines.
   If we are going to always executed the end to end tests, we can consider moving the pre-commit tests.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378351802
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -166,7 +171,7 @@ print_stacktraces () {
 put_yarn_logs_to_artifacts() {
 	# Make sure to be in project root
 	cd $HERE/../
-	for file in `find ./flink-yarn-tests/target/flink-yarn-tests* -type f -name '*.log'`; do
+	for file in `find ./flink-yarn-tests/target -type f -name '*.log'`; do
 
 Review comment:
   why is this required?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] asfgit closed pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378441114
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   try clearing the cache, we had the exact same issue on travis a few days ago.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378978694
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,219 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
 
 Review comment:
   Testing now https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5147&view=results

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375832273
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
 
 Review comment:
   Does using the poms as a cache key implies that whenever we add a dependency we have to rebuild the maven cache?
   
   Can we write down some basics on how the caching works on azure?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378348962
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
+MVN="mvn clean install $MAVEN_OPTS -nsu -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.wagon.http.pool=false -Dmaven.javadoc.skip=true -B -U -DskipTests $PROFILE"
+
+# Run actual compile&test steps
+if [ $STAGE == "$STAGE_COMPILE" ]; then
+    # run mvn clean install:
+    $MVN
+    EXIT_CODE=$?
+
+    if [ $EXIT_CODE == 0 ]; then
+        echo "\n\n==============================================================================\n"
+        echo "Checking scala suffixes\n"
+        echo "==============================================================================\n"
+
+        ./tools/verify_scala_suffixes.sh "${PROFILE}"
+        EXIT_CODE=$?
+    else
+        echo "\n==============================================================================\n"
+        echo "Previous build failure detected, skipping scala-suffixes check.\n"
+        echo "==============================================================================\n"
+    fi
+    
+    if [ $EXIT_CODE == 0 ]; then
+        check_shaded_artifacts
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_s3_fs hadoop
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_s3_fs presto
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 2
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 5
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 6
+        EXIT_CODE=$(($EXIT_CODE+$?))
+    else
+        echo "=============================================================================="
+        echo "Previous build failure detected, skipping shaded dependency check."
+        echo "=============================================================================="
+    fi
+
+    if [ $EXIT_CODE == 0 ]; then
+        echo "Creating cache build directory $CACHE_FLINK_DIR"
+    
+        cp -r . "$CACHE_FLINK_DIR"
+
+        function minimizeCachedFiles() {
+            # reduces the size of the cached directory to speed up
+            # the packing&upload / download&unpacking process
+            # by removing files not required for subsequent stages
+    
+            # jars are re-built in subsequent stages, so no need to cache them (cannot be avoided)
+            find "$CACHE_FLINK_DIR" -maxdepth 8 -type f -name '*.jar' \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-csv/target/flink-csv*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-json/target/flink-json*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-avro/target/flink-avro*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-runtime/target/flink-runtime*tests.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-streaming-java/target/flink-streaming-java*tests.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-dist*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-table_*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-table-blink*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/opt/flink-python*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-connectors/flink-connector-elasticsearch-base/target/flink-*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-connectors/flink-connector-kafka-base/target/flink-*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-table/flink-table-planner/target/flink-table-planner*tests.jar" | xargs rm -rf
+    
+            # .git directory
+            # not deleting this can cause build stability issues
+            # merging the cached version sometimes fails
+            rm -rf "$CACHE_FLINK_DIR/.git"
+
+            # AZ Pipelines has a problem with links.
+            rm "$CACHE_FLINK_DIR/build-target"
+        }
+    
+        echo "Minimizing cache"
+        minimizeCachedFiles
+    else
+        echo "=============================================================================="
+        echo "Previous build failure detected, skipping cache setup."
+        echo "=============================================================================="
+    fi
+elif [ $STAGE != "$STAGE_CLEANUP" ]; then
+    if ! [ -e $CACHE_FLINK_DIR ]; then
+        echo "Cached flink dir $CACHE_FLINK_DIR does not exist. Exiting build."
+        exit 1
+    fi
+    # merged compiled flink into local clone
+    # this prevents the cache from being re-uploaded
+    echo "Merging cache"
+    cp -RT "$CACHE_FLINK_DIR" "."
+
+    echo "Adjusting timestamps"
+    # adjust timestamps to prevent recompilation
+    find . -type f -name '*.java' | xargs touch
+    find . -type f -name '*.scala' | xargs touch
+    # wait a bit for better odds of different timestamps
+    sleep 5
+    find . -type f -name '*.class' | xargs touch
+    find . -type f -name '*.timestamp' | xargs touch
+
+    if [ $STAGE == $STAGE_PYTHON ]; then
+        echo "=============================================================================="
+        echo "Python stage found. Re-compiling (this is required on Azure for the python tests to pass)"
 
 Review comment:
   I thought you found a solution for that, am I remembering wrong?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378392492
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   the end to end tests are fairly invasive on the underlying system, that's why I wanted to use ephemeral machines. Also since the e2e tests use docker extensively, we would have much more docker in docker trouble

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378743130
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   But the pre-commit-tests are still being run in docker, no?
   
   azure-controller.sh#L191: `    # Some tests in the "run-pre-commit-tests.sh" collection launch Docker containers.
       # Since the regular build is executed in Docker (on Azure), we'll be launching those 
       # containers outside of the current container (on the host, alongside the build&test container).
       # Some of these containers mount a path. Currently, these scripts mount relative to the build container,
       # thus this path is not available on the host (where the test container is launched).
       # 
       # Here, we figure out the path on the host machine, and set it.
       #`
   
   If running end-to-end tests in a docker container is troublesome, shouldn't the pre-commit-tests also be executed separately? As it stands we have to handle both setups.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-586191197
 
 
   Awesome, thanks
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378852616
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   The cherry-picked build now fails with https://maven-central.storage-download.googleapis.com/repos/central/data/org/apache/beam/beam-runners-java-fn-execution/2.19.0/beam-runners-java-fn-execution-2.19.0.pom missing. The artifact has been released on Feb 03.
   I just send an email to the guys running the google mirror to see what they have to say :) 
   
   I'm starting to wonder whether we should see if it makes sense to set up our own Maven mirror on one of our build machines: https://www.sonatype.com/download-oss-sonatype

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378868088
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   Setting up a maven proxy is indeed pretty simple. Got something working locally in a few minutes.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378166539
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
 
 Review comment:
   changed to
   ```
     workspace:
       clean: all # this cleans the entire workspace directory before running a new job
       # It is necessary because the custom build machines are reused for tests.
       # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
   ``` 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580719339
 
 
   > Do we have to make any changes to CiBot?
   
   Not that I'm aware of.  If we want to trigger e2e tests through CiBot, the variable "MODE=e2e" needs to be passed. But I can change that to whatever you like.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378740151
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   I'll downgrade groovy to 2.5.8, this one seems to be available on the google mirror.
   
   Although it's pretty stupid we have to worry about stuff like this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373379409
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_kinesis.sh
 ##########
 @@ -27,12 +27,16 @@ export AWS_ACCESS_KEY_ID=flinkKinesisTestFakeAccessKeyId
 export AWS_SECRET_KEY=flinkKinesisTestFakeAccessKey
 
 KINESALITE_PORT=4567
+KINESALITE_HOST=kinesalite-container
+KINESALITE_NETWORK=some
 
 Review comment:
   I'll revert this change. These changes are leftovers of my attempt to make this test pass in a docker-in-docker scenario.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-586147566
 
 
   ```
   [FAIL] 'State Migration end-to-end test from 1.6' failed after 0 minutes and 0 seconds! Test exited with exit code 1
   
   /home/vsts/work/1/s/flink-end-to-end-tests/test-scripts/common.sh: line 414: /bin/stop-cluster.sh: No such file or directory
   rm: cannot remove '/lib/terminfo/x/xterm-mono': Permission denied
   rm: cannot remove '/lib/terminfo/x/xterm-debian': Permission denied
   ```
   
   You weren't kidding when you said the tests are invasive :P

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375836625
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
 
 Review comment:
   What parts of this file are related to `Flink fork and custom AZP account`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373033219
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   where are we actually setting up jdk11?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373384117
 
 

 ##########
 File path: flink-end-to-end-tests/run-nightly-tests.sh
 ##########
 @@ -88,8 +88,11 @@ run_test "Resuming Externalized Checkpoint after terminal failure (rocks, increm
 # Docker
 ################################################################################
 
-run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh"
-run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs"
+# Ignore these tests on Azure
 
 Review comment:
   In these tests, the TaskManagers are not starting on YARN, probably due to memory constraints.
   Do you agree to file a JIRA ticket to fix this?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373454619
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   separate PR

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378363259
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
 
 Review comment:
   what is this ´@1` thing?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379381874
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   It was again downloading all dependencies.
   It's not compiling Flink (so that caching works), but the maven caching seems broken.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378362653
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
 
 Review comment:
   If both caches miss, do we then create a cache under both keys at the end?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373454541
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   separate PR, java 11 runs only on cron jobs anyway which are a follow-up in any case.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375831503
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
 
 Review comment:
   when is this key used?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375846551
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
+  pool: ${{parameters.e2e_pool_definition}}
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+    - task: CacheBeta@1
 
 Review comment:
   can templates also refer to other templates? This task is duplicated across both jobs.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-586183086
 
 
   I have adjusted CiBot; if everything is setup correctly it should start linking azure builds in this PR for new commits.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378392237
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
 
 Review comment:
   I think its the version of the task

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375844125
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
 
 Review comment:
   what does this do?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379302723
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   Didn't the build take so long because it had to download all dependencies?
   
   Btw, also looks like the precommit build used the wrong maven version.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378167572
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
 
 Review comment:
   See: https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml
   It sounds like this makes azure try harder to cancel a running task. Sadly, task cancellation is not very reliable on the custom machines (they usually continue running till the current task has finished).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375833913
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
 
 Review comment:
   Document where this is controlled.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378391274
 
 

 ##########
 File path: tools/azure-pipelines/setup_kubernetes.sh
 ##########
 @@ -0,0 +1,26 @@
+# 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.
+
+echo "Replace moby by docker"
+docker version
+sudo apt-get remove -y moby-engine
+curl -fsSL https://download.docker.com/linux/ubuntu/gpg | sudo apt-key add -
+sudo add-apt-repository \
+   "deb [arch=amd64] https://download.docker.com/linux/ubuntu \
+   $(lsb_release -cs) \
+   stable"
+sudo apt-get update
+sudo apt-get install -y docker-ce docker-ce-cli containerd.io
 
 Review comment:
   The end to end tests are not executed in our custom docker image, but on the build machines provided by Azure (because the end to end tests are fairly invasive on the underlying system, that's why I wanted to use ephemeral machines. Also since the e2e tests use docker extensively, we would have much more docker in docker trouble)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378842875
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -166,7 +171,7 @@ print_stacktraces () {
 put_yarn_logs_to_artifacts() {
 	# Make sure to be in project root
 	cd $HERE/../
-	for file in `find ./flink-yarn-tests/target/flink-yarn-tests* -type f -name '*.log'`; do
+	for file in `find ./flink-yarn-tests/target -type f -name '*.log'`; do
 
 Review comment:
   then let's put this into a separate commit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373031699
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_kinesis.sh
 ##########
 @@ -27,12 +27,16 @@ export AWS_ACCESS_KEY_ID=flinkKinesisTestFakeAccessKeyId
 export AWS_SECRET_KEY=flinkKinesisTestFakeAccessKey
 
 KINESALITE_PORT=4567
+KINESALITE_HOST=kinesalite-container
+KINESALITE_NETWORK=some
 
 Review comment:
   what's this about? probably deserves a separate commit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-586170977
 
 
   > > Maybe you actually need to do a minor change in there, as the name of the builds has changed.
   > 
   > Good to know; it's just a simple configuration change that I can already apply. Will they be called like in the build you linked? (`compile_ci_build`, `e2e_ci_build`, `test_ci_*`)
   
   Yes

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378164978
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
 
 Review comment:
   at the time of your review, I would have agreed. But I have spend the past few days fixing the end to end test execution, and I would suggest to keep the e2e mode in.
   This would allow people to run the end to end tests at least in their private forks, and hopefully soon through CI bot as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580302104
 
 
   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 322bf186867ed0ec1a867ba7919e39e3c437aba0 (Thu Jan 30 15:17:40 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379301493
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   And these numbers still look _fine_ to me; they aren't increasing the total build duration.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149168315) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373455630
 
 

 ##########
 File path: flink-end-to-end-tests/run-nightly-tests.sh
 ##########
 @@ -88,8 +88,11 @@ run_test "Resuming Externalized Checkpoint after terminal failure (rocks, increm
 # Docker
 ################################################################################
 
-run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh"
-run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs"
+# Ignore these tests on Azure
 
 Review comment:
   of course, but also document in the file why they are excluded

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379419236
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh
 ##########
 @@ -39,10 +39,31 @@ on_exit test_cleanup
 TEST_ES_JAR=${END_TO_END_DIR}/flink-elasticsearch${ELASTICSEARCH_VERSION}-test/target/Elasticsearch${ELASTICSEARCH_VERSION}SinkExample.jar
 
 # run the Flink job
-$FLINK_DIR/bin/flink run -p 1 $TEST_ES_JAR \
+JOB_ID=$($FLINK_DIR/bin/flink run -d -p 1 $TEST_ES_JAR \
   --numRecords 20 \
   --index index \
-  --type type
+  --type type | awk '{print $NF}' | tail -n 1)
 
+
+# wait for 10 seconds
+wait_job_submitted ${JOB_ID}
 
 Review comment:
   I will undo my changes. I don't spend more time with this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373036012
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -273,27 +280,27 @@ cd ../../
 case $TEST in
     (misc)
         if [ $EXIT_CODE == 0 ]; then
-            printf "\n\n==============================================================================\n"
-            printf "Running bash end-to-end tests\n"
-            printf "==============================================================================\n"
+            echo "\n\n==============================================================================\n"
+            echo "Running bash end-to-end tests\n"
+            echo "==============================================================================\n"
 
             FLINK_DIR=build-target flink-end-to-end-tests/run-pre-commit-tests.sh
 
             EXIT_CODE=$?
         else
-            printf "\n==============================================================================\n"
-            printf "Previous build failure detected, skipping bash end-to-end tests.\n"
-            printf "==============================================================================\n"
+            echo "\n==============================================================================\n"
+            echo "Previous build failure detected, skipping bash end-to-end tests.\n"
+            echo "==============================================================================\n"
         fi
         if [ $EXIT_CODE == 0 ]; then
-            printf "\n\n==============================================================================\n"
-            printf "Running java end-to-end tests\n"
 
 Review comment:
   if we intend to drop this file anyway soon-ish, why are we making this kind of change?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585761557
 
 
   I have seen that, but that's not a _reason_. You're explaining _what_ you are doing, not _why_.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378169655
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
+  pool: ${{parameters.e2e_pool_definition}}
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+    - task: CacheBeta@1
 
 Review comment:
   In this case, yes. However, the Cache definition is only 5 lines of code. I'm afraid that people might have a hard time understanding the build definition if there are too many indirections.
   I vote to keep it as-is.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379543557
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   I fixed the downloading of the dependencies, and it uses maven 3.2.5: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5189&view=results

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379419236
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh
 ##########
 @@ -39,10 +39,31 @@ on_exit test_cleanup
 TEST_ES_JAR=${END_TO_END_DIR}/flink-elasticsearch${ELASTICSEARCH_VERSION}-test/target/Elasticsearch${ELASTICSEARCH_VERSION}SinkExample.jar
 
 # run the Flink job
-$FLINK_DIR/bin/flink run -p 1 $TEST_ES_JAR \
+JOB_ID=$($FLINK_DIR/bin/flink run -d -p 1 $TEST_ES_JAR \
   --numRecords 20 \
   --index index \
-  --type type
+  --type type | awk '{print $NF}' | tail -n 1)
 
+
+# wait for 10 seconds
+wait_job_submitted ${JOB_ID}
 
 Review comment:
   I will undo my changes. I don't want spend more time with this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378345883
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_mesos_multiple_submissions.sh
 ##########
 @@ -29,7 +29,7 @@ TEST_PROGRAM_JAR=$END_TO_END_DIR/flink-cli-test/target/PeriodicStreamingJob.jar
 
 function submit_job {
     local output_path=$1
-    docker exec -it mesos-master bash -c "${FLINK_DIR}/bin/flink run -d -p 1 ${TEST_PROGRAM_JAR} --durationInSecond ${DURATION} --outputPath ${output_path}" \
+    docker exec mesos-master bash -c "${FLINK_DIR}/bin/flink run -d -p 1 ${TEST_PROGRAM_JAR} --durationInSecond ${DURATION} --outputPath ${output_path}" \
 
 Review comment:
   What's this change about?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375834022
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
 
 jobs:
-- template: flink-build-jobs.yml@templates
+  - template: tools/azure-pipelines/jobs-template.yml
+    parameters:
+      stage_name: ci_build
+      test_pool_definition:
 
 Review comment:
   what are these definitions for?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378743130
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   But the pre-commit-tests are still being run in docker, no?
   
   azure-controller.sh#L191: 
   ```
   # Some tests in the "run-pre-commit-tests.sh" collection launch Docker containers.
   # Since the regular build is executed in Docker (on Azure), we'll be launching those 
   # containers outside of the current container (on the host, alongside the build&test container).
   # Some of these containers mount a path. Currently, these scripts mount relative to the build container,
   # thus this path is not available on the host (where the test container is launched).
   ```
   
   If running end-to-end tests in a docker container is troublesome, shouldn't the pre-commit-tests also be executed separately? As it stands we have to handle both setups.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375837622
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
 
 Review comment:
   What does this control? (is it the name of the stage? If so, why is it defined under `jobs`)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378164317
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
 
 Review comment:
   I added the following documentation to the `jobs-template.yml`
   `defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)`
   It is a bit confusing that the stage name is under `jobs`. We can rename that parameter if we have a better name for it.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373382927
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -273,27 +280,27 @@ cd ../../
 case $TEST in
     (misc)
         if [ $EXIT_CODE == 0 ]; then
-            printf "\n\n==============================================================================\n"
-            printf "Running bash end-to-end tests\n"
-            printf "==============================================================================\n"
+            echo "\n\n==============================================================================\n"
+            echo "Running bash end-to-end tests\n"
+            echo "==============================================================================\n"
 
             FLINK_DIR=build-target flink-end-to-end-tests/run-pre-commit-tests.sh
 
             EXIT_CODE=$?
         else
-            printf "\n==============================================================================\n"
-            printf "Previous build failure detected, skipping bash end-to-end tests.\n"
-            printf "==============================================================================\n"
+            echo "\n==============================================================================\n"
+            echo "Previous build failure detected, skipping bash end-to-end tests.\n"
+            echo "==============================================================================\n"
         fi
         if [ $EXIT_CODE == 0 ]; then
-            printf "\n\n==============================================================================\n"
-            printf "Running java end-to-end tests\n"
 
 Review comment:
   I did not know that there are intentions to drop this file. 
   If I'm not mistaken, printf did not produce any outputs on azure?! I don't exactly remember. If this is a big problem for you, I can invest time into this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378393653
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
+MVN="mvn clean install $MAVEN_OPTS -nsu -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.wagon.http.pool=false -Dmaven.javadoc.skip=true -B -U -DskipTests $PROFILE"
+
+# Run actual compile&test steps
+if [ $STAGE == "$STAGE_COMPILE" ]; then
+    # run mvn clean install:
+    $MVN
+    EXIT_CODE=$?
+
+    if [ $EXIT_CODE == 0 ]; then
+        echo "\n\n==============================================================================\n"
+        echo "Checking scala suffixes\n"
+        echo "==============================================================================\n"
+
+        ./tools/verify_scala_suffixes.sh "${PROFILE}"
+        EXIT_CODE=$?
+    else
+        echo "\n==============================================================================\n"
+        echo "Previous build failure detected, skipping scala-suffixes check.\n"
+        echo "==============================================================================\n"
+    fi
+    
+    if [ $EXIT_CODE == 0 ]; then
+        check_shaded_artifacts
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_s3_fs hadoop
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_s3_fs presto
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 2
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 5
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 6
+        EXIT_CODE=$(($EXIT_CODE+$?))
+    else
+        echo "=============================================================================="
+        echo "Previous build failure detected, skipping shaded dependency check."
+        echo "=============================================================================="
+    fi
+
+    if [ $EXIT_CODE == 0 ]; then
+        echo "Creating cache build directory $CACHE_FLINK_DIR"
+    
+        cp -r . "$CACHE_FLINK_DIR"
+
+        function minimizeCachedFiles() {
+            # reduces the size of the cached directory to speed up
+            # the packing&upload / download&unpacking process
+            # by removing files not required for subsequent stages
+    
+            # jars are re-built in subsequent stages, so no need to cache them (cannot be avoided)
+            find "$CACHE_FLINK_DIR" -maxdepth 8 -type f -name '*.jar' \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-csv/target/flink-csv*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-json/target/flink-json*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-avro/target/flink-avro*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-runtime/target/flink-runtime*tests.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-streaming-java/target/flink-streaming-java*tests.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-dist*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-table_*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-table-blink*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/opt/flink-python*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-connectors/flink-connector-elasticsearch-base/target/flink-*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-connectors/flink-connector-kafka-base/target/flink-*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-table/flink-table-planner/target/flink-table-planner*tests.jar" | xargs rm -rf
+    
+            # .git directory
+            # not deleting this can cause build stability issues
+            # merging the cached version sometimes fails
+            rm -rf "$CACHE_FLINK_DIR/.git"
+
+            # AZ Pipelines has a problem with links.
+            rm "$CACHE_FLINK_DIR/build-target"
+        }
+    
+        echo "Minimizing cache"
+        minimizeCachedFiles
+    else
+        echo "=============================================================================="
+        echo "Previous build failure detected, skipping cache setup."
+        echo "=============================================================================="
+    fi
+elif [ $STAGE != "$STAGE_CLEANUP" ]; then
+    if ! [ -e $CACHE_FLINK_DIR ]; then
+        echo "Cached flink dir $CACHE_FLINK_DIR does not exist. Exiting build."
+        exit 1
+    fi
+    # merged compiled flink into local clone
+    # this prevents the cache from being re-uploaded
+    echo "Merging cache"
+    cp -RT "$CACHE_FLINK_DIR" "."
+
+    echo "Adjusting timestamps"
+    # adjust timestamps to prevent recompilation
+    find . -type f -name '*.java' | xargs touch
+    find . -type f -name '*.scala' | xargs touch
+    # wait a bit for better odds of different timestamps
+    sleep 5
+    find . -type f -name '*.class' | xargs touch
+    find . -type f -name '*.timestamp' | xargs touch
+
+    if [ $STAGE == $STAGE_PYTHON ]; then
+        echo "=============================================================================="
+        echo "Python stage found. Re-compiling (this is required on Azure for the python tests to pass)"
 
 Review comment:
   I hoped to have found a solution, but it still didn't work.
   I'm happy to consider this a quick fix and revisit this problem once the PR is merged.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585665184
 
 
   I believe I have reacted to all your comments. Please let me know if there's more feedback.
   I re-enabled the google mirror. Let's see if this build is passing: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5129&view=results
   
   >Do we have to make any changes to CiBot?
   
   Maybe you actually need to do a minor change in there, as the name of the builds has changed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375844066
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  steps:
+
+  # Preparation
+  - task: CacheBeta@1
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+      cacheHitVar: CACHE_RESTORED
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
 
 Review comment:
   what does this do?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378361535
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
 
 Review comment:
   ```suggestion
     # In this configuration, we use a hash over all committed (not generated) .pom files 
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-585745043
 
 
   > Maybe you actually need to do a minor change in there, as the name of the builds has changed.
   
   Good to know; it's just a simple configuration change that I can already apply. Will they be called like in the build you linked? (`compile_ci_build`, `e2e_ci_build`, `test_ci_*`)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148988925) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375834454
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
 
 Review comment:
   ```suggestion
     displayName: nightly build
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378927106
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   I'm trying out running the precommit tests in a VM: https://dev.azure.com/rmetzger/Flink/_build/results?buildId=5144&view=results

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378161079
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
 
 jobs:
-- template: flink-build-jobs.yml@templates
+  - template: tools/azure-pipelines/jobs-template.yml
+    parameters:
+      stage_name: ci_build
+      test_pool_definition:
 
 Review comment:
   This defines the hardware pool for compilation and unit test execution in the `job-template.yml`.
   I will add some clarifying comments into the source.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:PENDING URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [PENDING](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379302723
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   Didn't the build take so long because it had to download all dependencies?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373028729
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_kubernetes_embedded_job.sh
 ##########
 @@ -46,6 +46,7 @@ kubectl create -f ${KUBERNETES_MODULE_DIR}/job-cluster-service.yaml
 envsubst '${FLINK_IMAGE_NAME} ${FLINK_JOB} ${FLINK_JOB_PARALLELISM} ${FLINK_JOB_ARGUMENTS}' < ${CONTAINER_SCRIPTS}/job-cluster-job.yaml.template | kubectl create -f -
 envsubst '${FLINK_IMAGE_NAME} ${FLINK_JOB_PARALLELISM}' < ${CONTAINER_SCRIPTS}/task-manager-deployment.yaml.template | kubectl create -f -
 kubectl wait --for=condition=complete job/flink-job-cluster --timeout=1h
+
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379382262
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   This build is also still not using maven 3.2.5 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373035698
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,193 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+# Run actual compile&test steps
+if [ $STAGE == "$STAGE_COMPILE" ]; then
+    #adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+    MVN="mvn clean install --settings /tmp/az_settings.xml $MAVEN_OPTS -nsu -Dflink.convergence.phase=install -Pcheck-convergence -Dflink.forkCount=2 -Dflink.forkCountTestPackage=2 -Dmaven.wagon.http.pool=false -Dmaven.javadoc.skip=true -B -DskipTests $PROFILE"
+    $MVN
+    EXIT_CODE=$?
+
+    if [ $EXIT_CODE == 0 ]; then
+        echo "\n\n==============================================================================\n"
+        echo "Checking scala suffixes\n"
+        echo "==============================================================================\n"
+
+        ./tools/verify_scala_suffixes.sh "${PROFILE}"
+        EXIT_CODE=$?
+    else
+        echo "\n==============================================================================\n"
+        echo "Previous build failure detected, skipping scala-suffixes check.\n"
+        echo "==============================================================================\n"
+    fi
+    
+    if [ $EXIT_CODE == 0 ]; then
+        check_shaded_artifacts
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_s3_fs hadoop
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_s3_fs presto
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 2
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 5
+        EXIT_CODE=$(($EXIT_CODE+$?))
+        check_shaded_artifacts_connector_elasticsearch 6
+        EXIT_CODE=$(($EXIT_CODE+$?))
+    else
+        echo "=============================================================================="
+        echo "Previous build failure detected, skipping shaded dependency check."
+        echo "=============================================================================="
+    fi
+
+    if [ $EXIT_CODE == 0 ]; then
+        echo "Creating cache build directory $CACHE_FLINK_DIR"
+    
+        cp -r . "$CACHE_FLINK_DIR"
+
+        function minimizeCachedFiles() {
+            # reduces the size of the cached directory to speed up
+            # the packing&upload / download&unpacking process
+            # by removing files not required for subsequent stages
+    
+            # jars are re-built in subsequent stages, so no need to cache them (cannot be avoided)
+            find "$CACHE_FLINK_DIR" -maxdepth 8 -type f -name '*.jar' \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-csv/target/flink-csv*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-json/target/flink-json*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-formats/flink-avro/target/flink-avro*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-runtime/target/flink-runtime*tests.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-streaming-java/target/flink-streaming-java*tests.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-dist*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-table_*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/lib/flink-table-blink*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-dist/target/flink-*-bin/flink-*/opt/flink-python*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-connectors/flink-connector-elasticsearch-base/target/flink-*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-connectors/flink-connector-kafka-base/target/flink-*.jar" \
+            ! -path "$CACHE_FLINK_DIR/flink-table/flink-table-planner/target/flink-table-planner*tests.jar" | xargs rm -rf
+    
+            # .git directory
+            # not deleting this can cause build stability issues
+            # merging the cached version sometimes fails
+            rm -rf "$CACHE_FLINK_DIR/.git"
+
+            # AZ Pipelines has a problem with links.
+            rm "$CACHE_FLINK_DIR/build-target"
+        }
+    
+        echo "Minimizing cache"
+        minimizeCachedFiles
+    else
+        echo "=============================================================================="
+        echo "Previous build failure detected, skipping cache setup."
+        echo "=============================================================================="
+    fi
+elif [ $STAGE != "$STAGE_CLEANUP" ]; then
+    if ! [ -e $CACHE_FLINK_DIR ]; then
+        echo "Cached flink dir $CACHE_FLINK_DIR does not exist. Exiting build."
+        exit 1
+    fi
+    # merged compiled flink into local clone
+    # this prevents the cache from being re-uploaded
+    echo "Merging cache"
+    cp -RT "$CACHE_FLINK_DIR" "."
+
+    echo "Adjusting timestamps"
+    # adjust timestamps to prevent recompilation
+    find . -type f -name '*.java' | xargs touch
+    find . -type f -name '*.scala' | xargs touch
+    # wait a bit for better odds of different timestamps
+    sleep 5
+    find . -type f -name '*.class' | xargs touch
+    find . -type f -name '*.timestamp' | xargs touch
+
+    if [ $STAGE == $STAGE_PYTHON ]; then
+        echo "=============================================================================="
+        echo "Python stage found. Re-compiling (this is required on Azure for the python tests to pass)"
+        echo "=============================================================================="
+        mvn install -DskipTests -Drat.skip
 
 Review comment:
   should use the same options as the main compile step, particularly `$PROFILE`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5225 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149168315 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149168315) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5225) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149168315 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149168315) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r379300718
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   why isn't it re-using the cached compilation?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375839136
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
+  stage_name: # needed to make job names unique if they are included multiple times
+  environment: # used to pass environment variables into the downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
 
 Review comment:
   we may want to exclude all the stuff for custom triggered builds for now

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375834996
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
 
 Review comment:
   ```suggestion
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373454541
 
 

 ##########
 File path: tools/azure-pipelines/build-apache-repo.yml
 ##########
 @@ -0,0 +1,94 @@
+# 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.
+
+
+#
+# This file defines the Flink build for the "apache/flink" repository, including
+# the following:
+#  - PR builds
+#  - custom triggered e2e tests
+#  - nightly builds
+
+
+
+schedules:
+- cron: "0 0 * * *"
+  displayName: Daily midnight build
+  branches:
+    include:
+    - master
+  always: true # run even if there were no changes to the mentioned branches
+
+resources:
+  containers:
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
+  - container: flink-build-container
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
+  CACHE_FALLBACK_KEY: maven | $(Agent.OS)
+  CACHE_FLINK_DIR: $(Pipeline.Workspace)/flink_cache
+
+stages:
+  # CI / PR triggered stage:
+  - stage: ci_build
+    displayName: "CI Build (custom builders)"
+    condition: not(eq(variables['Build.Reason'], in('Schedule', 'Manual')))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: ci_build
+          test_pool_definition:
+            name: Default
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+
+  # Special stage for midnight builds:
+  - stage: cron_build_on_azure_os_free_pool
+    displayName: "Cron build on free Azure Resource Pool"
+    dependsOn: [] # depending on an empty array makes the stages run in parallel
+    condition: or(eq(variables['Build.Reason'], 'Schedule'), eq(variables['MODE'], 'nightly'))
+    jobs:
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_default
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.11"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_scala2_12
+          test_pool_definition:
+            vmImage: 'ubuntu-latest'
+          e2e_pool_definition:
+            vmImage: 'ubuntu-latest'
+          environment: PROFILE="-Dhadoop.version=2.8.3 -Dinclude_hadoop_aws -Dscala-2.12 -Phive-1.2.1"
+      - template: jobs-template.yml
+        parameters:
+          stage_name: cron_build_jdk11
 
 Review comment:
   separate PR, java 11 runs only on cron jobs anyway which are a follow-up in any case.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373459195
 
 

 ##########
 File path: flink-end-to-end-tests/run-nightly-tests.sh
 ##########
 @@ -88,8 +88,11 @@ run_test "Resuming Externalized Checkpoint after terminal failure (rocks, increm
 # Docker
 ################################################################################
 
-run_test "Running Kerberized YARN on Docker test (default input)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh"
-run_test "Running Kerberized YARN on Docker test (custom fs plugin)" "$END_TO_END_DIR/test-scripts/test_yarn_kerberos_docker.sh dummy-fs"
+# Ignore these tests on Azure
 
 Review comment:
   Ok --> https://issues.apache.org/jira/browse/FLINK-15833

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375838716
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,129 @@
+# 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.
+
+parameters:
+  test_pool_definition: # where is compiliation and unit test execution happening?
+  e2e_pool_definion: # where is e2e test execution happening?
 
 Review comment:
   let's not formulate comments as questions

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378389823
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_mesos_multiple_submissions.sh
 ##########
 @@ -29,7 +29,7 @@ TEST_PROGRAM_JAR=$END_TO_END_DIR/flink-cli-test/target/PeriodicStreamingJob.jar
 
 function submit_job {
     local output_path=$1
-    docker exec -it mesos-master bash -c "${FLINK_DIR}/bin/flink run -d -p 1 ${TEST_PROGRAM_JAR} --durationInSecond ${DURATION} --outputPath ${output_path}" \
+    docker exec mesos-master bash -c "${FLINK_DIR}/bin/flink run -d -p 1 ${TEST_PROGRAM_JAR} --durationInSecond ${DURATION} --outputPath ${output_path}" \
 
 Review comment:
   `docker exec -it` is basically for using Docker interactively (e.g. you want to run `bash` inside a container and talk to bash).
   `-i       Keep STDIN open even if not attached`
   `-t       Allocate a pseudo-TTY`
   
   It's just not necessary to pass `-it` (and IIRC it even logs a warning)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378844582
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/test_streaming_elasticsearch.sh
 ##########
 @@ -39,10 +39,31 @@ on_exit test_cleanup
 TEST_ES_JAR=${END_TO_END_DIR}/flink-elasticsearch${ELASTICSEARCH_VERSION}-test/target/Elasticsearch${ELASTICSEARCH_VERSION}SinkExample.jar
 
 # run the Flink job
-$FLINK_DIR/bin/flink run -p 1 $TEST_ES_JAR \
+JOB_ID=$($FLINK_DIR/bin/flink run -d -p 1 $TEST_ES_JAR \
   --numRecords 20 \
   --index index \
-  --type type
+  --type type | awk '{print $NF}' | tail -n 1)
 
+
+# wait for 10 seconds
+wait_job_submitted ${JOB_ID}
 
 Review comment:
   sounds like we can simplify this by checking whether the job has finished instead.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378347862
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373028672
 
 

 ##########
 File path: flink-end-to-end-tests/test-scripts/kafka-common.sh
 ##########
 @@ -67,10 +68,20 @@ function start_kafka_cluster {
   $KAFKA_DIR/bin/zookeeper-server-start.sh -daemon $KAFKA_DIR/config/zookeeper.properties
   $KAFKA_DIR/bin/kafka-server-start.sh -daemon $KAFKA_DIR/config/server.properties
 
+  start_time=$(date +%s)
   # zookeeper outputs the "Node does not exist" bit to stderr
   while [[ $($KAFKA_DIR/bin/zookeeper-shell.sh localhost:2181 get /brokers/ids/0 2>&1) =~ .*Node\ does\ not\ exist.* ]]; do
-    echo "Waiting for broker..."
-    sleep 1
+    current_time=$(date +%s)
+    time_diff=$((current_time - start_time))
+
+    if [ $time_diff -ge $MAX_RETRY_SECONDS ]; then
+        echo "Kafka cluster did not start after $MAX_RETRY_SECONDS seconds. Printing Kafka logs:"
 
 Review comment:
   belongs into separate commit

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r375878864
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
+
+variables:
+  MAVEN_CACHE_FOLDER: $(Pipeline.Workspace)/.m2/repository
+  MAVEN_OPTS: '-Dmaven.repo.local=$(MAVEN_CACHE_FOLDER)'
+  CACHE_KEY: maven | $(Agent.OS) | **/pom.xml, !**/target/**
 
 Review comment:
   I will add some comments to clarify.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378347602
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,193 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+
+cat << EOF > /tmp/az_settings.xml
 
 Review comment:
   the reason is still not documented

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378842487
 
 

 ##########
 File path: tools/azure-pipelines/jobs-template.yml
 ##########
 @@ -0,0 +1,137 @@
+# 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.
+
+parameters:
+  test_pool_definition: # defines the hardware pool for compilation and unit test execution.
+  e2e_pool_definion: # defines the hardware pool for end-to-end test execution
+  stage_name: # defines a unique identifier for all jobs in a stage (in case the jobs are added multiple times to a stage)
+  environment: # defines environment variables for downstream scripts
+
+jobs:
+- job: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all # this cleans the entire workspace directory before running a new job
+    # It is necessary because the custom build machines are reused for tests.
+    # See also https://docs.microsoft.com/en-us/azure/devops/pipelines/process/phases?view=azure-devops&tabs=yaml#workspace 
+
+  steps:
+  # The cache task is persisting the .m2 directory between builds, so that
+  # we do not have to re-download all dependencies from maven central for 
+  # each build. The hope is that downloading the cache is faster than
+  # all dependencies individually.
+  # In this configuration, we a hash over all committed (not generated) .pom files 
+  # as a key for the build cache (CACHE_KEY). If we have a cache miss on the hash
+  # (usually because a pom file has changed), we'll fall back to a key without
+  # the pom files (CACHE_FALLBACK_KEY).
+  # Offical documentation of the Cache task: https://docs.microsoft.com/en-us/azure/devops/pipelines/caching/?view=azure-devops
+  - task: Cache@2
+    inputs:
+      key: $(CACHE_KEY)
+      restoreKeys: $(CACHE_FALLBACK_KEY)
+      path: $(MAVEN_CACHE_FOLDER)
+    continueOnError: true # continue the build even if the cache fails.
+    displayName: Cache Maven local repo
+
+  # Compile
+  - script: STAGE=compile ${{parameters.environment}} ./tools/azure_controller.sh compile
+    displayName: Build
+
+  # upload artifacts for next stage
+  - task: PublishPipelineArtifact@1
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+- job: test_${{parameters.stage_name}}
+  dependsOn: compile_${{parameters.stage_name}}
+  condition: not(eq(variables['MODE'], 'e2e'))
+  pool: ${{parameters.test_pool_definition}}
+  container: flink-build-container
+  timeoutInMinutes: 240
+  cancelTimeoutInMinutes: 1
+  workspace:
+    clean: all
+  strategy:
+    matrix:
+      core:
+        module: core
+      python:
+        module: python
+      libraries:
+        module: libraries
+      blink_planner:
+        module: blink_planner
+      connectors:
+        module: connectors
+      kafka_gelly:
+        module: kafka/gelly
+      tests:
+        module: tests
+      legacy_scheduler_core:
+        module: legacy_scheduler_core
+      legacy_scheduler_tests:
+        module: legacy_scheduler_tests
+      misc:
+        module: misc
+  steps:
+
+  # download artifacts
+  - task: DownloadPipelineArtifact@2
+    inputs:
+      path: $(CACHE_FLINK_DIR)
+      artifact: FlinkCompileCacheDir-${{parameters.stage_name}}
+
+  # recreate "build-target" symlink for python tests
+  - script: |
+      ls -lisah $(CACHE_FLINK_DIR)
+      ls -lisah .
+      ln -snf $(CACHE_FLINK_DIR)/flink-dist/target/flink-*-SNAPSHOT-bin/flink-*-SNAPSHOT $(CACHE_FLINK_DIR)/build-target
+    displayName: Recreate 'build-target' symlink
+  # Test
+  - script: STAGE=test ${{parameters.environment}} ./tools/azure_controller.sh $(module)
+    displayName: Test - $(module)
+
+  - task: PublishTestResults@2
+    inputs:
+      testResultsFormat: 'JUnit'
+
+
+- job: e2e_${{parameters.stage_name}}
+  condition: eq(variables['MODE'], 'e2e')
+  # We are not running this job on a container, but in a VM.
 
 Review comment:
   Shouldn't we consider doing that now since it would make several changes in this PR unnecessary?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378968506
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,219 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
 
 Review comment:
   ```suggestion
         <url>https://maven-central.storage-download.googleapis.com/maven2/</url>
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   Hash:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/147715780 TriggerType:PUSH TriggerID:438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4
   Hash:5a81004ac5c7461921fa679bb1a428c155ae4a25 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148552963 TriggerType:PUSH TriggerID:5a81004ac5c7461921fa679bb1a428c155ae4a25
   Hash:4f4267a436da19e3216dddb529c5ace4c9d0efba Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148558147 TriggerType:PUSH TriggerID:4f4267a436da19e3216dddb529c5ace4c9d0efba
   Hash:0d869f3770aeb5cc565362e98d229003a827bca8 Status:CANCELED URL:https://travis-ci.com/flink-ci/flink/builds/148766925 TriggerType:PUSH TriggerID:0d869f3770aeb5cc565362e98d229003a827bca8
   Hash:56d01e5fd570d28bb86ff7ead98e90f664289bfe Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148771676 TriggerType:PUSH TriggerID:56d01e5fd570d28bb86ff7ead98e90f664289bfe
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/148988925 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:93946ab18cece030fa526fce3b7f34206fcca643 Status:FAILURE URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184 TriggerType:PUSH TriggerID:93946ab18cece030fa526fce3b7f34206fcca643
   Hash:2656304328dfe44278c4bcdd02e07ae9f2786747 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:2656304328dfe44278c4bcdd02e07ae9f2786747
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5225 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   Hash:af8c96151b79e85319ed0fa43a60cb58c8927f96 Status:SUCCESS URL:https://travis-ci.com/flink-ci/flink/builds/149168315 TriggerType:PUSH TriggerID:af8c96151b79e85319ed0fa43a60cb58c8927f96 
   Hash:da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a Status:SUCCESS URL:https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5223 TriggerType:PUSH TriggerID:da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a
   Hash:da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/149167200 TriggerType:PUSH TriggerID:da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a
   Hash:7c444b51f71427629a3865b65c49e884bb431a4c Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:7c444b51f71427629a3865b65c49e884bb431a4c
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   * 438cbcb4f6388f66b16e9b49f9e9c1f44dee89d4 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/147715780) 
   * 5a81004ac5c7461921fa679bb1a428c155ae4a25 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148552963) 
   * 4f4267a436da19e3216dddb529c5ace4c9d0efba Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148558147) 
   * 0d869f3770aeb5cc565362e98d229003a827bca8 Travis: [CANCELED](https://travis-ci.com/flink-ci/flink/builds/148766925) 
   * 56d01e5fd570d28bb86ff7ead98e90f664289bfe Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148771676) 
   * 93946ab18cece030fa526fce3b7f34206fcca643 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/148988925) Azure: [FAILURE](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5184) 
   * 2656304328dfe44278c4bcdd02e07ae9f2786747 UNKNOWN
   * af8c96151b79e85319ed0fa43a60cb58c8927f96 Travis: [SUCCESS](https://travis-ci.com/flink-ci/flink/builds/149168315) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5225) 
   * da5a1bf1b7ae7df4bcb90e0ddf46e7ea137d042a Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/149167200) Azure: [SUCCESS](https://dev.azure.com/rmetzger/5bd3ef0a-4359-41af-abca-811b04098d2e/_build/results?buildId=5223) 
   * 7c444b51f71427629a3865b65c49e884bb431a4c UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373028978
 
 

 ##########
 File path: tools/travis_watchdog.sh
 ##########
 @@ -25,6 +25,7 @@ if [ -z "$HERE" ] ; then
 	exit 1  # fail
 fi
 
+
 
 Review comment:
   revert

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:UNKNOWN URL:TBD TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r373030834
 
 

 ##########
 File path: azure-pipelines.yml
 ##########
 @@ -13,23 +13,44 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
+#
+# This file defines an Azure Pipeline build for testing Flink. It is intended to be used
+# with a free Azure Pipelines account.
+# It has the following features:
+#  - default builds for pushes / pull requests to a Flink fork and custom AZP account
+#  - end2end tests
+#
+#
+# For the "apache/flink" repository, we are using the pipeline definition located in
+#   tools/azure-pipelines/build-apache-repo.yml
+# That file points to custom, self-hosted build agents for faster pull request build processing and 
+# integration with Flinkbot.
+#
 
-trigger:
-  branches:
-    include:
-    - '*' 
 
 resources:
   containers:
-  # Container with Maven 3.2.5 to have the same environment everywhere.
+  # Container with Maven 3.2.5, SSL to have the same environment everywhere.
   - container: flink-build-container
-    image: rmetzger/flink-ci:3
-  repositories:
-    - repository: templates
-      type: github
-      name: flink-ci/flink-azure-builds
-      endpoint: flink-ci
+    image: rmetzger/flink-ci:ubuntu-jdk8-amd64-e005e00
 
 Review comment:
   why are we still pointing to your repo?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
rmetzger commented on a change in pull request #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#discussion_r378447373
 
 

 ##########
 File path: tools/azure_controller.sh
 ##########
 @@ -0,0 +1,220 @@
+#!/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.
+################################################################################
+
+echo $M2_HOME
+echo $PATH
+echo $MAVEN_OPTS
+
+mvn -version
+echo "Commit: $(git rev-parse HEAD)"
+
+# Set up a custom Maven settings file, configuring an Google-hosted maven central
+# mirror.
+cat << EOF > /tmp/az_settings.xml
+<settings>
+  <mirrors>
+    <mirror>
+      <id>google-maven-central</id>
+      <name>GCS Maven Central mirror</name>
+      <url>https://maven-central.storage-download.googleapis.com/repos/central/data/</url>
+      <mirrorOf>central</mirrorOf>
+    </mirror>
+  </mirrors>
+</settings>
+EOF
+
+
+HERE="`dirname \"$0\"`"             # relative
+HERE="`( cd \"$HERE\" && pwd )`"    # absolutized and normalized
+if [ -z "$HERE" ] ; then
+    # error; for some reason, the path is not accessible
+    # to the script (e.g. permissions re-evaled after suid)
+    exit 1  # fail
+fi
+
+source "${HERE}/travis/stage.sh"
+source "${HERE}/travis/shade.sh"
+
+print_system_info() {
+    echo "CPU information"
+    lscpu
+
+    echo "Memory information"
+    cat /proc/meminfo
+
+    echo "Disk information"
+    df -hH
+
+    echo "Running build as"
+    whoami
+}
+
+print_system_info
+
+
+STAGE=$1
+echo "Current stage: \"$STAGE\""
+
+EXIT_CODE=0
+
+#adding -Dmaven.wagon.http.pool=false (see https://developercommunity.visualstudio.com/content/problem/851041/microsoft-hosted-agents-run-into-maven-central-tim.html)
+# --settings /tmp/az_settings.xml 
 
 Review comment:
   I tried that already :( 
   Maven does not seem to try different mirrors when something is not available somewhere :(

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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

[GitHub] [flink] flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on issue #10976: [FLINK-13978][build system] Add experimental support for building on Azure Pipelines
URL: https://github.com/apache/flink/pull/10976#issuecomment-580319851
 
 
   <!--
   Meta data
   Hash:322bf186867ed0ec1a867ba7919e39e3c437aba0 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146777318 TriggerType:PUSH TriggerID:322bf186867ed0ec1a867ba7919e39e3c437aba0
   Hash:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Status:FAILURE URL:https://travis-ci.com/flink-ci/flink/builds/146897244 TriggerType:PUSH TriggerID:0e9afb2b74b214deabb3f09ddd06bca6c3d3b788
   -->
   ## CI report:
   
   * 322bf186867ed0ec1a867ba7919e39e3c437aba0 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146777318) 
   * 0e9afb2b74b214deabb3f09ddd06bca6c3d3b788 Travis: [FAILURE](https://travis-ci.com/flink-ci/flink/builds/146897244) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to 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