Hi Piotr, I did some analysis and realised that the temp files for s3 checkpoints are staged in /tmp although the *io.tmp.dirs *is set to a different directory.
ls -lrth drwxr-xr-x. 2 was was 32 Aug 20 17:52 hsperfdata_was -rw-------. 1 was was 505M Aug 20 18:45 presto-s3-8158855975833379228.tmp -rw-------. 1 was was 505M Aug 20 18:45 presto-s3-7048419193714606532.tmp drwxr-xr--. 2 root root 6 Aug 20 18:46 hsperfdata_root [was@sl73rspapd031 tmp]$ flink-conf.yaml configuration io.tmp.dirs: /usr/mware/flink/tmp The /tmp has only 2GB, is it possible to change the staging directory for s3 checkpoints ? Best, Vishwas On Thu, Aug 20, 2020 at 10:27 AM Vishwas Siravara <vsirav...@gmail.com> wrote: > Hi Piotr, > Thank you for your suggestion. I will try that, are the temporary files > created in the directory set in *io.tmp.dirs* in the flink-conf.yaml ? > Would these files be the same size as checkpoints ? > > > Thanks, > Vishwas > > On Thu, Aug 20, 2020 at 8:35 AM Piotr Nowojski <pnowoj...@apache.org> > wrote: > >> Hi, >> >> As far as I know when uploading a file to S3, the writer needs to first >> create some temporary files on the local disks. I would suggest to double >> check all of the partitions on the local machine and monitor available disk >> space continuously while the job is running. If you are just checking the >> free space manually, you can easily miss a point of time when you those >> temporary files are too big and approaching the available disk space usage, >> as I'm pretty sure those temporary files are cleaned up immediately after >> throwing this exception that you see. >> >> Piotrek >> >> czw., 20 sie 2020 o 00:56 Vishwas Siravara <vsirav...@gmail.com> >> napisaĆ(a): >> >>> Hi guys, >>> I have a deduplication job that runs on flink 1.7, that has some state >>> which uses FsState backend. My TM heap size is 16 GB. I see the below error >>> while trying to checkpoint a state of size 2GB. There is enough space >>> available in s3, I tried to upload larger files and they were all >>> successful. There is also enough disk space in the local file system, the >>> disk utility tool does not show anything suspicious. Whenever I try to >>> start my job from the last successful checkpoint , it runs into the same >>> error. Can someone tell me what is the cause of this issue? Many thanks. >>> >>> >>> Note: This error goes away when I delete io.tmp.dirs and restart the >>> job from last checkpoint , but the disk utility tool does not show much >>> usage before deletion, so I am not able to figure out what the problem is. >>> >>> 2020-08-19 21:12:01,909 WARN >>> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory - Could >>> not close the state stream for s3p://featuretoolkit.c >>> heckpoints/dev_dedup/9b64aafadcd6d367cfedef84706abcba/chk-189/f8e668dd-8019-4830-ab12-d48940ff5353. >>> 1363 java.io.IOException: No space left on device >>> 1364 at java.io.FileOutputStream.writeBytes(Native Method) >>> 1365 at java.io.FileOutputStream.write(FileOutputStream.java:326) >>> 1366 at >>> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82) >>> 1367 at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:140) >>> 1368 at java.io.FilterOutputStream.flush(FilterOutputStream.java:140) >>> 1369 at java.io.FilterOutputStream.close(FilterOutputStream.java:158) >>> 1370 at >>> org.apache.flink.fs.s3presto.shaded.com.facebook.presto.hive.PrestoS3FileSystem$PrestoS3OutputStream.close(PrestoS3FileSystem.java:986) >>> 1371 at >>> org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) >>> 1372 at >>> org.apache.flink.fs.shaded.hadoop3.org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) >>> 1373 at >>> org.apache.flink.fs.s3.common.hadoop.HadoopDataOutputStream.close(HadoopDataOutputStream.java:52) >>> 1374 at >>> org.apache.flink.core.fs.ClosingFSDataOutputStream.close(ClosingFSDataOutputStream.java:64) >>> 1375 at >>> org.apache.flink.runtime.state.filesystem.FsCheckpointStreamFactory$FsCheckpointStateOutputStream.close(FsCheckpointStreamFactory.java:269) >>> 1376 at >>> org.apache.flink.runtime.state.CheckpointStreamWithResultProvider.close(CheckpointStreamWithResultProvider.java:58) >>> 1377 at org.apache.flink.util.IOUtils.closeQuietly(IOUtils.java:263) >>> 1378 at org.apache.flink.util.IOUtils.closeAllQuietly(IOUtils.java:250) >>> 1379 at >>> org.apache.flink.util.AbstractCloseableRegistry.close(AbstractCloseableRegistry.java:122) >>> 1380 at >>> org.apache.flink.runtime.state.AsyncSnapshotCallable.closeSnapshotIO(AsyncSnapshotCallable.java:185) >>> 1381 at >>> org.apache.flink.runtime.state.AsyncSnapshotCallable.call(AsyncSnapshotCallable.java:84) >>> 1382 at java.util.concurrent.FutureTask.run(FutureTask.java:266) >>> 1383 at >>> org.apache.flink.util.FutureUtil.runIfNotDoneAndGet(FutureUtil.java:50) >>> 1384 at >>> org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer.<init>(OperatorSnapshotFinalizer.java:47) >>> 1385 at >>> org.apache.flink.streaming.runtime.tasks.StreamTask$AsyncCheckpointRunnable.run(StreamTask.java:853) >>> 1386 at >>> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) >>> 1387 at java.util.concurrent.FutureTask.run(FutureTask.java:266) >>> 1388 at >>> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) >>> 1389 at >>> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) >>> 1390 at java.lang.Thread.run(Thread.java:748) >>> 1391 Suppressed: java.io.IOException: No space left on device >>> 1392 at java.io.FileOutputStream.writeBytes(Native Method) >>> 1393 at java.io.FileOutputStream.write(FileOutputStream.java:326) >>> 1394 at >>> java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82) >>> 1395 at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:140) >>> 1396 at java.io.FilterOutputStream.close(FilterOutputStream.java:158) >>> 1397 at java.io.FilterOutputStream.close(FilterOutputStream.java:159) >>> 1398 ... 21 more >>> >>> >>> Thanks, >>> Vishwas >>> >>