Hi Shannon!

In the latest HA and BlobStore changes (1.3) it uses "/tmp" only for
caching and will re-obtain the files from the persistent storage.

I think we should make this a bigger point, even:
  - Flink should not use "/tmp" at all (except for mini cluster mode)
  - Yarn and Mesos should always use the "local directory" for temporary
files. They are cleaned up anyways.
  - For the Standalone Setup, one should configure a suitable temp storage
dir (and everything should be relative to that).

Stephan



On Mon, Feb 20, 2017 at 3:22 PM, Ufuk Celebi <u...@apache.org> wrote:

> Hey Shannon,
>
> good idea! We currently have this:
> https://ci.apache.org/projects/flink/flink-docs-
> release-1.3/ops/production_ready.html
>
> It has a strong focus on managed state and not the points you mentioned.
>
> Would you like to create an issue for adding this to the production
> check list? I think it's valuable feedback.
>
> – Ufuk
>
>
> On Fri, Feb 17, 2017 at 7:24 PM, Shannon Carey <sca...@expedia.com> wrote:
> > A few of my jobs recently failed and showed this exception:
> >
> >
> > org.apache.flink.streaming.runtime.tasks.StreamTaskException: Cannot
> load
> > user class: org.apache.flink.streaming.connectors.kafka.
> FlinkKafkaConsumer09
> > ClassLoader info: URL ClassLoader:
> >     file:
> > '/tmp/blobStore-5f023409-6af5-4de6-8ed0-e80a2eb9633e/cache/blob_
> d9a9fb884f3b436030afcf7b8e1bce678acceaf2'
> > (invalid JAR: zip file is empty)
> > Class not resolvable through given classloader.
> >       at
> > org.apache.flink.streaming.api.graph.StreamConfig.getStreamOperator(
> StreamConfig.java:208)
> >       at
> > org.apache.flink.streaming.runtime.tasks.StreamTask.
> invoke(StreamTask.java:224)
> >       at org.apache.flink.runtime.taskmanager.Task.run(Task.java:642)
> >       at java.lang.Thread.run(Thread.java:745)
> >
> >
> > As you can see, Flink is storing things underneath /tmp, which is the
> > (undocumented) default for the blob store. As you may know, on Linux,
> > there's typically a program such as tmpwatch which is run periodically to
> > clear out data from /tmp.
> >
> >
> > Flink also uses /tmp as the default for jobmanager.web.tmpdir (and
> > jobmanager.web.upload.dir in 1.2).
> >
> >
> > Therefore, assuming that this is indeed the cause of the job failure/the
> > exception, it seems highly advisable that when you run a Flink cluster
> you
> > configure blob.storage.directory and jobmanager.web.tmpdir to a specific
> > folder that is not beneath /tmp. I don't know if there is any material
> about
> > setting up a production cluster, but this would definitely seem to be a
> > necessary configuration to specify if you want to avoid problems.
> Enabling
> > High Availability mode should also be on that list, I think.
> >
> >
> > -Shannon
>

Reply via email to