Re: Flink job manager doesn't remove stale checkmarks

2019-09-26 Thread Clay Teeter
I looked into the disk issues and found that Fabian was on the right path.
The checkpoints that were lingering were in-fact in use.

Thanks for the help!
Clay



On Thu, Sep 26, 2019 at 8:09 PM Clay Teeter  wrote:

> I see, I'll try turning off incremental checkpoints to see if that helps.
>
> re: Diskspace, i could see a scenario with my application where i could
> get 10,000+ checkpoints, if the checkpoints are additive.  I'll let you
> know what i see.
>
> Thanks!
> Clay
>
>
> On Wed, Sep 25, 2019 at 5:40 PM Fabian Hueske  wrote:
>
>> Hi,
>>
>> You enabled incremental checkpoints.
>> This means that parts of older checkpoints that did not change since the
>> last checkpoint are not removed because they are still referenced by the
>> incremental checkpoints.
>> Flink will automatically remove them once they are not needed anymore.
>>
>> Are you sure that the size of your application's state is not growing too
>> large?
>>
>> Best, Fabian
>>
>> Am Di., 24. Sept. 2019 um 10:47 Uhr schrieb Clay Teeter <
>> clay.tee...@maalka.com>:
>>
>>> Oh geez,  checkmarks  = checkpoints... sorry.
>>>
>>> What i mean by stale "checkpoints" are checkpoints that should be reaped
>>> by: "state.checkpoints.num-retained: 3".
>>>
>>> What is happening is that directories:
>>>   - state.checkpoints.dir: file:///opt/ha/49/checkpoints
>>>   - high-availability.storageDir: file:///opt/ha/49/ha
>>> are growing with every checkpoint and i'm running out of disk space.
>>>
>>> On Tue, Sep 24, 2019 at 4:55 AM Biao Liu  wrote:
>>>
 Hi Clay,

 Sorry I don't get your point. I'm not sure what the "stale checkmarks"
 exactly means. The HA storage and checkpoint directory left after shutting
 down cluster?

 Thanks,
 Biao /'bɪ.aʊ/



 On Tue, 24 Sep 2019 at 03:12, Clay Teeter 
 wrote:

> I'm trying to get my standalone cluster to remove stale checkmarks.
>
> The cluster is composed of a single job and task manager backed by
> rocksdb with high availability.
>
> The configuration on both the job and task manager are:
>
> state.backend: rocksdb
> state.checkpoints.dir: file:///opt/ha/49/checkpoints
> state.backend.incremental: true
> state.checkpoints.num-retained: 3
> jobmanager.heap.size: 1024m
> taskmanager.heap.size: 2048m
> taskmanager.numberOfTaskSlots: 24
> parallelism.default: 1
> high-availability.jobmanager.port: 6123
> high-availability.zookeeper.path.root: _49
> high-availability: zookeeper
> high-availability.storageDir: file:///opt/ha/49/ha
> high-availability.zookeeper.quorum: **t:2181
>
> Both machines have access to /opt/ha/49 and /opt/ha/49/checkpoints via
> NFS and are owned by the flink user.  Also, there are no errors that i can
> find.
>
> Does anyone have any ideas that i could try?
>
>


Re: Flink job manager doesn't remove stale checkmarks

2019-09-26 Thread Clay Teeter
I see, I'll try turning off incremental checkpoints to see if that helps.

re: Diskspace, i could see a scenario with my application where i could get
10,000+ checkpoints, if the checkpoints are additive.  I'll let you know
what i see.

Thanks!
Clay


On Wed, Sep 25, 2019 at 5:40 PM Fabian Hueske  wrote:

