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: [email protected]
