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/10/07 10:42:40 UTC

[GitHub] [flink] AHeise opened a new pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

AHeise opened a new pull request #13551:
URL: https://github.com/apache/flink/pull/13551


   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   With the larger refactoring of checkpoint alignment and the additional of more unaligned checkpoint settings, it becomes increasingly important to provide a large test coverage.
   
   Unfortunately, adding sufficient test cases in a test matrix appears to be unrealistic: many of the encountered issues were subtle, sometimes caused by race conditions or unusual test configurations and often only visible in e2e tests.
   
   Hence, we like to rely on all existing Flink tests to provide a sufficient coverage for checkpointing. However, as more and more options in unaligned checkpoint are going to be implemented in this and the upcoming release, running all Flink tests - especially e2e - in a test matrix is prohibitively expensive, even for nightly builds.
   
   Thus, we want to introduce test randomization for all tests that do not use a specific checkpointing mode. In a similar way, we switched from aligned checkpoints by default in tests to unaligned checkpoint during the last release cycle.
   
   To not burden the developers of other components too much, we set the following requirements:
   
   -     Randomization should be seeded in a way that both builds on Azure pipelines and local builds will result in the same settings to ease debugging and ensure reproducibility.
   -     Randomized options should be shown in the test log.
   -     Execution order of test cases will not influence the randomization.
   -     Randomization is hidden, no change on any test is needed.
   -     Randomization only happens during local/remote test execution. User deployments are not affected.
   -     Test developers are able to avoid randomization by explicitly providing checkpoint configs.
   
   ## Brief change log
   
   - Add generic method to choose random values for checkpoint config.
   - Use it for uninitialized "execution.checkpointing.unaligned".
   - Adds flag to enable randomization in root pom. User code will not trigger randomization but chooses constant default values.
   
   ## Verifying this change
   
   - Not covered (yet)
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4a2942d056be51f8064a94bfb064143decda77db Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321) 
   * 07e1b12f433dec764812f49b9c0348d25498d6f6 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



[GitHub] [flink] pnowojski commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
pnowojski commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741675386


   > Can we overcome it by reading some system property at the time of ConfigOption creation?
   
   We didn't like this idea, as it would require to modify the production code.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051






----------------------------------------------------------------
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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r568358724



