You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by NicoK <gi...@git.apache.org> on 2018/03/01 13:01:43 UTC

[GitHub] flink pull request #5611: [FLINK-8769][flip6] do not print error causing exc...

GitHub user NicoK opened a pull request:

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

    [FLINK-8769][flip6] do not print error causing exceptions without debugging

    ## What is the purpose of the change
    
    Quickstart job execution in IDE logs these two exceptions which may mislead the user in thinking something went wrong when indeed it did not:
    
    1)
    ```
    10:23:18,413 INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint    - Failed to load web based job submission extension.
    org.apache.flink.util.FlinkException: The module flink-runtime-web could not be found in the class path. Please add this jar in order to enable web based job submission.
    	at org.apache.flink.runtime.webmonitor.WebMonitorUtils.loadWebSubmissionExtension(WebMonitorUtils.java:255)
    	at org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint.initializeHandlers(DispatcherRestEndpoint.java:111)
    	at org.apache.flink.runtime.rest.RestServerEndpoint.start(RestServerEndpoint.java:127)
    	at org.apache.flink.runtime.minicluster.MiniCluster.start(MiniCluster.java:330)
    	at org.apache.flink.streaming.api.environment.Flip6LocalStreamEnvironment.execute(Flip6LocalStreamEnvironment.java:101)
    	at flink.test.StreamingJob.main(StreamingJob.java:64)
    ```
    2)
    ```
    10:23:19,099 INFO  org.apache.flink.runtime.taskexecutor.TaskExecutor            - Close ResourceManager connection 3ebc4874f7a952d9ba4e4fc6ac574b89.
    org.apache.flink.util.FlinkException: New ResourceManager leader found under: null(null)
    	at org.apache.flink.runtime.taskexecutor.TaskExecutor.notifyOfNewResourceManagerLeader(TaskExecutor.java:851)
    	at org.apache.flink.runtime.taskexecutor.TaskExecutor.access$900(TaskExecutor.java:127)
    	at org.apache.flink.runtime.taskexecutor.TaskExecutor$ResourceManagerLeaderListener.lambda$notifyLeaderAddress$0(TaskExecutor.java:1359)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:295)
    	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:150)
    	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)
    ```
    
    While for debugging they may be useful, for normal operation it is enough to print the messages
    themselves, especially since some more details were already logged before them.
    
    ## Brief change log
    
    - only print error causing exceptions in `DispatcherRestEndpoint` and `TaskExecutor` when in debug mode similar to how queryable state loading is done
    
    ## Verifying this change
    
    This change can be verified as follows:
    - started a fresh quickstart project with a simple streaming program like `env.fromElements(1,2,3,4,5).print();` in IntelliJ and verified there are no exceptions in the logs
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): **no**
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`:  **no**
      - The serializers:  **no**
      - The runtime per-record code paths (performance sensitive):  **no**
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: **yes**
      - The S3 file system connector:  **no**
    
    ## Documentation
    
      - Does this pull request introduce a new feature?  **no**
      - If yes, how is the feature documented? **not applicable**


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

    $ git pull https://github.com/NicoK/flink flink-8769

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

    https://github.com/apache/flink/pull/5611.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 #5611
    
----
commit d4e55e6a859c2b14edaab89797416b086fe609b6
Author: Nico Kruber <ni...@...>
Date:   2018-03-01T12:53:21Z

    [FLINK-8769][flip6] do not print error causing exceptions without debugging
    
    In DispatcherRestEndpoint and TaskExecutor, there were two places where without
    errors (running a job inside an IDE) exceptions were logged. While for debugging
    they may be useful, for normal operation it is enough to print the messages
    themselves, especially since some more details were already logged before.

----


---

[GitHub] flink pull request #5611: [FLINK-8769][flip6] do not print error causing exc...

Posted by GJL <gi...@git.apache.org>.
Github user GJL commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5611#discussion_r171883413
  
    --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/dispatcher/DispatcherRestEndpoint.java ---
    @@ -120,7 +120,12 @@ public DispatcherRestEndpoint(
     				// register extension handlers
     				handlers.addAll(webSubmissionExtension.getHandlers());
     			} catch (FlinkException e) {
    -				log.info("Failed to load web based job submission extension.", e);
    +				if (log.isDebugEnabled()) {
    +					log.debug("Failed to load web based job submission extension.", e);
    +				} else {
    +					log.info("Failed to load web based job submission extension. " +
    --- End diff --
    
    It still sounds like an error. Maybe `Web based job submission is disabled. Probable reason: [...]`


---

[GitHub] flink pull request #5611: [FLINK-8769][flip6] do not print error causing exc...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---