Hi,
the file that Flink is trying to create there is not meant to be in the
checkpointing location. It is a local file that is used for buffering
elements until a checkpoint barrier arrives (for certain cases). Can you
check whether the base path where it is trying to create that file exists?
For the exception that you posted that would be:
/tmp/flink-io-202fdf67-3f8c-47dd-8ebc-2265430644ed

Cheers,
Aljoscha

On Fri, 14 Oct 2016 at 17:37 <robert.lancas...@hyatt.com> wrote:

> I recently tried enabling checkpointing in a job (that previously works
> w/o checkpointing) and received the following failure on job execution:
>
>
>
> java.io.FileNotFoundException:
> /tmp/flink-io-202fdf67-3f8c-47dd-8ebc-2265430644ed/a426eb27761575b3b79e464719bba96e16a1869d85bae292a2ef7eb72fa8a14c.0.buffer
> (No such file or directory)
>
>         at java.io.RandomAccessFile.open0(Native Method)
>
>         at java.io.RandomAccessFile.open(RandomAccessFile.java:316)
>
>         at java.io.RandomAccessFile.<init>(RandomAccessFile.java:243)
>
>         at
> org.apache.flink.streaming.runtime.io.BufferSpiller.createSpillingChannel(BufferSpiller.java:247)
>
>         at
> org.apache.flink.streaming.runtime.io.BufferSpiller.<init>(BufferSpiller.java:117)
>
>         at
> org.apache.flink.streaming.runtime.io.BarrierBuffer.<init>(BarrierBuffer.java:94)
>
>         at
> org.apache.flink.streaming.runtime.io.StreamInputProcessor.<init>(StreamInputProcessor.java:96)
>
>         at
> org.apache.flink.streaming.runtime.tasks.OneInputStreamTask.init(OneInputStreamTask.java:49)
>
>         at
> org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:239)
>
>         at org.apache.flink.runtime.taskmanager.Task.run(Task.java:584)
>
>         at java.lang.Thread.run(Thread.java:745)
>
>
>
>
>
> The job then restarts and fails again in an endless cycle.
>
>
>
> This feels like a configuration issue.  My guess is that Flink is looking
> for the file above on local storage, though we’ve configured checkpointing
> to use hdfs (see below).
>
>
>
> To enable checkpointing, this is what I did:
>
> env.enableCheckpointing(3000l);
>
>
>
> Relevant configurations in flink-conf.yaml:
>
> state.backend: filesystem
>
> state.backend.fs.checkpointdir:
> hdfs://myhadoopnamenode:8020/apps/flink/checkpoints
>
>
>
> Note, the directory we’ve configured is not the same as the path indicated
> in the error.
>
>
>
> Interestingly, there are plenty of subdirs in my checkpoints directory,
> these appear to correspond to job start times, even though these jobs don’t
> have checkpointing enabled:
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-13 07:48
> /apps/flink/checkpoints/b4870565f148cff10478dca8bff27bf7
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-13 08:27
> /apps/flink/checkpoints/044b21a0f252b6142e7ddfee7bfbd7d5
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-13 08:36
> /apps/flink/checkpoints/a658b23c2d2adf982a2cf317bfb3d3de
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-14 07:38
> /apps/flink/checkpoints/1156bd1796105ad95a8625cb28a0b816
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-14 07:41
> /apps/flink/checkpoints/58fdd94b7836a3b3ed9abc5c8f3a1dd5
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-14 07:43
> /apps/flink/checkpoints/47a849a8ed6538b9e7d3826a628d38b9
>
> drwxr-xr-x   - rtap hdfs          0 2016-10-14 07:49
> /apps/flink/checkpoints/e6a9e2300ea5c36341fa160adab789f0
>
>
>
> Thanks!
>
>
>
>
>
> ------------------------------
> The information contained in this communication is confidential and
> intended only for the use of the recipient named above, and may be legally
> privileged and exempt from disclosure under applicable law. If the reader
> of this message is not the intended recipient, you are hereby notified that
> any dissemination, distribution or copying of this communication is
> strictly prohibited. If you have received this communication in error,
> please resend it to the sender and delete the original message and copy of
> it from your computer system. Opinions, conclusions and other information
> in this message that do not relate to our official business should be
> understood as neither given nor endorsed by the company.
>

Reply via email to