[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2019-11-20 Thread Yu Li (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16979029#comment-16979029
 ] 

Yu Li edited comment on FLINK-7289 at 11/21/19 6:58 AM:


Here is a quick update on progress of the work:

Overall we need below actions to fully control the rocksdb memory usage:
 # Enabling {{setWriteBufferManager}} through JNI (frocksdb PR 
[#4|https://github.com/dataArtisans/frocksdb/pull/4] and 
[#5|https://github.com/dataArtisans/frocksdb/pull/5]).
 # Enabling {{setCacheIndexAndFilterBlocksWithHighPriority}} and 
{{setPinTopLevelIndexAndFilter}} through JNI (frocksdb PR 
[#6|https://github.com/dataArtisans/frocksdb/pull/6]).
 # Build a new frocksdb release to include all required PRs.
 # Supply an option to enable limiting overall memory usage in RocksDB backend.
 # Document how to enable the memory control and the pros. (container won't be 
killed again due to memory overuse) and cons. (performance impact).

Now all PRs required are ready, and we are working on releasing the new 
frocksdb release (we decided to stay on rocksdb 5.17.2 due to the [performance 
issue|https://github.com/facebook/rocksdb/issues/5774] of 5.18.3, more details 
please refer to FLINK-14483) and preparing the changes for RocksDB backend. 
Generally we are in good progress and should be able to complete the work 
before 1.10 feature freeze.


was (Author: carp84):
Here is a quick update on progress of the work:

Overall we need below actions to fully control the rocksdb memory usage:
 # Enabling setWriteBufferManager through JNI (frocksdb PR 
[#4|https://github.com/dataArtisans/frocksdb/pull/4] and 
[#5|https://github.com/dataArtisans/frocksdb/pull/5]).
 # Enabling setCacheIndexAndFilterBlocksWithHighPriority and 
setPinTopLevelIndexAndFilter through JNI (frocksdb PR 
[#6|https://github.com/dataArtisans/frocksdb/pull/6]).
 # Build a new frocksdb release to include all required PRs.
 # Supply an option to enable limiting overall memory usage in RocksDB backend.
 # Document how to enable the memory control and the pros. (container won't be 
killed again due to memory overuse) and cons. (performance impact).

Now all PRs required are ready, and we are working on releasing the new 
frocksdb release (we decided to stay on rocksdb 5.17.2 due to the [performance 
issue|https://github.com/facebook/rocksdb/issues/5774] of 5.18.3, more details 
please refer to FLINK-14483) and preparing the changes for RocksDB backend. 
Generally we are in good progress and should be able to complete the work 
before 1.10 feature freeze.

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / State Backends
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.7.2, 1.8.2, 1.9.0
>Reporter: Stefan Richter
>Priority: Major
> Fix For: 1.10.0
>
> Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2019-10-22 Thread shengjk1 (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16956749#comment-16956749
 ] 

shengjk1 edited comment on FLINK-7289 at 10/22/19 7:09 AM:
---

 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
{code:java}
class BackendOptions implements ConfigurableOptionsFactory {
 
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)

.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)n
.setBlockSize(4 * 32 * 1024))
;
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) {
  return this;
   }
}
{code}
 


was (Author: shengjk1):
 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
{code:java}

class BackendOptions implements ConfigurableOptionsFactory {
   //
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)

.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)
//  increases read amplification but decreases memory 
useage and space amplification
.setBlockSize(4 * 32 * 1024))
;
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) {
  return this;
   }
}
{code}
 

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / State Backends
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.7.2, 1.8.2, 1.9.0
>Reporter: Stefan Richter
>Priority: Major
> Fix For: 1.10.0
>
> Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run in

[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2019-10-22 Thread shengjk1 (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16956749#comment-16956749
 ] 

shengjk1 edited comment on FLINK-7289 at 10/22/19 7:09 AM:
---

 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
{code:java}

class BackendOptions implements ConfigurableOptionsFactory {
   //
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)

.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)
//  increases read amplification but decreases memory 
useage and space amplification
.setBlockSize(4 * 32 * 1024))
;
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) {
  return this;
   }
}
{code}
 


was (Author: shengjk1):
 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
class BackendOptions implements ConfigurableOptionsFactory \{
   //
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) \{
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)