> Hi,
>
> You enabled incremental checkpoints.
> This means that parts of older checkpoints that did not change since the
> last checkpoint are not removed because they are still referenced by the
> incremental checkpoints.
> Flink will automatically remove them once they are not needed anymore.
>
> Are you sure that the size of your application's state is not growing too
> large?
>
> Best, Fabian
>
> Am Di., 24. Sept. 2019 um 10:47 Uhr schrieb Clay Teeter <
> clay.tee...@maalka.com>:
>
>> Oh geez,  checkmarks  = checkpoints... sorry.
>>
>> What i mean by stale "checkpoints" are checkpoints that should be reaped
>> by: "state.checkpoints.num-retained: 3".
>>
>> What is happening is that directories:
>>   - state.checkpoints.dir: file:///opt/ha/49/checkpoints
>>   - high-availability.storageDir: file:///opt/ha/49/ha
>> are growing with every checkpoint and i'm running out of disk space.
>>
>> On Tue, Sep 24, 2019 at 4:55 AM Biao Liu  wrote:
>>
>>> Hi Clay,
>>>
>>> Sorry I don't get your point. I'm not sure what the "stale checkmarks"
>>> exactly means. The HA storage and checkpoint directory left after shutting
>>> down cluster?
>>>
>>> Thanks,
>>> Biao /'bɪ.aʊ/
>>>
>>>
>>>
>>> On Tue, 24 Sep 2019 at 03:12, Clay Teeter 
>>> wrote:
>>>
 I'm trying to get my standalone cluster to remove stale checkmarks.

 The cluster is composed of a single job and task manager backed by
 rocksdb with high availability.

 The configuration on both the job and task manager are:

 state.backend: rocksdb
 state.checkpoints.dir: file:///opt/ha/49/checkpoints
 state.backend.incremental: true
 state.checkpoints.num-retained: 3
 jobmanager.heap.size: 1024m
 taskmanager.heap.size: 2048m
 taskmanager.numberOfTaskSlots: 24
 parallelism.default: 1
 high-availability.jobmanager.port: 6123
 high-availability.zookeeper.path.root: _49
 high-availability: zookeeper
 high-availability.storageDir: file:///opt/ha/49/ha
 high-availability.zookeeper.quorum: **t:2181

 Both machines have access to /opt/ha/49 and /opt/ha/49/checkpoints via
 NFS and are owned by the flink user.  Also, there are no errors that i can
 find.

 Does anyone have any ideas that i could try?




Re: Flink job manager doesn't remove stale checkmarks

2019-09-25 Thread Fabian Hueske
Hi,

You enabled incremental checkpoints.
This means that parts of older checkpoints that did not change since the
last checkpoint are not removed because they are still referenced by the
incremental checkpoints.
Flink will automatically remove them once they are not needed anymore.

Are you sure that the size of your application's state is not growing too
large?

Best, Fabian

Am Di., 24. Sept. 2019 um 10:47 Uhr schrieb Clay Teeter <
clay.tee...@maalka.com>:

> Oh geez,  checkmarks  = checkpoints... sorry.
>
> What i mean by stale "checkpoints" are checkpoints that should be reaped
> by: "state.checkpoints.num-retained: 3".
>
> What is happening is that directories:
>   - state.checkpoints.dir: file:///opt/ha/49/checkpoints
>   - high-availability.storageDir: file:///opt/ha/49/ha
> are growing with every checkpoint and i'm running out of disk space.
>
> On Tue, Sep 24, 2019 at 4:55 AM Biao Liu  wrote:
>
>> Hi Clay,
>>
>> Sorry I don't get your point. I'm not sure what the "stale checkmarks"
>> exactly means. The HA storage and checkpoint directory left after shutting
>> down cluster?
>>
>> Thanks,
>> Biao /'bɪ.aʊ/
>>
>>
>>
>> On Tue, 24 Sep 2019 at 03:12, Clay Teeter  wrote:
>>
>>> I'm trying to get my standalone cluster to remove stale checkmarks.
>>>
>>> The cluster is composed of a single job and task manager backed by
>>> rocksdb with high availability.
>>>
>>> The configuration on both the job and task manager are:
>>>
>>> state.backend: rocksdb
>>> state.checkpoints.dir: file:///opt/ha/49/checkpoints
>>> state.backend.incremental: true
>>> state.checkpoints.num-retained: 3
>>> jobmanager.heap.size: 1024m
>>> taskmanager.heap.size: 2048m
>>> taskmanager.numberOfTaskSlots: 24
>>> parallelism.default: 1
>>> high-availability.jobmanager.port: 6123
>>> high-availability.zookeeper.path.root: _49
>>> high-availability: zookeeper
>>> high-availability.storageDir: file:///opt/ha/49/ha
>>> high-availability.zookeeper.quorum: **t:2181
>>>
>>> Both machines have access to /opt/ha/49 and /opt/ha/49/checkpoints via
>>> NFS and are owned by the flink user.  Also, there are no errors that i can
>>> find.
>>>
>>> Does anyone have any ideas that i could try?
>>>
>>>


Re: Flink job manager doesn't remove stale checkmarks

