Re: external checkpoints

2017-11-27 Thread Aljoscha Krettek
Hi, I think we might have a slight misunderstanding here. In most situations you do in fact have to configure two directories: - the directory where the metadata for externalised checkpoints is stored, this would be "state.checkpoints.dir" - the directory where the actual checkpoint data shoul

Re: external checkpoints

2017-11-24 Thread Fabian Hueske
Hi Aviad, sorry for the late reply. You can configure the checkpoint directory (which is also used for externalized checkpoints) when you create the state backend: env.setStateBackend(new RocksDBStateBackend("hdfs:///checkpoints-data/"); This configures the checkpoint directory to be hdfs:///che

Re: external checkpoints

2017-11-16 Thread aviad
Hi, thanks for the answer. I can use the first option (REST API). for some reason it is undocumented in flink documentation (https://ci.apache.org/projects/flink/flink-docs-release-1.3/monitoring/rest_api.html) regarding the second option, configure each job with an externalized checkpoint direct

Re: external checkpoints

2017-11-15 Thread Jins George
Hi Aviad, I had a similar situation and my solution was to use the flink monitoring rest api (/jobs/{jobid}/checkpoints) to get the mapping between job and checkpoint file. Wrap this in a script and run periodically( in my case, it was 30 sec). You can also configure each job with an external

Re: External checkpoints not getting cleaned up/discarded - potentially causing high load

2017-09-04 Thread Stefan Richter
Hi Jared, I just wanted to follow up on this problem that you reported. Are there any new insights about this problem from your debugging efforts and does it still exists for you? Best, Stefan > Am 09.07.2017 um 18:37 schrieb Jared Stehler > : > > We are using the rocksDB state backend. We h

Re: External checkpoints not getting cleaned up/discarded - potentially causing high load

2017-07-09 Thread Jared Stehler
We are using the rocksDB state backend. We had not activated incremental checkpointing, but in the course of debugging this, we ended up doing so, and also moving back to S3 from EFS as it appeared that EFS was introducing large latencies. I will attempt to provide some profiler data as we are a

Re: External checkpoints not getting cleaned up/discarded - potentially causing high load

2017-07-03 Thread Ufuk Celebi
On Mon, Jul 3, 2017 at 12:02 PM, Stefan Richter wrote: > Another thing that could be really helpful, if possible, can you attach a > profiler/sampling to your job manager and figure out the hotspot methods > where most time is spend? This would be very helpful as a starting point > where the probl

Re: External checkpoints not getting cleaned up/discarded - potentially causing high load

2017-07-03 Thread Stefan Richter
Hi, I have two quick questions about this problem report: 1) Which state backend are you using? 2) In case you are using RocksDB, did you also activate incremental checkpointing when moving to Flink 1.3. Another thing that could be really helpful, if possible, can you attach a profiler/samplin