zentol commented on a change in pull request #15348: URL: https://github.com/apache/flink/pull/15348#discussion_r600296777
########## File path: flink-tests/src/test/java/org/apache/flink/runtime/jobmaster/JobMasterTriggerSavepointITCase.java ########## @@ -110,11 +111,8 @@ private void setUpWithCheckpointInterval(long checkpointInterval) throws Excepti 0), null); - jobGraph = - JobGraphBuilder.newStreamingJobGraphBuilder() - .addJobVertex(vertex) - .setJobCheckpointingSettings(jobCheckpointingSettings) - .build(); + jobGraph = JobGraphTestUtils.streamingJobGraph(vertex); + jobGraph.setSnapshotSettings(jobCheckpointingSettings); Review comment: I'm not sold on these changes; they revert us back to a pattern where we explicitly mutate the JobGraph and have to use badly-named APIs. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/jsonplan/JsonGeneratorTest.java ########## @@ -44,21 +44,20 @@ @Test public void testGeneratorWithoutAnyAttachements() { try { - JobVertex source1 = new JobVertex("source 1"); + JobVertex source1 = createNoOpVertex("source 1", 1); - JobVertex source2 = new JobVertex("source 2"); - source2.setInvokableClass(DummyInvokable.class); + JobVertex source2 = createNoOpVertex("source 2", 2); Review comment: ```suggestion JobVertex source2 = createNoOpVertex("source 2", 1); ``` ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphConfigurationUtilsAutoConfigurtionTest.java ########## @@ -0,0 +1,75 @@ +/* + * 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.runtime.jobgraph; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.flink.runtime.jobgraph.JobGraphTestUtils.createJobVertex; +import static org.apache.flink.runtime.jobgraph.JobGraphTestUtils.streamingJobGraph; + +@RunWith(Parameterized.class) +public class JobGraphConfigurationUtilsAutoConfigurtionTest { + + @Parameterized.Parameters(name = "parallelism = {0}, maxParallelism = {1}, expected max = {2}") + public static Object[][] data() { + return new Object[][] { + // default minimum + {1, JobVertex.MAX_PARALLELISM_DEFAULT, 128}, + // test round up part 1 + {171, JobVertex.MAX_PARALLELISM_DEFAULT, 256}, + // test round up part 2 + {172, JobVertex.MAX_PARALLELISM_DEFAULT, 512}, + // test round up limit + {1 << 15, JobVertex.MAX_PARALLELISM_DEFAULT, 1 << 15}, + // test configured / trumps computed default + {4, 1 << 15, 1 << 15}, + // test override trumps test configured 2 + {4, 7, 7}, + }; + } + + @Parameterized.Parameter(0) + public int parallelism; + + @Parameterized.Parameter(1) + public int maxParallelism; + + @Parameterized.Parameter(2) + public int expectedMaxParallelism; + + @Test + public void testMaxParallelismDefaulting() { + JobVertex jobVertex = createJobVertex(parallelism, maxParallelism); + JobGraph jobGraph = streamingJobGraph(jobVertex); + JobGraphConfigurationUtils.autoConfigureMaxParallelism(jobGraph); Review comment: Technically this shouldn't even be allowed because we are now auto-configuring the maxParallelism twice. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java ########## @@ -269,19 +256,15 @@ public ExecutionJobVertex( return jobVertex.getOperatorIDs(); } - public void setMaxParallelism(int maxParallelismDerived) { + public void setMaxParallelism(int maxParallelism) { Preconditions.checkState( - !maxParallelismConfigured, + isMaxParallelismAutoConfigured(), Review comment: It also doesn't really make sense for the maxParallelism to only be settable if it was previously autoConfigured by the system. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/executiongraph/ExecutionJobVertex.java ########## @@ -269,19 +256,15 @@ public ExecutionJobVertex( return jobVertex.getOperatorIDs(); } - public void setMaxParallelism(int maxParallelismDerived) { + public void setMaxParallelism(int maxParallelism) { Preconditions.checkState( - !maxParallelismConfigured, + isMaxParallelismAutoConfigured(), Review comment: This is not sufficient based on the JobVertex API, because it is possible for the maxParallelism to both be configured manually and being auto-configured. There are no assertions to ensure only one approach was used. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/jobgraph/JobGraphConfigurationUtilsAutoConfigurtionTest.java ########## @@ -0,0 +1,75 @@ +/* + * 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.runtime.jobgraph; + +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import static org.apache.flink.runtime.jobgraph.JobGraphTestUtils.createJobVertex; +import static org.apache.flink.runtime.jobgraph.JobGraphTestUtils.streamingJobGraph; + +@RunWith(Parameterized.class) +public class JobGraphConfigurationUtilsAutoConfigurtionTest { Review comment: ```suggestion public class JobGraphConfigurationUtilsAutoConfigurtionTest extends 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: us...@infra.apache.org