Skip to content

Commit

Permalink
optimize ManagedLedger cacheEvictionTask implement
Browse files Browse the repository at this point in the history
  • Loading branch information
aloyszhang committed Feb 28, 2022
1 parent 4532c15 commit a77e021
Show file tree
Hide file tree
Showing 10 changed files with 42 additions and 33 deletions.
8 changes: 5 additions & 3 deletions conf/broker.conf
Original file line number Diff line number Diff line change
Expand Up @@ -960,8 +960,8 @@ managedLedgerCacheCopyEntries=false
# Threshold to which bring down the cache level when eviction is triggered
managedLedgerCacheEvictionWatermark=0.9

# Configure the cache eviction frequency for the managed ledger cache (evictions/sec)
managedLedgerCacheEvictionFrequency=100.0
# Configure the cache eviction interval for the managed ledger cache
managedLedgerCacheEvictionIntervalMs=10

# All entries that have stayed in cache for more than the configured time, will be evicted
managedLedgerCacheEvictionTimeThresholdMillis=1000
Expand Down Expand Up @@ -1429,4 +1429,6 @@ tlsEnabled=false

# Enable Key_Shared subscription (default is enabled)
# @deprecated since 2.8.0 subscriptionTypesEnabled is preferred over subscriptionKeySharedEnable.
subscriptionKeySharedEnable=true
subscriptionKeySharedEnable=true
# Deprecated - Use managedLedgerCacheEvictionIntervalMs instead
managedLedgerCacheEvictionFrequency=100.0
7 changes: 5 additions & 2 deletions deployment/terraform-ansible/templates/broker.conf
Original file line number Diff line number Diff line change
Expand Up @@ -775,8 +775,8 @@ managedLedgerCacheCopyEntries=false
# Threshold to which bring down the cache level when eviction is triggered
managedLedgerCacheEvictionWatermark=0.9

# Configure the cache eviction frequency for the managed ledger cache (evictions/sec)
managedLedgerCacheEvictionFrequency=100.0
# Configure the cache eviction interval for the managed ledger cache
managedLedgerCacheEvictionIntervalMs=10

# All entries that have stayed in cache for more than the configured time, will be evicted
managedLedgerCacheEvictionTimeThresholdMillis=1000
Expand Down Expand Up @@ -1134,6 +1134,9 @@ replicationTlsEnabled=false
# Deprecated. Use brokerDeleteInactiveTopicsFrequencySeconds
brokerServicePurgeInactiveFrequencyInSeconds=60

# Deprecated - Use managedLedgerCacheEvictionIntervalMs instead
managedLedgerCacheEvictionFrequency=100.0

### --- Transaction config variables --- ###

