forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 1
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-24489][CEP] The size of entryCache & eventsBufferCache in the …
…SharedBuffer should be defined with a threshold to limit the number of the elements in the cache Co-authored-by: Arvid Heise <[email protected]>
- Loading branch information
Showing
19 changed files
with
470 additions
and
101 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
30 changes: 30 additions & 0 deletions
30
docs/layouts/shortcodes/generated/cep_cache_configuration.html
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,30 @@ | ||
<table class="configuration table table-bordered"> | ||
<thead> | ||
<tr> | ||
<th class="text-left" style="width: 20%">Key</th> | ||
<th class="text-left" style="width: 15%">Default</th> | ||
<th class="text-left" style="width: 10%">Type</th> | ||
<th class="text-left" style="width: 55%">Description</th> | ||
</tr> | ||
</thead> | ||
<tbody> | ||
<tr> | ||
<td><h5>pipeline.cep.sharedbuffer.cache.entry-slots</h5></td> | ||
<td style="word-wrap: break-word;">1024</td> | ||
<td>Integer</td> | ||
<td>The Config option to set the maximum element number the entryCache of SharedBuffer could hold. And it could accelerate the CEP operate process speed with state.And it could accelerate the CEP operate process speed and limit the capacity of cache in pure memory. Note: It's only effective to limit usage of memory when 'state.backend' was set as 'rocksdb', which would transport the elements exceeded the number of the cache into the rocksdb state storage instead of memory state storage.</td> | ||
</tr> | ||
<tr> | ||
<td><h5>pipeline.cep.sharedbuffer.cache.event-slots</h5></td> | ||
<td style="word-wrap: break-word;">1024</td> | ||
<td>Integer</td> | ||
<td>The Config option to set the maximum element number the eventsBufferCache of SharedBuffer could hold. And it could accelerate the CEP operate process speed and limit the capacity of cache in pure memory. Note: It's only effective to limit usage of memory when 'state.backend' was set as 'rocksdb', which would transport the elements exceeded the number of the cache into the rocksdb state storage instead of memory state storage.</td> | ||
</tr> | ||
<tr> | ||
<td><h5>pipeline.cep.sharedbuffer.cache.statistics-interval</h5></td> | ||
<td style="word-wrap: break-word;">30 min</td> | ||
<td>Duration</td> | ||
<td>The interval to log the information of cache state statistics in CEP operator.</td> | ||
</tr> | ||
</tbody> | ||
</table> |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
65 changes: 65 additions & 0 deletions
65
...libraries/flink-cep/src/main/java/org/apache/flink/cep/configuration/CEPCacheOptions.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http:https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.cep.configuration; | ||
|
||
import org.apache.flink.configuration.ConfigOption; | ||
import org.apache.flink.configuration.ConfigOptions; | ||
import org.apache.flink.util.TimeUtils; | ||
|
||
import java.time.Duration; | ||
|
||
/** CEP Cache Options. */ | ||
public class CEPCacheOptions { | ||
|
||
private CEPCacheOptions() {} | ||
|
||
private static final String COMMON_HINT = | ||
"And it could accelerate the CEP operate process " | ||
+ "speed and limit the capacity of cache in pure memory. Note: It's only effective to " | ||
+ "limit usage of memory when 'state.backend' was set as 'rocksdb', which would " | ||
+ "transport the elements exceeded the number of the cache into the rocksdb state " | ||
+ "storage instead of memory state storage."; | ||
|
||
public static final ConfigOption<Integer> CEP_SHARED_BUFFER_EVENT_CACHE_SLOTS = | ||
ConfigOptions.key("pipeline.cep.sharedbuffer.cache.event-slots") | ||
.intType() | ||
.defaultValue(1024) | ||
.withDescription( | ||
"The Config option to set the maximum element number the " | ||
+ "eventsBufferCache of SharedBuffer could hold. " | ||
+ COMMON_HINT); | ||
|
||
public static final ConfigOption<Integer> CEP_SHARED_BUFFER_ENTRY_CACHE_SLOTS = | ||
ConfigOptions.key("pipeline.cep.sharedbuffer.cache.entry-slots") | ||
.intType() | ||
.defaultValue(1024) | ||
.withDescription( | ||
"The Config option to set the maximum element number the entryCache" | ||
+ " of SharedBuffer could hold. And it could accelerate the" | ||
+ " CEP operate process speed with state." | ||
+ COMMON_HINT); | ||
|
||
public static final ConfigOption<Duration> CEP_CACHE_STATISTICS_INTERVAL = | ||
ConfigOptions.key("pipeline.cep.sharedbuffer.cache.statistics-interval") | ||
.durationType() | ||
.defaultValue(TimeUtils.parseDuration("30 min")) | ||
.withDescription( | ||
"The interval to log the information of cache state statistics in " | ||
+ "CEP operator."); | ||
} |
70 changes: 70 additions & 0 deletions
70
...s/flink-cep/src/main/java/org/apache/flink/cep/configuration/SharedBufferCacheConfig.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,70 @@ | ||
/* | ||
* Licensed to the Apache Software Foundation (ASF) under one | ||
* or more contributor license agreements. See the NOTICE file | ||
* distributed with this work for additional information | ||
* regarding copyright ownership. The ASF licenses this file | ||
* to you under the Apache License, Version 2.0 (the | ||
* "License"); you may not use this file except in compliance | ||
* with the License. You may obtain a copy of the License at | ||
* | ||
* http:https://www.apache.org/licenses/LICENSE-2.0 | ||
* | ||
* Unless required by applicable law or agreed to in writing, software | ||
* distributed under the License is distributed on an "AS IS" BASIS, | ||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
* See the License for the specific language governing permissions and | ||
* limitations under the License. | ||
*/ | ||
|
||
package org.apache.flink.cep.configuration; | ||
|
||
import org.apache.flink.configuration.ReadableConfig; | ||
|
||
import java.io.Serializable; | ||
import java.time.Duration; | ||
|
||
/** Configuration immutable class. */ | ||
public final class SharedBufferCacheConfig implements Serializable { | ||
private final int eventsBufferCacheSlots; | ||
private final int entryCacheSlots; | ||
private final Duration cacheStatisticsInterval; | ||
|
||
public int getEventsBufferCacheSlots() { | ||
return eventsBufferCacheSlots; | ||
} | ||
|
||
public int getEntryCacheSlots() { | ||
return entryCacheSlots; | ||
} | ||
|
||
public Duration getCacheStatisticsInterval() { | ||
return cacheStatisticsInterval; | ||
} | ||
|
||
public SharedBufferCacheConfig() { | ||
this.cacheStatisticsInterval = CEPCacheOptions.CEP_CACHE_STATISTICS_INTERVAL.defaultValue(); | ||
this.entryCacheSlots = CEPCacheOptions.CEP_SHARED_BUFFER_ENTRY_CACHE_SLOTS.defaultValue(); | ||
this.eventsBufferCacheSlots = | ||
CEPCacheOptions.CEP_SHARED_BUFFER_EVENT_CACHE_SLOTS.defaultValue(); | ||
} | ||
|
||
public SharedBufferCacheConfig( | ||
final int eventsBufferCacheSlots, | ||
final int entryCacheSlots, | ||
final Duration cacheStatisticsInterval) { | ||
this.cacheStatisticsInterval = cacheStatisticsInterval; | ||
this.entryCacheSlots = entryCacheSlots; | ||
this.eventsBufferCacheSlots = eventsBufferCacheSlots; | ||
} | ||
|
||
public static SharedBufferCacheConfig of(ReadableConfig readableConfig) { | ||
int eventsBufferCacheSlots = | ||
readableConfig.get(CEPCacheOptions.CEP_SHARED_BUFFER_EVENT_CACHE_SLOTS); | ||
int entryCacheSlots = | ||
readableConfig.get(CEPCacheOptions.CEP_SHARED_BUFFER_ENTRY_CACHE_SLOTS); | ||
Duration cacheStatisticsInterval = | ||
readableConfig.get(CEPCacheOptions.CEP_CACHE_STATISTICS_INTERVAL); | ||
return new SharedBufferCacheConfig( | ||
eventsBufferCacheSlots, entryCacheSlots, cacheStatisticsInterval); | ||
} | ||
} |
Oops, something went wrong.