Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
  • Loading branch information
smiklosovic committed Aug 28, 2024
1 parent 6da9141 commit 6a20dd3
Show file tree
Hide file tree
Showing 6 changed files with 35 additions and 31 deletions.
6 changes: 3 additions & 3 deletions src/java/org/apache/cassandra/config/Config.java
Original file line number Diff line number Diff line change
Expand Up @@ -305,16 +305,16 @@ public class Config
public volatile int sstable_preemptive_open_interval_in_mb = 50;

public volatile boolean key_cache_migrate_during_compaction = true;
public Long key_cache_size_in_mb = null;
public volatile Long key_cache_size_in_mb = null;
public volatile int key_cache_save_period = 14400;
public volatile int key_cache_keys_to_save = Integer.MAX_VALUE;

public String row_cache_class_name = "org.apache.cassandra.cache.OHCProvider";
public long row_cache_size_in_mb = 0;
public volatile long row_cache_size_in_mb = 0;
public volatile int row_cache_save_period = 0;
public volatile int row_cache_keys_to_save = Integer.MAX_VALUE;

public Long counter_cache_size_in_mb = null;
public volatile Long counter_cache_size_in_mb = null;
public volatile int counter_cache_save_period = 7200;
public volatile int counter_cache_keys_to_save = Integer.MAX_VALUE;

Expand Down
35 changes: 21 additions & 14 deletions src/java/org/apache/cassandra/config/DatabaseDescriptor.java
Original file line number Diff line number Diff line change
Expand Up @@ -128,8 +128,6 @@ public class DatabaseDescriptor

private static long preparedStatementsCacheSizeInMB;

private static long keyCacheSizeInMB;
private static long counterCacheSizeInMB;
private static long indexSummaryCapacityInMB;

private static String localDC;
Expand Down Expand Up @@ -712,9 +710,9 @@ else if (conf.repair_session_space_in_mb > (int) (Runtime.getRuntime().maxMemory
try
{
// if key_cache_size_in_mb option was set to "auto" then size of the cache should be "min(5% of Heap (in MB), 100MB)
keyCacheSizeInMB = (conf.key_cache_size_in_mb == null)
? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024)), 100)
: conf.key_cache_size_in_mb;
long keyCacheSizeInMB = (conf.key_cache_size_in_mb == null)
? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024)), 100)
: conf.key_cache_size_in_mb;

if (keyCacheSizeInMB < 0)
throw new NumberFormatException(); // to escape duplicating error message
Expand All @@ -731,22 +729,22 @@ else if (conf.repair_session_space_in_mb > (int) (Runtime.getRuntime().maxMemory
try
{
// if counter_cache_size_in_mb option was set to "auto" then size of the cache should be "min(2.5% of Heap (in MB), 50MB)
counterCacheSizeInMB = (conf.counter_cache_size_in_mb == null)
? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.025 / 1024 / 1024)), 50)
: conf.counter_cache_size_in_mb;
long counterCacheSizeInMB = (conf.counter_cache_size_in_mb == null)
? Math.min(Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.025 / 1024 / 1024)), 50)
: conf.counter_cache_size_in_mb;

if (counterCacheSizeInMB < 0)
throw new NumberFormatException(); // to escape duplicating error message

// we need this assignment for the Settings virtual table - CASSANDRA-17735
conf.counter_cache_size_in_mb = counterCacheSizeInMB;
}
catch (NumberFormatException e)
{
throw new ConfigurationException("counter_cache_size_in_mb option was set incorrectly to '"
+ conf.counter_cache_size_in_mb + "', supported values are <integer> >= 0.", false);
}

// we need this assignment for the Settings virtual table - CASSANDRA-17735
conf.counter_cache_size_in_mb = counterCacheSizeInMB;

// if set to empty/"auto" then use 5% of Heap size
indexSummaryCapacityInMB = (conf.index_summary_capacity_in_mb == null)
? Math.max(1, (int) (Runtime.getRuntime().totalMemory() * 0.05 / 1024 / 1024))
Expand Down Expand Up @@ -2702,7 +2700,12 @@ public static int getTrickleFsyncIntervalInKb()

public static long getKeyCacheSizeInMB()
{
return keyCacheSizeInMB;
return conf.key_cache_size_in_mb;
}

public static void setKeyCacheSizeInMB(long size)
{
conf.key_cache_size_in_mb = size;
}

public static long getIndexSummaryCapacityInMB()
Expand Down Expand Up @@ -2740,7 +2743,6 @@ public static long getRowCacheSizeInMB()
return conf.row_cache_size_in_mb;
}