##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
##########
@@ -41,6 +41,8 @@
  * startup options, or the JVM version.
  */
 public class EnvironmentInformation {
+    public static final String UNKNOWN_COMMIT_ID = "DecafC0ffeeD0d0F00d";
+    public static final String UNKNOWN_COMMIT_ID_ABBREV = "DeadD0d0";

Review comment:
       Technically you are correct - the best kind of correct. Fixed 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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r553530514



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -44,6 +47,12 @@ public TestStreamEnvironment(
 				null);
 
 		setParallelism(parallelism);
+
+		if (Randomization) {
+			final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       I'm missing a hook where I can mutate the configuration per test method. `MiniClusterResource` is mostly used on class level. Any guidance is greatly appreciated.




----------------------------------------------------------------
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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r570610177



##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.configuration.CheckpointingOptions.SAVEPOINT_DIRECTORY;
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.TOLERABLE_FAILURE_NUMBER;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {

Review comment:
       extend TestLogger




----------------------------------------------------------------
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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r540528885



##########
File path: flink-test-utils-parent/flink-test-utils-junit/src/main/java/org/apache/flink/util/TestLogger.java
##########
@@ -67,6 +67,9 @@ public void failed(Throwable e, Description description) {
 		}
 	};
 
+	@Rule
+	public TestRule nameProvider = new TestNameProvider();

Review comment:
       ```suggestion
   	public final TestRule nameProvider = new TestNameProvider();
   ```




----------------------------------------------------------------
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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r568125836



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe

Review comment:
       What part of this class is not thread safe?

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);
+        T value = alternatives[choice];
+        LOG.info("Randomly selected {} for {}", value, option.key());
+        configuration.set(option, value);
+    }
+
+    public static PseudoRandomValueSelector create(Object entryPointSeed) {
+        final long combinedSeed = GlobalSeed | entryPointSeed.hashCode();
+        final Random random = new Random(combinedSeed);
+        return new PseudoRandomValueSelector(random::nextInt);
+    }
+
+    private static String getGlobalSeed() {
+        // manual seed or set by maven
+        final String seed = System.getProperty("test.randomization.seed");
+        if (seed != null) {
+            return seed;
+        }
+
+        // Read with git command (if installed)
+        final Optional<String> gitCommitId = getGitCommitId();
+        if (gitCommitId.isPresent()) {
+            return gitCommitId.get();
+        }
+
+        // try EnvironmentInformation, which is set in the maven process
+        final String commitId = EnvironmentInformation.getGitCommitId();
+        if (!commitId.equals(EnvironmentInformation.UNKNOWN_COMMIT_ID)) {
+            return commitId;
+        }
+
+        LOG.warn(
+                "Cannot initialize maven property test.randomization.seed with commit id, please set manually to receive reproducible builds.");
+        // return any constant
+        return "";
+    }
+
+    @VisibleForTesting
+    static Optional<String> getGitCommitId() {
+        try {
+            Process process = new ProcessBuilder("git", "rev-parse", "HEAD").start();
+            try (InputStream input = process.getInputStream()) {
+                final String commit = IOUtils.toString(input, Charset.defaultCharset()).trim();
+                if (commit.matches("[a-f0-9]{40}")) return Optional.of(commit);

Review comment:
       braces! (I hope this will fail the CI, if not we might have to re-enable some checkstyle rules...)

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
##########
@@ -41,6 +41,8 @@
  * startup options, or the JVM version.
  */
 public class EnvironmentInformation {
+    public static final String UNKNOWN_COMMIT_ID = "DecafC0ffeeD0d0F00d";
+    public static final String UNKNOWN_COMMIT_ID_ABBREV = "DeadD0d0";

Review comment:
       technically speaking these should be  `@VisibleForTesting`.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07e1b12f433dec764812f49b9c0348d25498d6f6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931",
       "triggerID" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d69b21f05abca0de64a2e778c292a7d0694b263",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13184",
       "triggerID" : "3d69b21f05abca0de64a2e778c292a7d0694b263",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 UNKNOWN
   * 3d69b21f05abca0de64a2e778c292a7d0694b263 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13184) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4c3a8b4fa089a223d1d90476068a7ef3111d5bfa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757) 
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * df7d30894d9bf8f2d5ab361d2c0437d5aec065f9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761) 
   
   <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



[GitHub] [flink] AHeise commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-776707331


   AZP failure unrelated (https://issues.apache.org/jira/browse/FLINK-21103), merging.


----------------------------------------------------------------
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



[GitHub] [flink] rkhachatryan edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
rkhachatryan edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741691622


   > various drawbacks
   
   Could you elaborate?


----------------------------------------------------------------
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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r556428858



##########
File path: flink-runtime/pom.xml
##########
@@ -509,6 +509,16 @@ under the License.
 						<!-- It is useless due to the way Flink does branches and tags -->
 						<skip>true</skip>
 					</gitDescribe>
+					<!-- The git.properties is not really used, but works around an IntelliJ bug where

Review comment:
       Why is it important for this to work in IntelliJ?

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -69,13 +74,28 @@ public static void setAsContext(
                     TestStreamEnvironment env =
                             new TestStreamEnvironment(
                                     miniCluster, parallelism, jarFiles, classpaths);
+                    randomize(conf);
                     env.configure(conf, env.getUserClassloader());
                     return env;
                 };
 
         initializeContextEnvironment(factory);
     }
 
+    /**
+     * Randomizes configuration on test case level even if mini cluster is used in a class rule.
+     *
+     * @param conf
+     */
+    private static void randomize(Configuration conf) {
+        if (Randomization) {
+            final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       hmm...like, we _could_ add a hook to setAsContext to pass the name/modify the config/pass a factory, but it'd still suck since the TestNameProvider must live within the `TestLogger`; we just can't get around that singleton access can we...




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fded2a71d2064276f69335ec94c36b35b8789985 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807) 
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a88c6caabb50541ecdc8fcafa232adf0e799bbbb Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538) 
   
   <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



[GitHub] [flink] pnowojski commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
pnowojski commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741698338


   For one thing, that's a public api, so we wouldn't be able to easily change it. We could maybe mark it partially as `@Internal`, but I don't know how to do it in a clear way. Additionally it could just create confusion for the users, in the worst case they can start using it. Apart from that, if you need to modify production code for testing purposes, it usually means something is wrong with the design (having a testing mini cluster that randomly sets configuration options is a cleaner design). And also relaying on the `System.getProperty` is not the cleanest way.


----------------------------------------------------------------
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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r539316185



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if unset.
+ */
+@Internal
+class PseudoRandomValueSelector {
+	private final Function<Integer, Integer> randomValueSupplier;
+
+	private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+	private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+		this.randomValueSupplier = randomValueSupplier;
+	}
+
+	public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+		if (configuration.contains(option)) {
+			return;
+		}
+		final Integer choice = randomValueSupplier.apply(alternatives.length);
+		T value = alternatives[choice];
+		LOG.info("Randomly selected {} for {}", value, option.key());
+		configuration.set(option, value);
+	}
+
+	public static PseudoRandomValueSelector create(Object entryPointSeed) {
+		final long commitSeed = new BigInteger(EnvironmentInformation.getGitCommitId(), 16).hashCode();
+		final long combinedSeed = commitSeed << 32 | entryPointSeed.hashCode();

Review comment:
       Yes that was my intent: have it reproducible without changing anything by just checking out the particular commit that failed on AZP.
   
   I'm double-checking that `EnvironmentInformation.getGitCommitId()` works as expected or try a different approach to inject commit id.




----------------------------------------------------------------
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



[GitHub] [flink] pnowojski edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
pnowojski edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741675386


   > Can we overcome it by reading some system property at the time of ConfigOption creation?
   
   We didn't like this idea, as it would require to modify the production code (with various drawbacks as the result).


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775) 
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 UNKNOWN
   * 29e7176390cd2877927bb8a20ac327921e8ae936 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a88c6caabb50541ecdc8fcafa232adf0e799bbbb Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538) 
   * fded2a71d2064276f69335ec94c36b35b8789985 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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r553934486



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if unset.
+ */
+@Internal
+class PseudoRandomValueSelector {
+	private final Function<Integer, Integer> randomValueSupplier;
+
+	private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+	private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+		this.randomValueSupplier = randomValueSupplier;
+	}
+
+	public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+		if (configuration.contains(option)) {
+			return;
+		}
+		final Integer choice = randomValueSupplier.apply(alternatives.length);
+		T value = alternatives[choice];
+		LOG.info("Randomly selected {} for {}", value, option.key());
+		configuration.set(option, value);
+	}
+
+	public static PseudoRandomValueSelector create(Object entryPointSeed) {
+		final long commitSeed = new BigInteger(EnvironmentInformation.getGitCommitId(), 16).hashCode();
+		final long combinedSeed = commitSeed << 32 | entryPointSeed.hashCode();

Review comment:
       So I verified that randomized unaligned checkpoint configuration is actually picked up while creating the controllers.

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -44,6 +47,12 @@ public TestStreamEnvironment(
 				null);
 
 		setParallelism(parallelism);
+
+		if (Randomization) {
+			final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       I moved it a bit outside into the factory, but that's the best that I came up with.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a875c6b63432a371dd2928d6c3bdb2d4627c5f81 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296) 
   * 4a2942d056be51f8064a94bfb064143decda77db 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07e1b12f433dec764812f49b9c0348d25498d6f6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533) 
   * a88c6caabb50541ecdc8fcafa232adf0e799bbbb 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b037bbc8fd11fce426527e4caefccd075ddc4b6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931",
       "triggerID" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775) 
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 UNKNOWN
   * 29e7176390cd2877927bb8a20ac327921e8ae936 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931) 
   
   <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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r540530506



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -44,6 +47,12 @@ public TestStreamEnvironment(
 				null);
 
 		setParallelism(parallelism);