.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)
.setBlockSize(4 * 32 * 1024))
;
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) \{
  return this;
   }
}

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / State Backends
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.7.2, 1.8.2, 1.9.0
>Reporter: Stefan Richter
>Priority: Major
> Fix For: 1.10.0
>
> Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same p

[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2019-10-22 Thread shengjk1 (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16956749#comment-16956749
 ] 

shengjk1 edited comment on FLINK-7289 at 10/22/19 7:10 AM:
---

 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
{code:java}
class BackendOptions implements ConfigurableOptionsFactory {
 
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)
.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)n
.setBlockSize(4 * 32 * 1024));
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) {
  return this;
   }
}
{code}
 


was (Author: shengjk1):
 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
{code:java}
class BackendOptions implements ConfigurableOptionsFactory {
 
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)

.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)n
.setBlockSize(4 * 32 * 1024))
;
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) {
  return this;
   }
}
{code}
 

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / State Backends
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.7.2, 1.8.2, 1.9.0
>Reporter: Stefan Richter
>Priority: Major
> Fix For: 1.10.0
>
> Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory

[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2019-10-22 Thread shengjk1 (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16956749#comment-16956749
 ] 

shengjk1 edited comment on FLINK-7289 at 10/22/19 7:11 AM:
---

 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.

 


was (Author: shengjk1):
 I also rebuild flink and use writeBufferManager. For one specific job I'm 
working on, I went from a OOM kill every 5-7 hours to at least 20 hours of no 
OOM events . But finally it still OOM . So I think maybe we also need other 
parameters which easy to use to solve it.
{code:java}
class BackendOptions implements ConfigurableOptionsFactory {
 
   private static final WriteBufferManager writeBufferManager = new 
WriteBufferManager(1 << 30, new LRUCache(1 << 18));
   
   @Override
   public DBOptions createDBOptions(DBOptions currentOptions) {
  return currentOptions
.setMaxBackgroundJobs(4)
.setUseFsync(false)
.setMaxBackgroundFlushes(3)
.setWriteBufferManager(writeBufferManager);
   }
   
   @Override
   public ColumnFamilyOptions createColumnOptions(ColumnFamilyOptions 
currentOptions) {
  return currentOptions
.setLevelCompactionDynamicLevelBytes(true)
.setMinWriteBufferNumberToMerge(2)
.setMaxWriteBufferNumber(5)
.setOptimizeFiltersForHits(true)
.setMaxWriteBufferNumberToMaintain(3)
.setTableFormatConfig(
  new BlockBasedTableConfig()
.setCacheIndexAndFilterBlocks(true)
.setCacheIndexAndFilterBlocksWithHighPriority(true)
.setBlockCacheSize(256 * 1024 * 1024)n
.setBlockSize(4 * 32 * 1024));
   }
   
   @Override
   public OptionsFactory configure(Configuration configuration) {
  return this;
   }
}
{code}
 

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / State Backends
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.7.2, 1.8.2, 1.9.0
>Reporter: Stefan Richter
>Priority: Major
> Fix For: 1.10.0
>
> Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2019-10-28 Thread shengjk1 (Jira)


[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16961652#comment-16961652
 ] 

shengjk1 edited comment on FLINK-7289 at 10/29/19 3:55 AM:
---

Thanks [~yunta] and [~mikekap]. I have solved it. As Mike Kaplinskiy said, i 
put the jar  which only include BackendOptions.class  in the flink boot 
classpath - not in my application code classpath.


was (Author: shengjk1):
Thanks [~yunta] and [~mikekap]. I have solved it. As Mike Kaplinskiy said, i 
put the jar  which only include BackendOptions.class  in the flink boot 
classpath - not in my application code classpath .

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / State Backends
>Affects Versions: 1.2.0, 1.3.0, 1.4.0, 1.7.2, 1.8.2, 1.9.0
>Reporter: Stefan Richter
>Priority: Major
> Fix For: 1.10.0
>
> Attachments: completeRocksdbConfig.txt
>
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2017-07-29 Thread Vinay (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16106133#comment-16106133
 ] 

Vinay edited comment on FLINK-7289 at 7/29/17 4:18 PM:
---

Hi Stefan,

I have mainly used RocksDB on EMR backed up by SSD's and 122GB memory. 
Although  FLASH_SSD_OPTION is good, it does not provide control over the amount 
of memory to be used. So I had tuned some parameters with the below 
configurations :

{code:java}

DBOptions:
 (along with the FLASH_SSD_OPTIONS add the following)
 maxBackgroundCompactions(4)

ColumnFamilyOptions:
  max_buffer_size : 512 MB
  block_cache_size : 128 MB
  max_write_buffer_number : 5
  minimum_buffer_number_to_merge : 2
  cacheIndexAndFilterBlocks : true
  optimizeFilterForHits: true
{code}

According to the documentation when {code:java}  optimizeFilterForHits: true 
{code} is set, RocksDB will not build bloom filters on the last level which 
contains 90% of DB. Thus the memory usage for bloom filters will be 10x less.

As RocksDB uses a lot of memory , if we cancel the job in between the memory 
used is not reclaimed. For Example: assuming that the job is running for 1 hour 
and the memory used is 50GB , now when we cancel the job from UI the memory is 
not reclaimed.
I have observed this case when I had run the job on YARN.

In order to reclaim the memory I had to manually run the following command on 
each node of EMR:
{code:java}
sync; echo 3 > /proc/sys/vm/drop_caches
sync; echo 2 > /proc/sys/vm/drop_caches
sync; echo 1 > /proc/sys/vm/drop_caches
{code}



was (Author: vinaypatil18):
Hi Stefan,

I have mainly used RocksDB on EMR backed up by SSD's and 122GB memory. 
Although  FLASH_SSD_OPTION is good, it does not provide control over the amount 
of memory to be used. So I had tuned some parameters with the below 
configurations :

{code:java}

DBOptions:
 (along with the FLASH_SSD_OPTIONS add the following)
 maxBackgroundCompactions(4)

ColumnFamilyOptions:
  max_buffer_size : 512 MB
  block_cache_size : 128 MB
  max_write_buffer_number : 5
  minimum_buffer_number_to_merge : 2
  cacheIndexAndFilterBlocks : true
  optimizeFilterForHits: true
{code}

According to the documentation when {code:java}  optimizeFilterForHits: true 
{code} is set, RocksDB will not build bloom filters on the last level which 
contains 90% of DB. Thus the memory usage for bloom filters will be 10x less.

As RocksDB uses a lot of memory , if we cancel the job in between the memory 
used is not reclaimed. For Example: assuming that the job is running for 1 hour 
and the memory used is 50GB , now when we cancel the job from UI the memory is 
not reclaimed.
I have observed this case when I had run the job on YARN.

I order to reclaim the memory I had to manually run the following command on 
each node of EMR:
{code:java}
sync; echo 3 > /proc/sys/vm/drop_caches
sync; echo 2 > /proc/sys/vm/drop_caches
sync; echo 1 > /proc/sys/vm/drop_caches
{code}


> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: State Backends, Checkpointing
>Affects Versions: 1.2.0, 1.3.0, 1.4.0
>Reporter: Stefan Richter
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2017-07-31 Thread Stefan Richter (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16107244#comment-16107244
 ] 

Stefan Richter edited comment on FLINK-7289 at 7/31/17 12:33 PM:
-

Cache memory is usually not reclaimed by applications, it is managed by the OS. 
When the OS is not replacing something in the file system cache, it should 
indicate that the OS thinks that there is still plenty of free memory available 
and no need to already start with cache replacements. The caching behaviour in 
general can be configured in Linux, e.g. through the "swappiness" parameter. So 
this can potentially be a problem with how the OS is configured, maybe w.r.t. 
the additional constraints the container environment puts on it?


was (Author: srichter):
Cache memory is usually not reclaimed by applications, it is managed by the OS. 
When the OS is not replacing something in the file system cache, it should 
indicate that the OS thinks that there is still plenty of free memory available 
and no need to already start with cache replacements. The caching behaviour in 
general can be configured Linux, e.g. through the "swappiness" parameter. So 
this can potentially be a problem with how the OS is configured, maybe w.r.t. 
the additional constraints the container environment puts on it?

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: State Backends, Checkpointing
>Affects Versions: 1.2.0, 1.3.0, 1.4.0
>Reporter: Stefan Richter
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2017-08-01 Thread Vinay (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16109261#comment-16109261
 ] 

Vinay edited comment on FLINK-7289 at 8/1/17 4:42 PM:
--

Hi Stephan,

I agree with what you are saying, But I am saying this from the end user 
perspective. The user will assume that enough memory is available when the job 
gets canceled or killed and will re-run it.

I am just suggesting that if Flink could somehow clean the memory or flush it 
to disk when the job is canceled or killed.


was (Author: vinaypatil18):
Hi Stephan,

I agree with what you are saying, But I am saying this from the end user 
perspective. The user will assume that enough memory is available when the job 
gets canceled or killed and will re-run it.

I am just suggesting that if Flink could somehow clean the memory or flush it 
disk when the job is canceled or killed.

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: State Backends, Checkpointing
>Affects Versions: 1.2.0, 1.3.0, 1.4.0
>Reporter: Stefan Richter
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2017-08-01 Thread Stefan Richter (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16109411#comment-16109411
 ] 

Stefan Richter edited comment on FLINK-7289 at 8/1/17 5:54 PM:
---

Hi,

I understand that cache memory is not freed, and this sounds exactly like what 
I would expect. I would also expect that the consumed cache memory will not go 
down as long as it is still below available cache memory configured from the OS 
perspective. If this becomes a problem under YARN, this sounds like a problem 
of the cluster setup to me. Take this opinion with a grain of sand, I am not an 
expert on YARN or container setups. 

Then this should not be an end user problem, but also not a Flink problem. It 
sounds like an administrator and configuration problem to me. For example, this 
caching scenario should also apply to all other filesystem reads / writes and 
not only to RocksDB. Manually dropping OS file caches should never been 
required from any application or user, and if so it seems like this is fixing 
the symptoms of a different problem. We can try to figure out more about root 
problem and then maybe come up with a better documentation for the setup and 
configuration. But I would disagree about introducing cache cleaning to Flink 
because writing to {{/proc/sys/vm/drop_caches}} is an operation that requires 
root privileges and can affect other processes performance.

Can you provide us with some more information, e.g. a detailed breakdown of the 
os memory consumption after the process ended and the logs about the killed 
containers?


was (Author: srichter):
Hi,

I understand that cache memory is not freed, and this sounds exactly like what 
I would expect. I would also expect that the consumed cache memory will not go 
down as long as it is still below available cache memory configured from the OS 
perspective. If this becomes a problem under YARN, this sounds like a problem 
of the cluster setup to me. Take this opinion with a grain of sand, I am not an 
expert on YARN or container setups. 

Then this should not be an end user problem, but also not a Flink problem. It 
sounds like an administrator and configuration problem to me. For example, this 
caching scenario should also apply to all other filesystem reads / writes and 
not only to RocksDB. Manually dropping OS file caches should never been 
required from any application or user, and if so it seems like this is fixing 
the symptoms of a different problem. We can try to figure out more about root 
problem and then maybe come up with a better documentation for the setup and 
configuration. But I would disagree about introducing cache cleaning to Flink 
because writing to {{/proc/sys/vm/drop_caches}} is an operation that requires 
root privileges.

Can you provide us with some more information, e.g. a detailed breakdown of the 
os memory consumption after the process ended and the logs about the killed 
containers?

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: State Backends, Checkpointing
>Affects Versions: 1.2.0, 1.3.0, 1.4.0
>Reporter: Stefan Richter
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
> can (theoretically) grow without limits.
> In container environments, this can be problematic because the process can 
> exceed the memory budget of the container, and the process will get killed. 
> Currently, there is no other option than trusting RocksDB to be well behaved 
> and to follow its memory configurations. However, limiting RocksDB's memory 
> usage is not as easy as setting a single limit parameter. The memory limit is 
> determined by an interplay of several configuration parameters, which is 
> almost impossible to get right for users. Even worse, multiple RocksDB 
> instances can run inside the same process and make reasoning about the 
> configuration also dependent on the Flink job.
> Some information about the memory management in RocksDB can be found here:
> https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB
> https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide
> We should try to figure out ways to help users in one or more of the 
> following ways:
> - Some way to autotune or calculate the RocksDB configuration.
> - Conservative default values.
> - Additional documentation.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)