@VisibleForTesting
public static void setRowCacheSizeInMB(long val)
{
conf.row_cache_size_in_mb = val;
Expand All @@ -2763,7 +2765,12 @@ public static int getRowCacheKeysToSave()

public static long getCounterCacheSizeInMB()
{
return counterCacheSizeInMB;
return conf.counter_cache_size_in_mb;
}

public static void setCounterCacheSizeInMB(long val)
{
conf.counter_cache_size_in_mb = val;
}

public static void setRowCacheKeysToSave(int rowCacheKeysToSave)
Expand Down
6 changes: 3 additions & 3 deletions src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Original file line number Diff line number Diff line change
Expand Up @@ -2697,19 +2697,19 @@ public boolean isEmpty()
public boolean isRowCacheEnabled()
{

boolean retval = metadata().params.caching.cacheRows() && CacheService.instance.rowCache.getCapacity() > 0;
boolean retval = metadata().params.caching.cacheRows() && DatabaseDescriptor.getRowCacheSizeInMB() > 0;
assert(!retval || !isIndex());
return retval;
}

public boolean isCounterCacheEnabled()
{
return metadata().isCounter() && CacheService.instance.counterCache.getCapacity() > 0;
return metadata().isCounter() && DatabaseDescriptor.getCounterCacheSizeInMB() > 0;
}

public boolean isKeyCacheEnabled()
{
return metadata().params.caching.cacheKeys() && CacheService.instance.keyCache.getCapacity() > 0;
return metadata().params.caching.cacheKeys() && DatabaseDescriptor.getKeyCacheSizeInMB() > 0;
}

/**
Expand Down
3 changes: 1 addition & 2 deletions src/java/org/apache/cassandra/db/CounterMutation.java
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,6 @@
import org.apache.cassandra.io.util.DataOutputPlus;
import org.apache.cassandra.locator.AbstractReplicationStrategy;
import org.apache.cassandra.schema.TableId;
import org.apache.cassandra.service.CacheService;
import org.apache.cassandra.tracing.Tracing;
import org.apache.cassandra.utils.*;
import org.apache.cassandra.utils.btree.BTreeSet;
Expand Down Expand Up @@ -203,7 +202,7 @@ private PartitionUpdate processModifications(PartitionUpdate changes)

List<PartitionUpdate.CounterMark> marks = changes.collectCounterMarks();

if (CacheService.instance.counterCache.getCapacity() != 0)
if (DatabaseDescriptor.getCounterCacheSizeInMB() > 0)
{
Tracing.trace("Fetching {} counter values from cache", marks.size());
updateWithCurrentValuesFromCache(marks, cfs);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -510,7 +510,7 @@ public static SSTableReader open(Descriptor descriptor,
if (validate)
sstable.validate();

if (sstable.getKeyCache() != null)
if (sstable.getKeyCache() != null && logger.isTraceEnabled())
logger.trace("key cache contains {}/{} keys", sstable.getKeyCache().size(), sstable.getKeyCache().getCapacity());

return sstable;
Expand Down Expand Up @@ -701,9 +701,8 @@ public void setupOnline()
// under normal operation we can do this at any time, but SSTR is also used outside C* proper,
// e.g. by BulkLoader, which does not initialize the cache. As a kludge, we set up the cache
// here when we know we're being wired into the rest of the server infrastructure.
InstrumentingCache<KeyCacheKey, RowIndexEntry> maybeKeyCache = CacheService.instance.keyCache;
if (maybeKeyCache.getCapacity() > 0)
keyCache = maybeKeyCache;
if (DatabaseDescriptor.getKeyCacheSizeInMB() > 0)
keyCache = CacheService.instance.keyCache;

final ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(metadata().id);
if (cfs != null)
Expand Down Expand Up @@ -1318,7 +1317,7 @@ public void cacheKey(DecoratedKey key, RowIndexEntry info)
{
CachingParams caching = metadata().params.caching;

if (!caching.cacheKeys() || keyCache == null || keyCache.getCapacity() == 0)
if (!caching.cacheKeys() || keyCache == null || DatabaseDescriptor.getKeyCacheSizeInMB() == 0)
return;

KeyCacheKey cacheKey = new KeyCacheKey(metadata(), descriptor, key.getKey());
Expand Down
7 changes: 3 additions & 4 deletions src/java/org/apache/cassandra/service/CacheService.java
Original file line number Diff line number Diff line change
Expand Up @@ -301,24 +301,22 @@ public void invalidateCounterCache()
counterCache.clear();
}




public void setRowCacheCapacityInMB(long capacity)
{
if (capacity < 0)
throw new RuntimeException("capacity should not be negative.");

rowCache.setCapacity(capacity * 1024 * 1024);
DatabaseDescriptor.setRowCacheSizeInMB(capacity);
}


public void setKeyCacheCapacityInMB(long capacity)
{
if (capacity < 0)
throw new RuntimeException("capacity should not be negative.");

keyCache.setCapacity(capacity * 1024 * 1024);
DatabaseDescriptor.setKeyCacheSizeInMB(capacity);
}

public void setCounterCacheCapacityInMB(long capacity)
Expand All @@ -327,6 +325,7 @@ public void setCounterCacheCapacityInMB(long capacity)
throw new RuntimeException("capacity should not be negative.");

counterCache.setCapacity(capacity * 1024 * 1024);
DatabaseDescriptor.setCounterCacheSizeInMB(capacity);
}

public void saveCaches() throws ExecutionException, InterruptedException
Expand Down

0 comments on commit 6a20dd3

Please sign in to comment.