+
+		if (Randomization) {
+			final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       This seems a bit...janky? Why can we not mutate the configuration within the MiniClusterResource?




----------------------------------------------------------------
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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r568358199



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe

Review comment:
       It doesn't adhere to the contract that it will produce the same randomized configuration for the same seeds if you start using it in parallel (the state is hidden in `Random` itself).

##########
File path: flink-runtime/src/main/java/org/apache/flink/runtime/util/EnvironmentInformation.java
##########
@@ -41,6 +41,8 @@
  * startup options, or the JVM version.
  */
 public class EnvironmentInformation {
+    public static final String UNKNOWN_COMMIT_ID = "DecafC0ffeeD0d0F00d";
+    public static final String UNKNOWN_COMMIT_ID_ABBREV = "DeadD0d0";

Review comment:
       Technically you are correct - the best kind of correct. Fixed 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe4a6228d31108e71a6e9faab97598ffa9a09b8b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268) 
   * a875c6b63432a371dd2928d6c3bdb2d4627c5f81 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296) 
   
   <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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r570070905



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);
+        T value = alternatives[choice];
+        LOG.info("Randomly selected {} for {}", value, option.key());
+        configuration.set(option, value);
+    }
+
+    public static PseudoRandomValueSelector create(Object entryPointSeed) {
+        final long combinedSeed = GlobalSeed | entryPointSeed.hashCode();

Review comment:
       ```suggestion
           final long combinedSeed = GLOBAL_SEED | entryPointSeed.hashCode();
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);

Review comment:
       ```suggestion
           final int choice = randomValueSupplier.apply(alternatives.length);
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */
+    @Test
+    public void testStableRandomization() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test");
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));

Review comment:
       maybe add another set for the second value, to test that a sequence of selections is also deterministic.

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());

Review comment:
       assertThat(uniqueValues.size(); greaterThan(1))?

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull

Review comment:
       ```suggestion
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */
+    @Test
+    public void testStableRandomization() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test");
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertEquals(1, uniqueValues.size());
+    }
+
+    /**
+     * Tests that reading through git command yields the same as {@link EnvironmentInformation}.
+     *
+     * <p>This test assumes that both sources of information are available (CI).
+     */
+    @Test
+    public void readCommitId() {
+        assumeNotNull(System.getProperty("CI"));

Review comment:
       re-use `org.apache.flink.runtime.testutils.ZooKeeperTestUtils#runsOnCIInfrastructure` instead

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -69,13 +74,30 @@ public static void setAsContext(
                     TestStreamEnvironment env =
                             new TestStreamEnvironment(
                                     miniCluster, parallelism, jarFiles, classpaths);
+                    randomize(conf);
                     env.configure(conf, env.getUserClassloader());
                     return env;
                 };
 
         initializeContextEnvironment(factory);
     }
 