[jira] [Comment Edited] (FLINK-7289) Memory allocation of RocksDB can be problematic in container environments

2017-08-01 Thread Stefan Richter (JIRA)

[ 
https://issues.apache.org/jira/browse/FLINK-7289?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16109411#comment-16109411
 ] 

Stefan Richter edited comment on FLINK-7289 at 8/1/17 6:01 PM:
---

Hi,

I understand that cache memory is not freed, and this sounds exactly like what 
I would expect. I would also expect that the consumed cache memory will not go 
down as long as it is still below available cache memory configured from the OS 
perspective. If this becomes a problem under YARN, this sounds like a problem 
of the cluster setup to me. Take this opinion with a grain of sand, I am not an 
expert on YARN or container setups. 

Then this should not be an end user problem, but also not a Flink problem. It 
sounds like an administrator and configuration problem to me. For example, this 
caching scenario should also apply to all other filesystem reads / writes and 
not only to RocksDB. Manually dropping OS file caches should never been 
required from any application or user, and if so it seems like this is fixing 
the symptoms of a different problem. We can try to figure out more about root 
problem and then maybe come up with a better documentation for the setup and 
configuration. But I would disagree about introducing cache cleaning to Flink 
because writing to {{/proc/sys/vm/drop_caches}} is an operation that requires 
root privileges and can affect other processes performance. From the 
documentation:

