Skip to content

Commit

Permalink
skip reading saved key cache if CF's caching strategy is NONE or ROWS…
Browse files Browse the repository at this point in the history
…_ONLY

patch by Pavel Yaskevich; reviewed by Jonathan Ellis for CASSANDRA-3954
  • Loading branch information
xedin committed Mar 16, 2012
1 parent c290372 commit 6423bfe
Show file tree
Hide file tree
Showing 3 changed files with 22 additions and 5 deletions.
1 change: 1 addition & 0 deletions CHANGES.txt
Expand Up @@ -22,6 +22,7 @@
* Add type information to new schema_ columnfamilies and remove thrift
serialization for schema (CASSANDRA-3792)
* add missing column validator options to the CLI help (CASSANDRA-3926)
* skip reading saved key cache if CF's caching strategy is NONE or ROWS_ONLY (CASSANDRA-3954)
Merged from 1.0:
* always compact away deleted hints immediately after handoff (CASSANDRA-3955)
* delete hints from dropped ColumnFamilies on handoff instead of
Expand Down
1 change: 1 addition & 0 deletions src/java/org/apache/cassandra/config/CFMetaData.java
Expand Up @@ -324,6 +324,7 @@ public static CFMetaData newIndexMetadata(CFMetaData parent, ColumnDefinition in
.keyValidator(info.getValidator())
.readRepairChance(0.0)
.dcLocalReadRepairChance(0.0)
.caching(Caching.NONE)
.reloadSecondaryIndexMetadata(parent);
}

Expand Down
25 changes: 20 additions & 5 deletions src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Expand Up @@ -71,6 +71,8 @@
import org.apache.cassandra.utils.IntervalTree.Interval;
import org.cliffc.high_scale_lib.NonBlockingHashMap;

import static org.apache.cassandra.config.CFMetaData.Caching;

public class ColumnFamilyStore implements ColumnFamilyStoreMBean
{
private static Logger logger = LoggerFactory.getLogger(ColumnFamilyStore.class);
Expand Down Expand Up @@ -199,7 +201,12 @@ public void setCompressionParameters(Map<String,String> opts) throws Configurati
metadata.compressionParameters = CompressionParameters.create(opts);
}

private ColumnFamilyStore(Table table, String columnFamilyName, IPartitioner partitioner, int generation, CFMetaData metadata, Directories directories)
private ColumnFamilyStore(Table table,
String columnFamilyName,
IPartitioner partitioner,
int generation,
CFMetaData metadata,
Directories directories)
{
assert metadata != null : "null metadata for " + table + ":" + columnFamilyName;

Expand All @@ -213,12 +220,17 @@ private ColumnFamilyStore(Table table, String columnFamilyName, IPartitioner par
this.indexManager = new SecondaryIndexManager(this);
fileIndexGenerator.set(generation);

Caching caching = metadata.getCaching();

if (logger.isDebugEnabled())
logger.debug("Starting CFS {}", columnFamily);

// scan for sstables corresponding to this cf and load them
data = new DataTracker(this);
Set<DecoratedKey> savedKeys = CacheService.instance.keyCache.readSaved(table.name, columnFamily);
Set<DecoratedKey> savedKeys = caching == Caching.NONE || caching == Caching.ROWS_ONLY
? Collections.<DecoratedKey>emptySet()
: CacheService.instance.keyCache.readSaved(table.name, columnFamily);

Directories.SSTableLister sstables = directories.sstableLister().skipCompacted(true).skipTemporary(true);
data.addInitialSSTables(SSTableReader.batchOpen(sstables.list().entrySet(), savedKeys, data, metadata, this.partitioner));

Expand Down Expand Up @@ -298,7 +310,10 @@ public static ColumnFamilyStore createColumnFamilyStore(Table table, String colu
return createColumnFamilyStore(table, columnFamily, StorageService.getPartitioner(), Schema.instance.getCFMetaData(table.name, columnFamily));
}

public static synchronized ColumnFamilyStore createColumnFamilyStore(Table table, String columnFamily, IPartitioner partitioner, CFMetaData metadata)
public static synchronized ColumnFamilyStore createColumnFamilyStore(Table table,
String columnFamily,
IPartitioner partitioner,
CFMetaData metadata)
{
// get the max generation number, to prevent generation conflicts
Directories directories = Directories.create(table.name, columnFamily);
Expand Down Expand Up @@ -1940,8 +1955,8 @@ public boolean isEmpty()

private boolean isRowCacheEnabled()
{
return !(metadata.getCaching() == CFMetaData.Caching.NONE
|| metadata.getCaching() == CFMetaData.Caching.KEYS_ONLY
return !(metadata.getCaching() == Caching.NONE
|| metadata.getCaching() == Caching.KEYS_ONLY
|| CacheService.instance.rowCache.getCapacity() == 0);
}

Expand Down

0 comments on commit 6423bfe

Please sign in to comment.