+    /**
+     * Randomizes configuration on test case level even if mini cluster is used in a class rule.
+     *
+     * <p>Note that only unset properties are randomized.
+     *
+     * @param conf the configuration to randomize
+     */
+    private static void randomize(Configuration conf) {
+        if (Randomization) {

Review comment:
       ```suggestion
           if (RANDOMIZE_CHECKPOINTING_CONFIG) {
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);

Review comment:
       ```suggestion
       private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
   
       private final Function<Integer, Integer> randomValueSupplier;
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");

Review comment:
       ```suggestion
           final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("seed");
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);
+        T value = alternatives[choice];
+        LOG.info("Randomly selected {} for {}", value, option.key());
+        configuration.set(option, value);
+    }
+
+    public static PseudoRandomValueSelector create(Object entryPointSeed) {
+        final long combinedSeed = GlobalSeed | entryPointSeed.hashCode();
+        final Random random = new Random(combinedSeed);
+        return new PseudoRandomValueSelector(random::nextInt);
+    }
+
+    private static String getGlobalSeed() {
+        // manual seed or set by maven
+        final String seed = System.getProperty("test.randomization.seed");
+        if (seed != null) {
+            return seed;
+        }
+
+        // Read with git command (if installed)
+        final Optional<String> gitCommitId = getGitCommitId();
+        if (gitCommitId.isPresent()) {
+            return gitCommitId.get();
+        }
+
+        // try EnvironmentInformation, which is set in the maven process
+        final String commitId = EnvironmentInformation.getGitCommitId();
+        if (!commitId.equals(EnvironmentInformation.UNKNOWN_COMMIT_ID)) {
+            return commitId;
+        }
+
+        LOG.warn(
+                "Cannot initialize maven property test.randomization.seed with commit id, please set manually to receive reproducible builds.");

Review comment:
       ```
   Test randomization was enabled but test.randomization.seed was not configured, nor could the commit hash be retrieved from git or the EnvironmentInformation. Please set the test.randomization.seed property manually to make the build reproducible.
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -18,18 +18,23 @@
 
 package org.apache.flink.streaming.util;
 
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory;
 import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader;
+import org.apache.flink.util.TestNameProvider;
 
 import java.net.URL;
 import java.util.Collection;
 import java.util.Collections;
 
 /** A {@link StreamExecutionEnvironment} that executes its jobs on {@link MiniCluster}. */
 public class TestStreamEnvironment extends StreamExecutionEnvironment {
+    private static final boolean Randomization =

Review comment:
       ```suggestion
       private static final boolean RANDOMIZE_CHECKPOINTING_CONFIG =
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */

Review comment:
       ```suggestion
       /** Tests that the selector produces the same value for the same seed. */
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;

Review comment:
       ```suggestion
       private static final long GLOBAL_SEED = (long) getGlobalSeed().hashCode() << 32;
   ```




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * df7d30894d9bf8f2d5ab361d2c0437d5aec065f9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761) 
   * 0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 07e1b12f433dec764812f49b9c0348d25498d6f6 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533) 
   * a88c6caabb50541ecdc8fcafa232adf0e799bbbb Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538) 
   
   <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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r557270935



##########
File path: flink-runtime/pom.xml
##########
@@ -509,6 +509,16 @@ under the License.
 						<!-- It is useless due to the way Flink does branches and tags -->
 						<skip>true</skip>
 					</gitDescribe>
+					<!-- The git.properties is not really used, but works around an IntelliJ bug where

Review comment:
       I had mixed success when executing tests in IntelliJ. 
   `EnvironmentInformation.getCommitId` depends on successful resource filtering of `.flink-runtime.version.properties` in `flink-runtime`, which worked for me fine with `mvn`.
   However, in IntelliJ, quite often the `git.*` variables have not been replaced successfully. What's worse they have been unset even after having executed the task in `mvn resources -pl flink-runtime` successfully first.
   
   I did not have a good explanation so I added some code to generate the `git.properties` first and then merge it into `.flink-runtime.version.properties`. I noticed that generating `git.properties` worked very reliably. However, something unexpected happened: `.flink-runtime.version.properties` was also now properly filtered. 🤷 
   
   Since this is a bit hacky, an alternative could be to use two files to fill in the `EnvironmentInformation`: `git.properties` for git stuff and for `.flink-runtime.version.properties` for everything else.




----------------------------------------------------------------
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



[GitHub] [flink] pnowojski commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
pnowojski commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741859327


   > But I assume you want me to check for unintentional overrides.
   
   Yes, exactly.


----------------------------------------------------------------
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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r570128810



##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */
+    @Test
+    public void testStableRandomization() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test");
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));

Review comment:
       I'm checking a triple of config options 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775) 
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 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



[GitHub] [flink] flinkbot commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704852805


   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 5b037bbc8fd11fce426527e4caefccd075ddc4b6 (Wed Oct 07 10:44:49 UTC 2020)
   
   **Warnings:**
    * **1 pom.xml files were touched**: Check for build and licensing issues.
    * 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a875c6b63432a371dd2928d6c3bdb2d4627c5f81 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296) 
   
   <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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r553934486



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if unset.
+ */
+@Internal
+class PseudoRandomValueSelector {
+	private final Function<Integer, Integer> randomValueSupplier;
+
+	private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+	private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+		this.randomValueSupplier = randomValueSupplier;
+	}
+
+	public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+		if (configuration.contains(option)) {
+			return;
+		}
+		final Integer choice = randomValueSupplier.apply(alternatives.length);
+		T value = alternatives[choice];
+		LOG.info("Randomly selected {} for {}", value, option.key());
+		configuration.set(option, value);
+	}
+
+	public static PseudoRandomValueSelector create(Object entryPointSeed) {
+		final long commitSeed = new BigInteger(EnvironmentInformation.getGitCommitId(), 16).hashCode();
+		final long combinedSeed = commitSeed << 32 | entryPointSeed.hashCode();

Review comment:
       So I verified that randomized unaligned checkpoint configuration is actually picked up while creating the controllers.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fded2a71d2064276f69335ec94c36b35b8789985 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe4a6228d31108e71a6e9faab97598ffa9a09b8b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268) 
   * a875c6b63432a371dd2928d6c3bdb2d4627c5f81 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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r557279916



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -69,13 +74,28 @@ public static void setAsContext(
                     TestStreamEnvironment env =
                             new TestStreamEnvironment(
                                     miniCluster, parallelism, jarFiles, classpaths);
+                    randomize(conf);
                     env.configure(conf, env.getUserClassloader());
                     return env;
                 };
 
         initializeContextEnvironment(factory);
     }
 