# Enable transaction coordinator in broker
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -44,9 +44,16 @@ public class ManagedLedgerFactoryConfig {

/**
* Frequency of cache eviction triggering. Default is 100 times per second.
* @Deprecated Use {@link #cacheEvictionIntervalMs} instead.
*/
@Deprecated
private double cacheEvictionFrequency = 100;

/**
* Interval of cache eviction triggering. Default is 10 ms times.
*/
private long cacheEvictionIntervalMs = 10;

/**
* All entries that have stayed in cache for more than the configured time, will be evicted.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,8 +33,8 @@
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.TimeUnit;
import java.util.function.Supplier;
Expand Down Expand Up @@ -90,8 +90,7 @@ public class ManagedLedgerFactoryImpl implements ManagedLedgerFactory {
private final boolean isBookkeeperManaged;
private final ManagedLedgerFactoryConfig config;
protected final OrderedScheduler scheduledExecutor;

private final ExecutorService cacheEvictionExecutor;
private final ScheduledExecutorService cacheEvictionExecutor;

protected final ManagedLedgerFactoryMBeanImpl mbean;

Expand Down Expand Up @@ -179,7 +178,7 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
.name("bookkeeper-ml-scheduler")
.build();
cacheEvictionExecutor = Executors
.newSingleThreadExecutor(new DefaultThreadFactory("bookkeeper-ml-cache-eviction"));
.newSingleThreadScheduledExecutor(new DefaultThreadFactory("bookkeeper-ml-cache-eviction"));
this.metadataServiceAvailable = true;
this.bookkeeperFactory = bookKeeperGroupFactory;
this.isBookkeeperManaged = isBookkeeperManaged;
Expand All @@ -197,8 +196,9 @@ private ManagedLedgerFactoryImpl(MetadataStoreExtended metadataStore,
this.cacheEvictionTimeThresholdNanos = TimeUnit.MILLISECONDS
.toNanos(config.getCacheEvictionTimeThresholdMillis());


cacheEvictionExecutor.execute(this::cacheEvictionTask);
long evictionTaskInterval = Math.min(config.getCacheEvictionIntervalMs(), 10);
cacheEvictionExecutor.scheduleWithFixedDelay(cacheEvictionTask,
evictionTaskInterval, evictionTaskInterval, TimeUnit.MILLISECONDS);
closed = false;

metadataStore.registerSessionListener(this::handleMetadataStoreNotification);
Expand Down Expand Up @@ -252,23 +252,13 @@ private synchronized void refreshStats() {
lastStatTimestamp = now;
}

private void cacheEvictionTask() {
double evictionFrequency = Math.max(Math.min(config.getCacheEvictionFrequency(), 1000.0), 0.001);
long waitTimeMillis = (long) (1000 / evictionFrequency);

while (true) {
try {
doCacheEviction();

Thread.sleep(waitTimeMillis);
} catch (InterruptedException e) {
// Factory is shutting down
return;
} catch (Throwable t) {
log.warn("Exception while performing cache eviction: {}", t.getMessage(), t);
}
Runnable cacheEvictionTask = () -> {
try {
doCacheEviction();
} catch (Throwable t) {
log.warn("Exception while performing cache eviction: {}", t.getMessage(), t);
}
}
};

private synchronized void doCacheEviction() {
long maxTimestamp = System.nanoTime() - cacheEvictionTimeThresholdNanos;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -256,7 +256,7 @@ public void verifyHitsMisses() throws Exception {
ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig();
config.setMaxCacheSize(7 * 10);
config.setCacheEvictionWatermark(0.8);
config.setCacheEvictionFrequency(1);
config.setCacheEvictionIntervalMs(1000);

@Cleanup("shutdown")
ManagedLedgerFactoryImpl factory2 = new ManagedLedgerFactoryImpl(metadataStore, bkc, config);
Expand Down Expand Up @@ -325,7 +325,7 @@ public void verifyHitsMisses() throws Exception {
public void verifyTimeBasedEviction() throws Exception {
ManagedLedgerFactoryConfig config = new ManagedLedgerFactoryConfig();
config.setMaxCacheSize(1000);
config.setCacheEvictionFrequency(100);
config.setCacheEvictionIntervalMs(10);
config.setCacheEvictionTimeThresholdMillis(100);

@Cleanup("shutdown")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2327,7 +2327,7 @@ public void testGetNextValidPosition() throws Exception {
@Test
public void testActiveDeactiveCursorWithDiscardEntriesFromCache() throws Exception {
ManagedLedgerFactoryConfig conf = new ManagedLedgerFactoryConfig();
conf.setCacheEvictionFrequency(0.1);
conf.setCacheEvictionIntervalMs(10000);

@Cleanup("shutdown")
ManagedLedgerFactory factory = new ManagedLedgerFactoryImpl(metadataStore, bkc, conf);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -1685,7 +1685,14 @@ public class ServiceConfiguration implements PulsarConfiguration {
private double managedLedgerCacheEvictionWatermark = 0.9;
@FieldContext(category = CATEGORY_STORAGE_ML,
doc = "Configure the cache eviction frequency for the managed ledger cache. Default is 100/s")
@Deprecated
private double managedLedgerCacheEvictionFrequency = 100.0;

@FieldContext(category = CATEGORY_STORAGE_ML,
doc = "Configure the cache eviction interval for the managed ledger cache. Must >= 1 ms, "
+ " and default is 10 ms")
private long managedLedgerCacheEvictionIntervalMs = 10;

@FieldContext(category = CATEGORY_STORAGE_ML,
doc = "All entries that have stayed in cache for more than the configured time, will be evicted")
private long managedLedgerCacheEvictionTimeThresholdMillis = 1000;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,7 +59,7 @@ public void initialize(ServiceConfiguration conf, MetadataStoreExtended metadata
managedLedgerFactoryConfig.setMaxCacheSize(conf.getManagedLedgerCacheSizeMB() * 1024L * 1024L);
managedLedgerFactoryConfig.setCacheEvictionWatermark(conf.getManagedLedgerCacheEvictionWatermark());
managedLedgerFactoryConfig.setNumManagedLedgerSchedulerThreads(conf.getManagedLedgerNumSchedulerThreads());
managedLedgerFactoryConfig.setCacheEvictionFrequency(conf.getManagedLedgerCacheEvictionFrequency());
managedLedgerFactoryConfig.setCacheEvictionIntervalMs(conf.getManagedLedgerCacheEvictionIntervalMs());
managedLedgerFactoryConfig.setCacheEvictionTimeThresholdMillis(
conf.getManagedLedgerCacheEvictionTimeThresholdMillis());
managedLedgerFactoryConfig.setCopyEntriesInCache(conf.isManagedLedgerCacheCopyEntries());
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -53,7 +53,7 @@ public class ConsumerRedeliveryTest extends ProducerConsumerBase {
@BeforeClass
@Override
protected void setup() throws Exception {
conf.setManagedLedgerCacheEvictionFrequency(0.1);
conf.setManagedLedgerCacheEvictionIntervalMs(10000);
super.internalSetup();
super.producerBaseSetup();
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,7 @@ public Object[][] topicDomainProvider() {
@BeforeClass
@Override
protected void setup() throws Exception {
conf.setManagedLedgerCacheEvictionFrequency(0.1);
conf.setManagedLedgerCacheEvictionIntervalMs(10000);
super.internalSetup();
super.producerBaseSetup();
}
Expand Down

0 comments on commit a77e021

Please sign in to comment.