Hi Oleksandr,

the requestJob call should only take longer if either the `JobMaster` is
overloaded and too busy to respond to the request or if the
ArchivedExecutionGraph is very large (e.g. very large accumulators) and
generating it and sending it over to the RestServerEndpoint takes too long.
This is also the change which was introduced with Flink 1.5. Instead of
simply handing over a reference to the RestServerEndpoint from the
JobMaster, the ArchivedExecutionGraph now needs to be sent through the
network stack to the RestServerEndpoint.

If you did not change the akka.framesize then the maximum size of the
ArchivedExecutionGraph should only be 10 MB, though. Therefore, I would
guess that your `JobMaster` must be quite busy if the requests time out.

Cheers,
Till

On Wed, Jan 2, 2019 at 10:58 AM Oleksandr Nitavskyi <o.nitavs...@criteo.com>
wrote:

> Hello guys. Happy new year!
>
>
>
> Context: we started to have some troubles with UI after bumping our Flink
> version from 1.4 to 1.6.3. UI couldn’t render Job details page, so
> inspecting of the jobs for us has become impossible with the new version.
>
>
>
> And looks like we have a workaround for our UI issue.
>
> After some investigation we realized that starting from Flink 1.5 version
> we started to have a timeout on the actor call: 
> *restfulGateway.requestJob(jobId,
> timeout)* in *ExecutionGraphCache*. So we have increased *web.timeout*
> parameter and we have stopped to have timeout exception on the JobManager
> side.
>
>
>
> Also in *SingleJobController* on the Angular JS side we needed to tweak
> *web.refresh-interval* in order to ensure that Front-End is waiting for
> back-end request to be finished. Otherwise Angular JS side can make another
> request in SingleJobController and don’t know why when older request is
> finished no UI has been changed. We will have a look closer on this
> behavior.
>
>
>
> Does it ring a bell for you probably?
>
>
>
> Thank you
>
>
>
> Kind Regards
>
> Oleksandr
>
>
>
> *From: *Till Rohrmann <trohrm...@apache.org>
> *Date: *Wednesday 19 December 2018 at 16:52
> *To: *Juan Gentile <j.gent...@criteo.com>
> *Cc: *"dwysakow...@apache.org" <dwysakow...@apache.org>, Jeff Bean <
> j...@data-artisans.com>, Oleksandr Nitavskyi <o.nitavs...@criteo.com>
> *Subject: *Re: 1.6 UI issues
>
>
>
> Hi Juan,
>
>
>
> thanks for the log. The log file does not contain anything suspicious. Are
> you sure that you sent me the right file? The timestamps don't seem to
> match. In the attached log, the job seems to run without problems.
>
>
>
> Cheers,
>
> Till
>
>
>
> On Wed, Dec 19, 2018 at 10:26 AM Juan Gentile <j.gent...@criteo.com>
> wrote:
>
> Hello Till, Dawid
>
> Sorry for the late response on this issue and thank you Jeff for helping
> us with this.
>
> Yes we are using 1.6.2
>
> I attach the logs from the Job Master.
>
> Also we noticed this exception:
>
> 2018-12-19 08:50:10,497 ERROR
> org.apache.flink.runtime.rest.handler.job.JobDetailsHandler   -
> Implementation error: Unhandled exception.
>
> java.util.concurrent.CancellationException
>
>     at
> java.util.concurrent.CompletableFuture.cancel(CompletableFuture.java:2263)
>
>     at
> org.apache.flink.runtime.rest.handler.legacy.ExecutionGraphCache.getExecutionGraph(ExecutionGraphCache.java:124)
>
>     at
> org.apache.flink.runtime.rest.handler.job.AbstractExecutionGraphHandler.handleRequest(AbstractExecutionGraphHandler.java:76)
>
>     at
> org.apache.flink.runtime.rest.handler.AbstractRestHandler.respondToRequest(AbstractRestHandler.java:78)
>
>     at
> org.apache.flink.runtime.rest.handler.AbstractHandler.respondAsLeader(AbstractHandler.java:154)
>
>     at
> org.apache.flink.runtime.rest.handler.RedirectHandler.lambda$null$0(RedirectHandler.java:142)
>
>     at
> java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)
>
>     at
> java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)
>
>     at
> java.util.concurrent.CompletableFuture$Completion.run(CompletableFuture.java:442)
>
>     at
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.AbstractEventExecutor.safeExecute(AbstractEventExecutor.java:163)
>
>     at
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:404)
>
>     at
> org.apache.flink.shaded.netty4.io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:463)
>
>     at
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.SingleThreadEventExecutor$5.run(SingleThreadEventExecutor.java:884)
>
>     at
> org.apache.flink.shaded.netty4.io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
>
>     at java.lang.Thread.run(Thread.java:748)
>
> 2018-12-19 08:50:17,977 ERROR
> org.apache.flink.runtime.rest.handler.job.JobsOverviewHandler  -
> Implementation error: Unhandled exception.
>
> akka.pattern.AskTimeoutException: Ask timed out on
> [Actor[akka://flink/user/dispatcher#-760166654]] after [10000 ms].
> Sender[null] sent message of type
> "org.apache.flink.runtime.rpc.messages.LocalFencedMessage".
>
>     at
> akka.pattern.PromiseActorRef$$anonfun$1.apply$mcV$sp(AskSupport.scala:604)
>
>     at akka.actor.Scheduler$$anon$4.run(Scheduler.scala:126)
>
>     at
> scala.concurrent.Future$InternalCallbackExecutor$.unbatchedExecute(Future.scala:601)
>
>     at
> scala.concurrent.BatchingExecutor$class.execute(BatchingExecutor.scala:109)
>
>     at
> scala.concurrent.Future$InternalCallbackExecutor$.execute(Future.scala:599)
>
>     at
> akka.actor.LightArrayRevolverScheduler$TaskHolder.executeTask(LightArrayRevolverScheduler.scala:329)
>
>     at
> akka.actor.LightArrayRevolverScheduler$$anon$4.executeBucket$1(LightArrayRevolverScheduler.scala:280)
>
>     at
> akka.actor.LightArrayRevolverScheduler$$anon$4.nextTick(LightArrayRevolverScheduler.scala:284)
>
>     at
> akka.actor.LightArrayRevolverScheduler$$anon$4.run(LightArrayRevolverScheduler.scala:236)
>
>     at java.lang.Thread.run(Thread.java:748)
>
>
>
> For which we tested with this parameter: -Dakka.ask.timeout=60s
>
> But the issue remains.
>
>
>
> Thank you
>
> Juan
>
>
>
> *From: *Till Rohrmann <trohrm...@apache.org>
> *Date: *Thursday, 8 November 2018 at 16:06
> *To: *"dwysakow...@apache.org" <dwysakow...@apache.org>
> *Cc: *Juan Gentile <j.gent...@criteo.com>, "myas...@live.com" <
> myas...@live.com>, user <user@flink.apache.org>
> *Subject: *Re: 1.6 UI issues
>
>
>
> Hi Juan,
>
>
>
> could you share the cluster entrypoint logs with us? They should contain
> more information about the internal server error.
>
>
>
> Just to make sure, you are using Flink 1.6.2, right?
>
>
>
> Cheers,
>
> Till
>
>
>
> On Thu, Nov 8, 2018 at 3:29 PM Dawid Wysakowicz <dwysakow...@apache.org>
> wrote:
>
> Hi Juan,
>
> It doesn't look similar to the issue linked to me. What cluster setup are
> you using? Are you running HA mode?
>
> I am adding Till to cc, who might be able to help you more.
>
> Best,
>
> Dawid
>
> On 02/11/2018 17:26, Juan Gentile wrote:
>
> Hello Yun,
>
>
>
> We haven’t seen the error in the log as you mentioned. We also checked the
> GC and it seems to be okay. Inspecting the UI we found the following error:
> *Error! Filename not specified.*
>
>
>
> {"errors":["Could not retrieve the redirect address of the current leader.
> Please try to refresh."]}
>
> *Error! Filename not specified.*
>
>
>
> We suspect we are running into the same issue as described here (
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/akka-timeout-td14996.html
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__apache-2Dflink-2Duser-2Dmailing-2Dlist-2Darchive.2336050.n4.nabble.com_akka-2Dtimeout-2Dtd14996.html&d=DwMFaQ&c=nxfEpP1JWHVKAq835DW4mA&r=z5BFHEFwsu2ghSzcXn1_8T3-VzeesIO2aULbUy2urus&m=hidCcXGD2aiyfZuADm1v4XCzvKL2Rsww7WJWxETtxRY&s=zMBuP5aTcdQ5VMavXw1dGvz72efTyTSq6tpbFcPSHxU&e=>)
> but we are not so sure.
>
>
>
> Have you encountered this issue before?
>
>
>
> Thank you,
>
>
>
> *From: *Yun Tang <myas...@live.com> <myas...@live.com>
> *Date: *Thursday, 1 November 2018 at 12:31
> *To: *Juan Gentile <j.gent...@criteo.com> <j.gent...@criteo.com>,
> "user@flink.apache.org" <user@flink.apache.org> <user@flink.apache.org>
> <user@flink.apache.org>
> *Subject: *Re: 1.6 UI issues
>
>
>
> Hi Juan
>
>
>
> From our experience, you could check the jobmanager.log first to see
> whether existing similar logs below:
>
> max allowed size 128000 bytes, actual size of encoded class 
> akka.actor.Status$Success was xxx bytes
>
>
>
> If you see these logs, you should increase the akka.framesize to larger value 
> (default value is '10485760b') [1].
>
>
>
> Otherwise, you could check the gc-log of job manager to see whether the gc 
> overhead is too heavy for your job manager, consider to increase the memory 
> for your job manager if so.
>
> [1]
> https://ci.apache.org/projects/flink/flink-docs-stable/ops/config.html#distributed-coordination-via-akka
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__ci.apache.org_projects_flink_flink-2Ddocs-2Dstable_ops_config.html-23distributed-2Dcoordination-2Dvia-2Dakka&d=DwMFAg&c=nxfEpP1JWHVKAq835DW4mA&r=z5BFHEFwsu2ghSzcXn1_8T3-VzeesIO2aULbUy2urus&m=WJfyB8WEQI3uCNuLBz9E_TtrMgZAFOjY_tUhvp3iiNs&s=kf9PiKegay46-MmjlmWIQiIwY0_EhOZqMZooObLId_Y&e=>
>
> Apache Flink 1.6 Documentation: Configuration
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__ci.apache.org_projects_flink_flink-2Ddocs-2Dstable_ops_config.html-23distributed-2Dcoordination-2Dvia-2Dakka&d=DwMFAg&c=nxfEpP1JWHVKAq835DW4mA&r=z5BFHEFwsu2ghSzcXn1_8T3-VzeesIO2aULbUy2urus&m=WJfyB8WEQI3uCNuLBz9E_TtrMgZAFOjY_tUhvp3iiNs&s=kf9PiKegay46-MmjlmWIQiIwY0_EhOZqMZooObLId_Y&e=>
>
> Key Default Description; jobmanager.heap.size "1024m" JVM heap size for
> the JobManager. taskmanager.heap.size "1024m" JVM heap size for the
> TaskManagers, which are the parallel workers of the system.
>
> ci.apache.org
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__ci.apache.org&d=DwMFaQ&c=nxfEpP1JWHVKAq835DW4mA&r=z5BFHEFwsu2ghSzcXn1_8T3-VzeesIO2aULbUy2urus&m=hidCcXGD2aiyfZuADm1v4XCzvKL2Rsww7WJWxETtxRY&s=JGgSIUxh1k57R0OSjnAG8GxwbzWUo6MFercAY-3JL3k&e=>
>
> Best
>
> Yun Tang
>
>
> ------------------------------
>
> *From:* Juan Gentile <j.gent...@criteo.com> <j.gent...@criteo.com>
> *Sent:* Wednesday, October 31, 2018 22:05
> *To:* user@flink.apache.org
> *Subject:* 1.6 UI issues
>
>
>
> Hello!
>
>
>
> We are migrating the the last 1.6 version and all the jobs seem to work
> fine, but when we check individual jobs through the web interface we
> encounter the issue that after clicking on a job, either it takes too long
> to load the information of the job or it never loads at all.
>
>
>
> Has anyone had this issue? Any clues as to why?
>
>
>
> Thank you,
>
> Juan
>
>

Reply via email to