+    /**
+     * Randomizes configuration on test case level even if mini cluster is used in a class rule.
+     *
+     * @param conf
+     */
+    private static void randomize(Configuration conf) {
+        if (Randomization) {
+            final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       yeah I'd revisit it once junit5 is in.
   




----------------------------------------------------------------
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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r568358199



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe

Review comment:
       It doesn't adhere to the contract that it will produce the same randomized configuration for the same seeds if you start using it in parallel (the state is hidden in `Random` itself).




----------------------------------------------------------------
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



[GitHub] [flink] rkhachatryan commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741691622


   > various drawbacks
   Could you elaborate?


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b037bbc8fd11fce426527e4caefccd075ddc4b6 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262) 
   * fe4a6228d31108e71a6e9faab97598ffa9a09b8b Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4a2942d056be51f8064a94bfb064143decda77db Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931",
       "triggerID" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 UNKNOWN
   * 29e7176390cd2877927bb8a20ac327921e8ae936 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174) 
   * 4c3a8b4fa089a223d1d90476068a7ef3111d5bfa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757) 
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * df7d30894d9bf8f2d5ab361d2c0437d5aec065f9 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051






----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * a88c6caabb50541ecdc8fcafa232adf0e799bbbb Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538) 
   * fded2a71d2064276f69335ec94c36b35b8789985 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807) 
   
   <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



[GitHub] [flink] pnowojski commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
pnowojski commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r539175307



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,61 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.math.BigInteger;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if unset.
+ */
+@Internal
+class PseudoRandomValueSelector {
+	private final Function<Integer, Integer> randomValueSupplier;
+
+	private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+	private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+		this.randomValueSupplier = randomValueSupplier;
+	}
+
+	public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+		if (configuration.contains(option)) {
+			return;
+		}
+		final Integer choice = randomValueSupplier.apply(alternatives.length);
+		T value = alternatives[choice];
+		LOG.info("Randomly selected {} for {}", value, option.key());
+		configuration.set(option, value);
+	}
+
+	public static PseudoRandomValueSelector create(Object entryPointSeed) {
+		final long commitSeed = new BigInteger(EnvironmentInformation.getGitCommitId(), 16).hashCode();
+		final long combinedSeed = commitSeed << 32 | entryPointSeed.hashCode();

Review comment:
       So for the given commit id and given test name, the random selection choices will be always the same, on each run on any environment/machine?
   
   If so, maybe put that (^^^) as a java doc for this `create` method?




----------------------------------------------------------------
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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r553544866



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -44,6 +47,12 @@ public TestStreamEnvironment(
 				null);
 
 		setParallelism(parallelism);
+
+		if (Randomization) {
+			final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       I see. I can't come up with a good alternative unfortunately.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b037bbc8fd11fce426527e4caefccd075ddc4b6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262) 
   * fe4a6228d31108e71a6e9faab97598ffa9a09b8b 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174) 
   * 4c3a8b4fa089a223d1d90476068a7ef3111d5bfa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757) 
   * 019646dd12750ec211a543f858836637a164d845 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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r570128810



##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */
+    @Test
+    public void testStableRandomization() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test");
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));

Review comment:
       I'm checking a triple of config options 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



[GitHub] [flink] AHeise merged pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise merged pull request #13551:
URL: https://github.com/apache/flink/pull/13551


   


----------------------------------------------------------------
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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r557266156



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -69,13 +74,28 @@ public static void setAsContext(
                     TestStreamEnvironment env =
                             new TestStreamEnvironment(
                                     miniCluster, parallelism, jarFiles, classpaths);
+                    randomize(conf);
                     env.configure(conf, env.getUserClassloader());
                     return env;
                 };
 
         initializeContextEnvironment(factory);
     }
 
+    /**
+     * Randomizes configuration on test case level even if mini cluster is used in a class rule.
+     *
+     * @param conf
+     */
+    private static void randomize(Configuration conf) {
+        if (Randomization) {
+            final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       To make it independent of `TestLogger` we could use [RunListener](https://junit.org/junit4/javadoc/4.12/org/junit/runner/notification/RunListener.html) and then have a singleton listener to fetch the name.
   However, I didn't go fully into this direction as it's quite a bit of fiddling in junit4 and I hope to eventually migrate to junit 5. Then I'd check if it can be done in a better way.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931",
       "triggerID" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d69b21f05abca0de64a2e778c292a7d0694b263",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "3d69b21f05abca0de64a2e778c292a7d0694b263",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 UNKNOWN
   * 29e7176390cd2877927bb8a20ac327921e8ae936 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931) 
   * 3d69b21f05abca0de64a2e778c292a7d0694b263 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



[GitHub] [flink] AHeise commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r553934770



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -44,6 +47,12 @@ public TestStreamEnvironment(
 				null);
 
 		setParallelism(parallelism);
+
+		if (Randomization) {
+			final String testName = TestNameProvider.getCurrentTestName();

Review comment:
       I moved it a bit outside into the factory, but that's the best that I came up with.




----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174) 
   * 4c3a8b4fa089a223d1d90476068a7ef3111d5bfa Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757) 
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * df7d30894d9bf8f2d5ab361d2c0437d5aec065f9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174) 
   * 4c3a8b4fa089a223d1d90476068a7ef3111d5bfa 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051






----------------------------------------------------------------
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



[GitHub] [flink] flinkbot commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 5b037bbc8fd11fce426527e4caefccd075ddc4b6 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



[GitHub] [flink] AHeise commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-741781038


   > but via for example adding a temporary log message somewhere in the `StreamTask` or network stack what alignment option was used after all? I mean there could be a potential problem where for some (or all?) tests, the randomly selected value is still unintentionally overridden to some static value, yielding this change a no-op.
   
   Good point, I have actually completely relied on the log message to check for the result. I'd repeat your suggestion.
   
   Btw, it is my intention that an explicit value for the random options always take precedence. But I assume you want me to check for _unintentional_ overrides.


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4a2942d056be51f8064a94bfb064143decda77db Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321) 
   * 07e1b12f433dec764812f49b9c0348d25498d6f6 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533) 
   
   <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