{quote}
drop_caches

Writing to this will cause the kernel to drop clean caches, as well as
reclaimable slab objects like dentries and inodes.  Once dropped, their
memory becomes free.

To free pagecache:
echo 1 > /proc/sys/vm/drop_caches
To free reclaimable slab objects (includes dentries and inodes):
echo 2 > /proc/sys/vm/drop_caches
To free slab objects and pagecache:
echo 3 > /proc/sys/vm/drop_caches

This is a non-destructive operation and will not free any dirty objects.
To increase the number of objects freed by this operation, the user may run
`sync' prior to writing to /proc/sys/vm/drop_caches.  This will minimize the
number of dirty objects on the system and create more candidates to be
dropped.

This file is not a means to control the growth of the various kernel caches
(inodes, dentries, pagecache, etc...)  These objects are automatically
reclaimed by the kernel when memory is needed elsewhere on the system.

Use of this file can cause performance problems.  Since it discards cached
objects, it may cost a significant amount of I/O and CPU to recreate the
dropped objects, especially if they were under heavy use.  Because of this,
use outside of a testing or debugging environment is not recommended.
{quote}

Can you provide us with some more information, e.g. a detailed breakdown of the 
os memory consumption after the process ended and the logs about the killed 
containers?


was (Author: srichter):
Hi,

I understand that cache memory is not freed, and this sounds exactly like what 
I would expect. I would also expect that the consumed cache memory will not go 
down as long as it is still below available cache memory configured from the OS 
perspective. If this becomes a problem under YARN, this sounds like a problem 
of the cluster setup to me. Take this opinion with a grain of sand, I am not an 
expert on YARN or container setups. 

Then this should not be an end user problem, but also not a Flink problem. It 
sounds like an administrator and configuration problem to me. For example, this 
caching scenario should also apply to all other filesystem reads / writes and 
not only to RocksDB. Manually dropping OS file caches should never been 
required from any application or user, and if so it seems like this is fixing 
the symptoms of a different problem. We can try to figure out more about root 
problem and then maybe come up with a better documentation for the setup and 
configuration. But I would disagree about introducing cache cleaning to Flink 
because writing to {{/proc/sys/vm/drop_caches}} is an operation that requires 
root privileges and can affect other processes performance.

Can you provide us with some more information, e.g. a detailed breakdown of the 
os memory consumption after the process ended and the logs about the killed 
containers?

> Memory allocation of RocksDB can be problematic in container environments
> -
>
> Key: FLINK-7289
> URL: https://issues.apache.org/jira/browse/FLINK-7289
> Project: Flink
>  Issue Type: Improvement
>  Components: State Backends, Checkpointing
>Affects Versions: 1.2.0, 1.3.0, 1.4.0
>Reporter: Stefan Richter
>
> Flink's RocksDB based state backend allocates native memory. The amount of 
> allocated memory by RocksDB is not under the control of Flink or the JVM and 
>