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

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

GitHub user pnowojski opened a pull request:

    https://github.com/apache/flink/pull/5756

    [FLINK-8813][flip6] Disallow PARALLELISM_AUTO_MAX for flip6

    This deprecates 
`org.apache.flink.api.common.ExecutionConfig#PARALLELISM_AUTO_MAX`. From now 
on, with flip6 mode user will be hit with following error message.
    
    ```
    Caused by: org.apache.flink.runtime.client.JobSubmissionException: Could 
not start JobManager.
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.submitJob(Dispatcher.java:299)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:498)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:210)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:154)
        at 
org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleMessage(FencedAkkaRpcActor.java:66)
        at 
org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$onReceive$1(AkkaRpcActor.java:132)
        at 
akka.actor.ActorCell$$anonfun$become$1.applyOrElse(ActorCell.scala:544)
        at akka.actor.Actor$class.aroundReceive(Actor.scala:502)
        at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)
        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)
        at akka.actor.ActorCell.invoke(ActorCell.scala:495)
        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
        at akka.dispatch.Mailbox.run(Mailbox.scala:224)
        at akka.dispatch.Mailbox.exec(Mailbox.scala:234)
        at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
        at 
scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
        at 
scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
        at 
scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
    Caused by: org.apache.flink.runtime.client.JobExecutionException: Could not 
set up JobManager
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:167)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher$DefaultJobManagerRunnerFactory.createJobManagerRunner(Dispatcher.java:762)
        at 
org.apache.flink.runtime.dispatcher.Dispatcher.submitJob(Dispatcher.java:254)
        ... 20 more
    Caused by: org.apache.flink.runtime.client.JobSubmissionException: 
PARALLELISM_AUTO_MAX is no longer supported. Please specify a concrete value 
for the parallelism.
        at 
org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:206)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:290)
        at 
org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:147)
        ... 22 more
    ```
    
    Previously it was similar message but ending with: `Caused by: 
java.lang.IllegalArgumentException: The parallelism must be at least one.`
    
    ## What is the purpose of the change
    
    Both flip6 and old modes are covered by `AutoParallelismITCase`.
    
    ## 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) - deprecates 
`org.apache.flink.api.common.ExecutionConfig#PARALLELISM_AUTO_MAX`
      - 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, 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)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/pnowojski/flink f8813

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5756.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5756
    
----
commit e31a90b45619517dba7c0832cfcd1c697d0a8b0f
Author: Piotr Nowojski <piotr.nowojski@...>
Date:   2018-03-23T15:01:57Z

    [FLINK-8813][flip6] Disallow PARALLELISM_AUTO_MAX for flip6

----


> AutoParallellismITCase fails with Flip6
> ---------------------------------------
>
>                 Key: FLINK-8813
>                 URL: https://issues.apache.org/jira/browse/FLINK-8813
>             Project: Flink
>          Issue Type: Bug
>          Components: JobManager, Tests
>    Affects Versions: 1.5.0
>            Reporter: Chesnay Schepler
>            Assignee: Piotr Nowojski
>            Priority: Blocker
>              Labels: flip-6
>             Fix For: 1.5.0
>
>
> The {{AutoParallelismITCase}} fails when running against flip6. 
> ([https://travis-ci.org/zentol/flink/jobs/347373854)]
> It appears that the {{JobMaster}} does not properly handle 
> {{ExecutionConfig#PARALLELISM_AUTO_MAX}}.
>  
> Exception:
> {code:java}
> Caused by: org.apache.flink.runtime.client.JobSubmissionException: Could not 
> start JobManager.
>       at 
> org.apache.flink.runtime.dispatcher.Dispatcher.submitJob(Dispatcher.java:287)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:498)
>       at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcInvocation(AkkaRpcActor.java:210)
>       at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:154)
>       at 
> org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleMessage(FencedAkkaRpcActor.java:66)
>       at 
> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$onReceive$1(AkkaRpcActor.java:132)
>       at 
> akka.actor.ActorCell$$anonfun$become$1.applyOrElse(ActorCell.scala:544)
>       at akka.actor.Actor$class.aroundReceive(Actor.scala:502)
>       at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)
>       at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)
>       at akka.actor.ActorCell.invoke(ActorCell.scala:495)
>       at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
>       at akka.dispatch.Mailbox.run(Mailbox.scala:224)
>       at akka.dispatch.Mailbox.exec(Mailbox.scala:234)
>       at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
>       at 
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
>       at 
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
>       at 
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: org.apache.flink.runtime.client.JobExecutionException: Could not 
> set up JobManager
>       at 
> org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:181)
>       at 
> org.apache.flink.runtime.dispatcher.Dispatcher$DefaultJobManagerRunnerFactory.createJobManagerRunner(Dispatcher.java:747)
>       at 
> org.apache.flink.runtime.dispatcher.Dispatcher.submitJob(Dispatcher.java:243)
>       ... 20 more
> Caused by: java.lang.IllegalArgumentException: The parallelism must be at 
> least one.
>       at 
> org.apache.flink.runtime.jobgraph.JobVertex.setParallelism(JobVertex.java:290)
>       at 
> org.apache.flink.runtime.executiongraph.ExecutionGraphBuilder.buildGraph(ExecutionGraphBuilder.java:162)
>       at 
> org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:295)
>       at 
> org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:170)
>       ... 22 more{code}
>  
> The likely culprit is this call to {{ExecutionGraphBuilder#buildGraph}} in 
> the {{JobMaster}} constructor:
> {code:java}
> this.executionGraph = ExecutionGraphBuilder.buildGraph(
>    null,
>    jobGraph,
>    jobMasterConfiguration.getConfiguration(),
>    scheduledExecutorService,
>    scheduledExecutorService,
>    slotPool.getSlotProvider(),
>    userCodeLoader,
>    highAvailabilityServices.getCheckpointRecoveryFactory(),
>    rpcTimeout,
>    restartStrategy,
>    jobMetricGroup,
>    -1, // parallelismForAutoMax
>    blobServer,
>    jobMasterConfiguration.getSlotRequestTimeout(),
>    log);{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to