[GitHub] [flink] zentol commented on a change in pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #13551:
URL: https://github.com/apache/flink/pull/13551#discussion_r570070905



##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);
+        T value = alternatives[choice];
+        LOG.info("Randomly selected {} for {}", value, option.key());
+        configuration.set(option, value);
+    }
+
+    public static PseudoRandomValueSelector create(Object entryPointSeed) {
+        final long combinedSeed = GlobalSeed | entryPointSeed.hashCode();

Review comment:
       ```suggestion
           final long combinedSeed = GLOBAL_SEED | entryPointSeed.hashCode();
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);

Review comment:
       ```suggestion
           final int choice = randomValueSupplier.apply(alternatives.length);
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */
+    @Test
+    public void testStableRandomization() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test");
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));

Review comment:
       maybe add another set for the second value, to test that a sequence of selections is also deterministic.

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());

Review comment:
       assertThat(uniqueValues.size(); greaterThan(1))?

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull

Review comment:
       ```suggestion
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */
+    @Test
+    public void testStableRandomization() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test");
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertEquals(1, uniqueValues.size());
+    }
+
+    /**
+     * Tests that reading through git command yields the same as {@link EnvironmentInformation}.
+     *
+     * <p>This test assumes that both sources of information are available (CI).
+     */
+    @Test
+    public void readCommitId() {
+        assumeNotNull(System.getProperty("CI"));

Review comment:
       re-use `org.apache.flink.runtime.testutils.ZooKeeperTestUtils#runsOnCIInfrastructure` instead

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -69,13 +74,30 @@ public static void setAsContext(
                     TestStreamEnvironment env =
                             new TestStreamEnvironment(
                                     miniCluster, parallelism, jarFiles, classpaths);
+                    randomize(conf);
                     env.configure(conf, env.getUserClassloader());
                     return env;
                 };
 
         initializeContextEnvironment(factory);
     }
 
