[
https://issues.apache.org/jira/browse/FLINK-26586?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17521637#comment-17521637
]
Anton Kalashnikov commented on FLINK-26586:
-------------------------------------------
[~Matthias Schwalbe], big thanks for reporting this problem. Did you have this
problem only when you using StateProcessor or for recovery this problem is also
relevant?
According to implementation, I don't sure that we should implement
bufferization on StreamWrapper I mostly tend to implement it inside
`LocalDataInputStream`. Right now we use `FileInputStream` there but perhaps we
should use something with bufferization instead. But implementation is
discussable.
We also should check do we have a benchmark for a such scenario. If we don't
have such a benchmark(I think we don't) we will need to create one.
Do you want to work on this task? (I can assign it to you if so)
> FileSystem uses unbuffered read I/O
> -----------------------------------
>
> Key: FLINK-26586
> URL: https://issues.apache.org/jira/browse/FLINK-26586
> Project: Flink
> Issue Type: Improvement
> Components: API / State Processor, Connectors / FileSystem, Runtime
> / Checkpointing
> Affects Versions: 1.13.0, 1.14.0
> Reporter: Matthias Schwalbe
> Priority: Major
> Attachments: BufferedFSDataInputStreamWrapper.java,
> BufferedLocalFileSystem.java
>
>
> - I found out that, at least when using LocalFileSystem on a windows system,
> read I/O to load a savepoint is unbuffered,
> - See example stack [1]
> - i.e. in order to load only a long in a serializer, it needs to go into
> kernel mode 8 times and load the 8 bytes one by one
> - I coded a BufferedFSDataInputStreamWrapper that allows to opt-in buffered
> reads on any FileSystem implementation
> - In our setting savepoint load is now 30 times faster
> - I’ve once seen a Jira ticket as to improve savepoint load time in general
> (lost the link unfortunately), maybe this approach can help with it
> - not sure if HDFS has got the same problem
> - I can contribute my implementation of a BufferedFSDataInputStreamWrapper
> which can be integrated in any
> [1] unbuffered reads stack:
> read:207, FileInputStream (java.io)
> read:68, LocalDataInputStream (org.apache.flink.core.fs.local)
> read:50, FSDataInputStreamWrapper (org.apache.flink.core.fs)
> read:42, ForwardingInputStream (org.apache.flink.runtime.util)
> readInt:390, DataInputStream (java.io)
> deserialize:80, BytePrimitiveArraySerializer
> (org.apache.flink.api.common.typeutils.base.array)
> next:298, FullSnapshotRestoreOperation$KeyGroupEntriesIterator
> (org.apache.flink.runtime.state.restore)
> next:273, FullSnapshotRestoreOperation$KeyGroupEntriesIterator
> (org.apache.flink.runtime.state.restore)
> restoreKVStateData:147, RocksDBFullRestoreOperation
> (org.apache.flink.contrib.streaming.state.restore)
--
This message was sent by Atlassian Jira
(v8.20.1#820001)