2019-09-24 Thread Clay Teeter
Oh geez,  checkmarks  = checkpoints... sorry.

What i mean by stale "checkpoints" are checkpoints that should be reaped
by: "state.checkpoints.num-retained: 3".

What is happening is that directories:
  - state.checkpoints.dir: file:///opt/ha/49/checkpoints
  - high-availability.storageDir: file:///opt/ha/49/ha
are growing with every checkpoint and i'm running out of disk space.

On Tue, Sep 24, 2019 at 4:55 AM Biao Liu  wrote:

> Hi Clay,
>
> Sorry I don't get your point. I'm not sure what the "stale checkmarks"
> exactly means. The HA storage and checkpoint directory left after shutting
> down cluster?
>
> Thanks,
> Biao /'bɪ.aʊ/
>
>
>
> On Tue, 24 Sep 2019 at 03:12, Clay Teeter  wrote:
>
>> I'm trying to get my standalone cluster to remove stale checkmarks.
>>
>> The cluster is composed of a single job and task manager backed by
>> rocksdb with high availability.
>>
>> The configuration on both the job and task manager are:
>>
>> state.backend: rocksdb
>> state.checkpoints.dir: file:///opt/ha/49/checkpoints
>> state.backend.incremental: true
>> state.checkpoints.num-retained: 3
>> jobmanager.heap.size: 1024m
>> taskmanager.heap.size: 2048m
>> taskmanager.numberOfTaskSlots: 24
>> parallelism.default: 1
>> high-availability.jobmanager.port: 6123
>> high-availability.zookeeper.path.root: _49
>> high-availability: zookeeper
>> high-availability.storageDir: file:///opt/ha/49/ha
>> high-availability.zookeeper.quorum: **t:2181
>>
>> Both machines have access to /opt/ha/49 and /opt/ha/49/checkpoints via
>> NFS and are owned by the flink user.  Also, there are no errors that i can
>> find.
>>
>> Does anyone have any ideas that i could try?
>>
>>


Re: Flink job manager doesn't remove stale checkmarks

2019-09-23 Thread Biao Liu
Hi Clay,

Sorry I don't get your point. I'm not sure what the "stale checkmarks"
exactly means. The HA storage and checkpoint directory left after shutting
down cluster?

Thanks,
Biao /'bɪ.aʊ/



On Tue, 24 Sep 2019 at 03:12, Clay Teeter  wrote:

> I'm trying to get my standalone cluster to remove stale checkmarks.
>
> The cluster is composed of a single job and task manager backed by rocksdb
> with high availability.
>
> The configuration on both the job and task manager are:
>
> state.backend: rocksdb
> state.checkpoints.dir: file:///opt/ha/49/checkpoints
> state.backend.incremental: true
> state.checkpoints.num-retained: 3
> jobmanager.heap.size: 1024m
> taskmanager.heap.size: 2048m
> taskmanager.numberOfTaskSlots: 24
> parallelism.default: 1
> high-availability.jobmanager.port: 6123
> high-availability.zookeeper.path.root: _49
> high-availability: zookeeper
> high-availability.storageDir: file:///opt/ha/49/ha
> high-availability.zookeeper.quorum: **t:2181
>
> Both machines have access to /opt/ha/49 and /opt/ha/49/checkpoints via NFS
> and are owned by the flink user.  Also, there are no errors that i can find.
>
> Does anyone have any ideas that i could try?
>
>


Flink job manager doesn't remove stale checkmarks

2019-09-23 Thread Clay Teeter
I'm trying to get my standalone cluster to remove stale checkmarks.

The cluster is composed of a single job and task manager backed by rocksdb
with high availability.

The configuration on both the job and task manager are:

state.backend: rocksdb
state.checkpoints.dir: file:///opt/ha/49/checkpoints
state.backend.incremental: true
state.checkpoints.num-retained: 3
jobmanager.heap.size: 1024m
taskmanager.heap.size: 2048m
taskmanager.numberOfTaskSlots: 24
parallelism.default: 1
high-availability.jobmanager.port: 6123
high-availability.zookeeper.path.root: _49
high-availability: zookeeper
high-availability.storageDir: file:///opt/ha/49/ha
high-availability.zookeeper.quorum: **t:2181

Both machines have access to /opt/ha/49 and /opt/ha/49/checkpoints via NFS
and are owned by the flink user.  Also, there are no errors that i can find.

Does anyone have any ideas that i could try?