+    /**
+     * Randomizes configuration on test case level even if mini cluster is used in a class rule.
+     *
+     * <p>Note that only unset properties are randomized.
+     *
+     * @param conf the configuration to randomize
+     */
+    private static void randomize(Configuration conf) {
+        if (Randomization) {

Review comment:
       ```suggestion
           if (RANDOMIZE_CHECKPOINTING_CONFIG) {
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);

Review comment:
       ```suggestion
       private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
   
       private final Function<Integer, Integer> randomValueSupplier;
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");

Review comment:
       ```suggestion
           final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("seed");
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;
+
+    private PseudoRandomValueSelector(Function<Integer, Integer> randomValueSupplier) {
+        this.randomValueSupplier = randomValueSupplier;
+    }
+
+    public <T> void select(Configuration configuration, ConfigOption<T> option, T... alternatives) {
+        if (configuration.contains(option)) {
+            return;
+        }
+        final Integer choice = randomValueSupplier.apply(alternatives.length);
+        T value = alternatives[choice];
+        LOG.info("Randomly selected {} for {}", value, option.key());
+        configuration.set(option, value);
+    }
+
+    public static PseudoRandomValueSelector create(Object entryPointSeed) {
+        final long combinedSeed = GlobalSeed | entryPointSeed.hashCode();
+        final Random random = new Random(combinedSeed);
+        return new PseudoRandomValueSelector(random::nextInt);
+    }
+
+    private static String getGlobalSeed() {
+        // manual seed or set by maven
+        final String seed = System.getProperty("test.randomization.seed");
+        if (seed != null) {
+            return seed;
+        }
+
+        // Read with git command (if installed)
+        final Optional<String> gitCommitId = getGitCommitId();
+        if (gitCommitId.isPresent()) {
+            return gitCommitId.get();
+        }
+
+        // try EnvironmentInformation, which is set in the maven process
+        final String commitId = EnvironmentInformation.getGitCommitId();
+        if (!commitId.equals(EnvironmentInformation.UNKNOWN_COMMIT_ID)) {
+            return commitId;
+        }
+
+        LOG.warn(
+                "Cannot initialize maven property test.randomization.seed with commit id, please set manually to receive reproducible builds.");

Review comment:
       ```
   Test randomization was enabled but test.randomization.seed was not configured, nor could the commit hash be retrieved from git or the EnvironmentInformation. Please set the test.randomization.seed property manually to make the build reproducible.
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/TestStreamEnvironment.java
##########
@@ -18,18 +18,23 @@
 
 package org.apache.flink.streaming.util;
 
+import org.apache.flink.configuration.Configuration;
 import org.apache.flink.core.fs.Path;
 import org.apache.flink.runtime.minicluster.MiniCluster;
+import org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
 import org.apache.flink.streaming.api.environment.StreamExecutionEnvironmentFactory;
 import org.apache.flink.test.util.MiniClusterPipelineExecutorServiceLoader;
+import org.apache.flink.util.TestNameProvider;
 
 import java.net.URL;
 import java.util.Collection;
 import java.util.Collections;
 
 /** A {@link StreamExecutionEnvironment} that executes its jobs on {@link MiniCluster}. */
 public class TestStreamEnvironment extends StreamExecutionEnvironment {
+    private static final boolean Randomization =

Review comment:
       ```suggestion
       private static final boolean RANDOMIZE_CHECKPOINTING_CONFIG =
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import javax.annotation.Nonnull;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {
+
+    /**
+     * Tests that the selector will return different values if invoked several times even for the
+     * same option.
+     */
+    @Test
+    public void testRandomizationOfValues() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final PseudoRandomValueSelector valueSelector = PseudoRandomValueSelector.create("test");
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final Duration selectedValue =
+                    selectValue(valueSelector, CHECKPOINTING_INTERVAL, alternatives);
+            uniqueValues.add(selectedValue);
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    @Nonnull
+    private <T> T selectValue(
+            PseudoRandomValueSelector valueSelector, ConfigOption<T> option, T... alternatives) {
+        final Configuration configuration = new Configuration();
+        assertNull(configuration.get(option));
+        valueSelector.select(configuration, option, alternatives);
+        final T selected = configuration.get(option);
+        assertNotNull(selected);
+        return selected;
+    }
+
+    /** Tests that the selector will return different values for different seeds. */
+    @Test
+    public void testRandomizationWithSeed() {
+        final Duration[] alternatives =
+                IntStream.range(0, 1000).boxed().map(Duration::ofMillis).toArray(Duration[]::new);
+
+        final Set<Duration> uniqueValues = new HashSet<>(1);
+        for (int i = 0; i < 100; i++) {
+            final PseudoRandomValueSelector selector = PseudoRandomValueSelector.create("test" + i);
+            uniqueValues.add(selectValue(selector, CHECKPOINTING_INTERVAL, alternatives));
+        }
+        assertNotEquals(1, uniqueValues.size());
+    }
+
+    /** Tests that it produces the same value for the same seed. */

Review comment:
       ```suggestion
       /** Tests that the selector produces the same value for the same seed. */
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/main/java/org/apache/flink/streaming/util/PseudoRandomValueSelector.java
##########
@@ -0,0 +1,124 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import net.jcip.annotations.NotThreadSafe;
+import org.apache.commons.io.IOUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.Charset;
+import java.util.Optional;
+import java.util.Random;
+import java.util.function.Function;
+
+/**
+ * Initializes the {@link Configuration} for particular {@link ConfigOption}s with random values if
+ * unset.
+ *
+ * <p>With the same seed, the same values are always selected if the {@link #select(Configuration,
+ * ConfigOption, Object[])} invocation happens in the same order. A different seed should select
+ * different values.
+ *
+ * <p>The seed is calculated from a global seed (~unique per build) and a seed specific to test
+ * cases. Thus, two different builds will mostly result in different values for the same test case.
+ * Similarly, two test cases in the same build will have different randomized values.
+ *
+ * <p>The seed can be set with the maven/system property test.randomization.seed and is set by
+ * default to commit id. If the seed is empty, {@link EnvironmentInformation} and as a last fallback
+ * git command is used to retrieve the commit id.
+ */
+@Internal
+@NotThreadSafe
+class PseudoRandomValueSelector {
+    private final Function<Integer, Integer> randomValueSupplier;
+
+    private static final Logger LOG = LoggerFactory.getLogger(PseudoRandomValueSelector.class);
+
+    private static final long GlobalSeed = (long) getGlobalSeed().hashCode() << 32;

Review comment:
       ```suggestion
       private static final long GLOBAL_SEED = (long) getGlobalSeed().hashCode() << 32;
   ```

##########
File path: flink-test-utils-parent/flink-test-utils/src/test/java/org/apache/flink/streaming/util/PseudoRandomValueSelectorTest.java
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.
+ */
+
+package org.apache.flink.streaming.util;
+
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.runtime.testutils.ZooKeeperTestUtils;
+import org.apache.flink.runtime.util.EnvironmentInformation;
+
+import org.junit.Test;
+
+import java.io.IOException;
+import java.time.Duration;
+import java.util.HashSet;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.IntStream;
+
+import static org.apache.flink.configuration.CheckpointingOptions.SAVEPOINT_DIRECTORY;
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.CHECKPOINTING_INTERVAL;
+import static org.apache.flink.streaming.api.environment.ExecutionCheckpointingOptions.TOLERABLE_FAILURE_NUMBER;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assume.assumeFalse;
+import static org.junit.Assume.assumeNoException;
+import static org.junit.Assume.assumeNotNull;
+
+/** Tests {@link PseudoRandomValueSelector}. */
+public class PseudoRandomValueSelectorTest {

Review comment:
       extend TestLogger




----------------------------------------------------------------
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



[GitHub] [flink] rkhachatryan edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
rkhachatryan edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-740558364


   Assuming that the priorities are:
   1. increase the probability of detecting an error
   1. in particular, before merging - i.e. by committer either locally or remotely
   1. improve the usability
   
   The "MiniClusterResource" approach compared to the "properties" approach:
   - is worse for (1) because a smaller subset of tests is randomized
   - is better for (2) as you mention above
   - is worse for (3) because it's not easy to see what else tests fail with a certain configuration (when I already know that some fail)
   
   It also seems more complex (though I haven't thought through how to set the property globally).
   
   Additionally, I can imagine a "mandatory" set of combinations that must pass (in several runs) in order to release or be sure to merge. 


----------------------------------------------------------------
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



[GitHub] [flink] rkhachatryan commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
rkhachatryan commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-740558364


   Assuming that the priorities are:
   1. increase probability of detecting an error
   1. in particular, before merging - i.e. by committer either locally or remotely
   1. improve the usability
   
   The "MiniClusterResource" approach compared to the "properties" approach:
   - is worse for (1) because a smaller subset of tests is randomized
   - is better for (2) as you mention above
   - is worse for (3) because it's not easy to see what else tests fail with a certain configuration (when I already know that some fail)
   
   It also seems more complex though I haven't thought through how to set the property globally.
   
   Additionally, I can imagine a "mandatory" set of combinations that must pass (in several runs) in order to release or be sure to merge. 


----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * df7d30894d9bf8f2d5ab361d2c0437d5aec065f9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761) 
   
   <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



[GitHub] [flink] AHeise commented on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
AHeise commented on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-740196009


   > The randomization here only applies to tests that use `MiniClusterResource`, right?
   > I think this is a significant limitation.
   > 
   > Can we overcome it by reading some system property at the time of `ConfigOption` creation?
   > 
   > ```
   > 	public static final ConfigOption<Boolean> ENABLE_UNALIGNED =
   > 		ConfigOptions.key("execution.checkpointing.unaligned")
   > 			.booleanType()
   > 			.defaultValue(System.getProperty("execution.checkpointing.unaligned", false)) // <-- here
   > ```
   > 
   > The property can be set either
   > 
   >     * by maven/CI at the very beginning to a random value (seed="nanoTime")
   > 
   >     * or by the developer or to a fixed value
   > 
   > 
   > I think it will be easier to reason about if it's set for the whole build (and logged in the beginning).
   > 
   > WDYT?
   
   That was the very first approach but we didn't like that this means that all tests run only with that particular configuration, which has two issues:
   - For a large enough test matrix, a certain configuration may happen very rarely. Assuming we have 10 different configuration options (aligned, unaligned with 0, 10s, 1m timeout x 0, 1kb, 1m size limit), then we have 10% chance for a particular combination to occur. But even when running 10 tests, you have 3.4% chance of a specific combination not occurring. If you add that to rarely occurring instabilities that are even now hard to detect, the overall goal of having a good coverage is probably not reached. It becomes worse if we have more interdependent value combinations related to checkpointing like different DSTL settings.
   - Even for more commonly occurring issues that happen while refactoring/new feature, it's disadvantage to just have one configuration for all tests. You have to manually cycle through the relevant settings through setting the system property to "force" your luck. Here having more or less all combinations being executed by the same AZP run on your feature branch will probably already find you quite a few issues.
   
   Btw (but this is an orthogonal discussion), I'd like to bind the randomization seed to commit id instead of timestamp, such that it's easy for us to debug into any issue (checkout particular commit and go). However, the current implementation might not be ideal in that regard (`EnvironmentalInformation` might not be updated without a full 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051






----------------------------------------------------------------
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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fded2a71d2064276f69335ec94c36b35b8789985 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807) 
   * 9751d3ffff922d7153fd4a77a28f2f810dcdc32b 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * df7d30894d9bf8f2d5ab361d2c0437d5aec065f9 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761) 
   * 0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83 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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fe4a6228d31108e71a6e9faab97598ffa9a09b8b Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268) 
   
   <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



[GitHub] [flink] flinkbot edited a comment on pull request #13551: [FLINK-19520][configuration] Add randomization of checkpoint config.

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13551:
URL: https://github.com/apache/flink/pull/13551#issuecomment-704858051


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7262",
       "triggerID" : "5b037bbc8fd11fce426527e4caefccd075ddc4b6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7268",
       "triggerID" : "fe4a6228d31108e71a6e9faab97598ffa9a09b8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7296",
       "triggerID" : "a875c6b63432a371dd2928d6c3bdb2d4627c5f81",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4a2942d056be51f8064a94bfb064143decda77db",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7321",
       "triggerID" : "4a2942d056be51f8064a94bfb064143decda77db",
       "triggerType" : "PUSH"
     }, {
       "hash" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10533",
       "triggerID" : "07e1b12f433dec764812f49b9c0348d25498d6f6",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=10538",
       "triggerID" : "a88c6caabb50541ecdc8fcafa232adf0e799bbbb",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=11807",
       "triggerID" : "fded2a71d2064276f69335ec94c36b35b8789985",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12174",
       "triggerID" : "9751d3ffff922d7153fd4a77a28f2f810dcdc32b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12757",
       "triggerID" : "4c3a8b4fa089a223d1d90476068a7ef3111d5bfa",
       "triggerType" : "PUSH"
     }, {
       "hash" : "019646dd12750ec211a543f858836637a164d845",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "019646dd12750ec211a543f858836637a164d845",
       "triggerType" : "PUSH"
     }, {
       "hash" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12761",
       "triggerID" : "df7d30894d9bf8f2d5ab361d2c0437d5aec065f9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12775",
       "triggerID" : "0e119ed52f21bc045418a0a3c2cf3cc3e1d69f83",
       "triggerType" : "PUSH"
     }, {
       "hash" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "32b4aa620e79c8aee446756a801d68814ddf7b31",
       "triggerType" : "PUSH"
     }, {
       "hash" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931",
       "triggerID" : "29e7176390cd2877927bb8a20ac327921e8ae936",
       "triggerType" : "PUSH"
     }, {
       "hash" : "3d69b21f05abca0de64a2e778c292a7d0694b263",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13184",
       "triggerID" : "3d69b21f05abca0de64a2e778c292a7d0694b263",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 019646dd12750ec211a543f858836637a164d845 UNKNOWN
   * 32b4aa620e79c8aee446756a801d68814ddf7b31 UNKNOWN
   * 29e7176390cd2877927bb8a20ac327921e8ae936 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=12931) 
   * 3d69b21f05abca0de64a2e778c292a7d0694b263 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=13184) 
   
   <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