[ 
https://issues.apache.org/jira/browse/FLINK-7124?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16154961#comment-16154961
 ] 

ASF GitHub Bot commented on FLINK-7124:
---------------------------------------

Github user tzulitai commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4510#discussion_r137196004
  
    --- Diff: 
flink-runtime/src/test/java/org/apache/flink/runtime/executiongraph/ExecutionGraphRescalingTest.java
 ---
    @@ -0,0 +1,235 @@
    +/*
    + * 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.executiongraph;
    +
    +import org.apache.flink.api.common.ExecutionConfig;
    +import org.apache.flink.configuration.Configuration;
    +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup;
    +import org.apache.flink.runtime.akka.AkkaUtils;
    +import 
org.apache.flink.runtime.checkpoint.StandaloneCheckpointRecoveryFactory;
    +import org.apache.flink.runtime.executiongraph.restart.NoRestartStrategy;
    +import org.apache.flink.runtime.io.network.partition.ResultPartitionType;
    +import org.apache.flink.runtime.jobgraph.DistributionPattern;
    +import org.apache.flink.runtime.jobgraph.JobGraph;
    +import org.apache.flink.runtime.jobgraph.JobVertex;
    +import org.apache.flink.runtime.jobgraph.tasks.AbstractInvokable;
    +import org.apache.flink.runtime.jobmanager.scheduler.Scheduler;
    +import org.apache.flink.runtime.testingUtils.TestingUtils;
    +
    +import org.junit.Ignore;
    +import org.junit.Test;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import java.util.Arrays;
    +import java.util.Collections;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.fail;
    +
    +/**
    + * This class contains tests that verify when rescaling a {@link JobGraph},
    + * constructed {@link ExecutionGraph}s are correct.
    + */
    +public class ExecutionGraphRescalingTest {
    +
    +   private static final Logger TEST_LOGGER = 
LoggerFactory.getLogger(ExecutionGraphRescalingTest.class);
    +
    +   @Test
    +   public void testExecutionGraphArbitraryDopConstructionTest() throws 
Exception {
    +
    +           final Configuration config = new Configuration();
    +
    +           final JobVertex[] jobVertices = 
createVerticesForSimpleBipartiteJobGraph();
    +           final JobGraph jobGraph = new JobGraph(jobVertices);
    +
    +           // TODO rescaling the JobGraph is currently only supported if 
the
    +           // TODO configured parallelism is 
ExecutionConfig.PARALLELISM_AUTO_MAX.
    +           // TODO this limitation should be removed.
    +           for (JobVertex jv : jobVertices) {
    +                   jv.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
    +           }
    +
    +           ExecutionGraph eg = ExecutionGraphBuilder.buildGraph(
    +                           null,
    +                           jobGraph,
    +                           config,
    +                           TestingUtils.defaultExecutor(),
    +                           TestingUtils.defaultExecutor(),
    +                           new 
Scheduler(TestingUtils.defaultExecutionContext()),
    +                           Thread.currentThread().getContextClassLoader(),
    +                           new StandaloneCheckpointRecoveryFactory(),
    +                           AkkaUtils.getDefaultTimeout(),
    +                           new NoRestartStrategy(),
    +                           new UnregisteredMetricsGroup(),
    +                           5,
    +                           TEST_LOGGER);
    +
    +           for (JobVertex jv : jobVertices) {
    +                   assertEquals(5, jv.getParallelism());
    +           }
    +           verifyGeneratedExecutionGraphOfSimpleBitartiteJobGraph(eg, 
jobVertices);
    +
    +           // --- verify scaling up works correctly ---
    +
    +           // TODO rescaling the JobGraph is currently only supported if 
the
    +           // TODO configured parallelism is 
ExecutionConfig.PARALLELISM_AUTO_MAX.
    +           // TODO this limitation should be removed.
    +           for (JobVertex jv : jobVertices) {
    +                   jv.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
    +           }
    +
    +           eg = ExecutionGraphBuilder.buildGraph(
    +                           null,
    +                           jobGraph,
    +                           config,
    +                           TestingUtils.defaultExecutor(),
    +                           TestingUtils.defaultExecutor(),
    +                           new 
Scheduler(TestingUtils.defaultExecutionContext()),
    +                           Thread.currentThread().getContextClassLoader(),
    +                           new StandaloneCheckpointRecoveryFactory(),
    +                           AkkaUtils.getDefaultTimeout(),
    +                           new NoRestartStrategy(),
    +                           new UnregisteredMetricsGroup(),
    +                           10,
    +                           TEST_LOGGER);
    +
    +           for (JobVertex jv : jobVertices) {
    +                   assertEquals(10, jv.getParallelism());
    +           }
    +           verifyGeneratedExecutionGraphOfSimpleBitartiteJobGraph(eg, 
jobVertices);
    +
    +           // --- verify down scaling works correctly ---
    +
    +           // TODO rescaling the JobGraph is currently only supported if 
the
    +           // TODO configured parallelism is 
ExecutionConfig.PARALLELISM_AUTO_MAX.
    +           // TODO this limitation should be removed.
    +           for (JobVertex jv : jobVertices) {
    +                   jv.setParallelism(ExecutionConfig.PARALLELISM_AUTO_MAX);
    +           }
    +
    +           eg = ExecutionGraphBuilder.buildGraph(
    +                   null,
    +                   jobGraph,
    +                   config,
    +                   TestingUtils.defaultExecutor(),
    +                   TestingUtils.defaultExecutor(),
    +                   new Scheduler(TestingUtils.defaultExecutionContext()),
    +                   Thread.currentThread().getContextClassLoader(),
    +                   new StandaloneCheckpointRecoveryFactory(),
    +                   AkkaUtils.getDefaultTimeout(),
    +                   new NoRestartStrategy(),
    +                   new UnregisteredMetricsGroup(),
    +                   2,
    +                   TEST_LOGGER);
    +
    +           for (JobVertex jv : jobVertices) {
    +                   assertEquals(2, jv.getParallelism());
    +           }
    +           verifyGeneratedExecutionGraphOfSimpleBitartiteJobGraph(eg, 
jobVertices);
    +   }
    +
    +   /**
    +    * Verifies that building an {@link ExecutionGraph} from a {@link 
JobGraph} with
    +    * parallelism higher than the maximum parallelism fails.
    +    *
    +    * TODO this test is ignored, since currently the rescale does not 
properly fail when rescaling to DOP above max.
    --- End diff --
    
    @tillrohrmann currently the rescale doesn't fail, and the created 
`ExecutionGraph` will have DOP as specified (above max).


> Allow to rescale JobGraph on JobManager
> ---------------------------------------
>
>                 Key: FLINK-7124
>                 URL: https://issues.apache.org/jira/browse/FLINK-7124
>             Project: Flink
>          Issue Type: Bug
>          Components: Distributed Coordination
>            Reporter: Till Rohrmann
>            Assignee: Tzu-Li (Gordon) Tai
>
> In order to support dynamic scaling, the {{JobManager}} has to be able to 
> change the parallelism of the submitted {{JobGraph}}. This basically entails 
> that we can change the parallelism settings on the cluster side.
> We already have the functionality that we can change the parallelism if it 
> was set to {{ExecutionConfig.PARALLELISM_AUTO_MAX}}. Therefore, I think the 
> task is mostly about making sure that things really properly work when 
> requesting a parallelism change.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to