Hi Micah,

the problem looks indeed similar to FLINK-10255. Could you tell me your
exact cluster setup (HA with stand by JobManagers?). Moreover, the logs of
all JobManagers on DEBUG level would be helpful for further debugging.

Cheers,
Till

On Tue, Dec 11, 2018 at 10:09 AM Stefan Richter <s.rich...@data-artisans.com>
wrote:

> Hi,
>
> Thanks for reporting the problem, I think the exception trace looks indeed
> very similar to traces in the discussion for FLINK-10184. I will pull in
> Till who worked on the fix to hear his opinion. Maybe the current fix only
> made the problem less likely to appear but is not complete, yet?
>
> Best,
> Stefan
>
> > On 11. Dec 2018, at 05:19, Micah Wylde <mwy...@lyft.com> wrote:
> >
> > Hello,
> >
> > We've been seeing an issue with several Flink 1.5.4 clusters that looks
> like this:
> >
> > 1. Job is cancelled with a savepoint
> > 2. The jar is deleted from our HA blobstore (S3)
> > 3. The jobgraph in ZK is *not* deleted
> > 4. We restart the cluster
> > 5. Startup fails in recovery because the jar is not available, with the
> stacktrace:
> >
> > 00:13:58.486 ERROR o.a.f.r.e.ClusterEntrypoint - Fatal error occurred in
> the cluster entrypoint.
> > {{ java.lang.RuntimeException:
> org.apache.flink.runtime.client.JobExecutionException: Could not set up
> JobManager}}
> > {{ at
> org.apache.flink.util.function.CheckedSupplier.lambda$unchecked$0(CheckedSupplier.java:36)}}
> > {{ at
> java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1590)}}
> > {{ at akka.dispatch.TaskInvocation.run(AbstractDispatcher.scala:39)}}
> > {{ at
> akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:415)}}
> > {{ 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: java.lang.Exception: Cannot set up the user code libraries: No such
> file or directory:
> s3://streamingplatform-production/{JOB_NAME}/flink/highavailability/{JOB_NAME}/blob/job_5a3fe2c00c05efd3a552a1c6707d2c10/blob_p-6d585831f5c947335ac505b400cf8f3630cc706a-42355c2885b668b0bc5e15b856141b0
> >
> > This superficially seems similar to several issues that have apparently
> been fixed in 1.5.4, like FLINK-10255 and FLINK-10184.
> >
> > Has anybody else seen this issue on 1.5.4 (or later) clusters? Or any
> advice for debugging?
> >
> > Thanks,
> > Micah
>
>

Reply via email to