Hi, if the problem is seemingly from reads, I think incremental checkpoints are less likely to cause the problem. What Flink version are you using? Since you mentioned the use of map state, what comes to my mind as a potential cause is described in this issue https://issues.apache.org/jira/browse/FLINK-8639 <https://issues.apache.org/jira/browse/FLINK-8639> . This was improved recently. Does the problem also exist for jobs without map state?
Best, Stefan > Am 24.05.2018 um 20:25 schrieb Stephan Ewen <[email protected]>: > > One thing that you can always to is disable fsync, because Flink does not > rely on RocksDBs fsync for persistence. > > If you disable incremental checkpoints, does that help? > If yes, it could be an issue with too many small SSTable files due to > incremental checkpoints (an issue we have on the roadmap to fix). > > On Thu, May 24, 2018 at 3:52 PM, Piotr Nowojski <[email protected] > <mailto:[email protected]>> wrote: > Hi, > > This issue might have something to do with compaction. Problems with > compaction can especially degrade reads performance (or just increase reads > IO). Have you tried to further enforce more compactions or change > CompactionStyle? > > Have you taken a look on > org.apache.flink.contrib.streaming.state.PredefinedOptions? > > Maybe Stefan or Andrey could share more input on this. > > Piotrek > > > > On 22 May 2018, at 08:12, Govindarajan Srinivasaraghavan > > <[email protected] <mailto:[email protected]>> wrote: > > > > Hi All, > > > > We are running flink in AWS and we are observing a strange behavior. We are > > using docker containers, EBS for storage and Rocks DB state backend. We > > have a few map and value states with checkpointing every 30 seconds and > > incremental checkpointing turned on. The issue we are noticing is the read > > IOPS and read throughput gradually increases over time and keeps constantly > > growing. The write throughput and write bytes are not increasing as much as > > reads. The checkpoints are written to a durable NFS storage. We are not > > sure what is causing this constant increase in read throughput but due to > > which we are running out of EBS burst balance and need to restart the job > > every once in a while. Attached the EBS read and write metrics. Has anyone > > encountered this issue and what could be the possible solution. > > > > We have also tried setting the below rocksdb options but didn't help. > > > > DBOptions: > > currentOptions.setOptimizeFiltersForHits(true) > > .setWriteBufferSize(536870912) > > .setMaxWriteBufferNumber(5) > > .setMinWriteBufferNumberToMerge(2); > > ColumnFamilyOptions: > > > > currentOptions.setMaxBackgroundCompactions(4) > > .setMaxManifestFileSize(1048576) > > .setMaxLogFileSize(1048576); > > > > > > > > Thanks. > > > > > > > >
