Skip to content

Commit

Permalink
Fix remaining dtest failures caused by CASSANDRA-8143
Browse files Browse the repository at this point in the history
patch by Branimir Lambov; reviewed by Aleksey Yeschenko for
CASSANDRA-8143
  • Loading branch information
blambov authored and iamaleksey committed Aug 4, 2015
1 parent be0eebd commit 64e93a7
Show file tree
Hide file tree
Showing 6 changed files with 19 additions and 28 deletions.
6 changes: 2 additions & 4 deletions src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Expand Up @@ -276,14 +276,12 @@ public void setCompressionParameters(Map<String,String> opts)
}
}

// FIXME: this is wrong, JMX should never update live CFMetaData objects
public void setCrcCheckChance(double crcCheckChance)
{
try
{
// TODO: this doesn't affect sstables being written
for (SSTableReader sstable : keyspace.getAllSSTables(SSTableSet.CANONICAL))
if (sstable.compression)
sstable.getCompressionMetadata().parameters.setCrcCheckChance(crcCheckChance);
metadata.params.compression.setCrcCheckChance(crcCheckChance);
}
catch (ConfigurationException e)
{
Expand Down
14 changes: 2 additions & 12 deletions src/java/org/apache/cassandra/db/lifecycle/Helpers.java
Expand Up @@ -71,10 +71,10 @@ static <T> Map<T, T> replace(Map<T, T> original, Set<T> remove, Iterable<T> add)
* A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
* @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
*/
static void setupKeyCache(Iterable<SSTableReader> readers)
static void setupOnline(Iterable<SSTableReader> readers)
{
for (SSTableReader reader : readers)
reader.setupKeyCache();
reader.setupOnline();
}

/**
Expand All @@ -97,16 +97,6 @@ static Throwable setReplaced(Iterable<SSTableReader> readers, Throwable accumula
return accumulate;
}

/**
* A convenience method for encapsulating this action over multiple SSTableReader with exception-safety
* @return accumulate if not null (with any thrown exception attached), or any thrown exception otherwise
*/
static void setupKeycache(Iterable<SSTableReader> readers)
{
for (SSTableReader reader : readers)
reader.setupKeyCache();
}

/**
* assert that none of these readers have been replaced
*/
Expand Down
Expand Up @@ -350,7 +350,7 @@ public void update(SSTableReader reader, boolean original)
staged.update.add(reader);
identities.add(reader.instanceId);
if (!isOffline())
reader.setupKeyCache();
reader.setupOnline();
}

/**
Expand Down
4 changes: 2 additions & 2 deletions src/java/org/apache/cassandra/db/lifecycle/Tracker.java
Expand Up @@ -177,7 +177,7 @@ Throwable updateSizeTracking(Iterable<SSTableReader> oldSSTables, Iterable<SSTab
public void addInitialSSTables(Iterable<SSTableReader> sstables)
{
if (!isDummy())
setupKeycache(sstables);
setupOnline(sstables);
apply(updateLiveSet(emptySet(), sstables));
maybeFail(updateSizeTracking(emptySet(), sstables, null));
// no notifications or backup necessary
Expand Down Expand Up @@ -341,7 +341,7 @@ public void replaceFlushed(Memtable memtable, SSTableReader sstable)
return;
}

sstable.setupKeyCache();
sstable.setupOnline();
// back up before creating a new Snapshot (which makes the new one eligible for compaction)
maybeIncrementallyBackup(sstable);

Expand Down
Expand Up @@ -222,7 +222,7 @@ private void moveStarts(SSTableReader newReader, DecoratedKey lowerbound)

final List<DecoratedKey> invalidateKeys = new ArrayList<>();
invalidateKeys.addAll(cachedKeys.keySet());
newReader.setupKeyCache();
newReader.setupOnline();
for (Map.Entry<DecoratedKey, RowIndexEntry> cacheKey : cachedKeys.entrySet())
newReader.cacheKey(cacheKey.getKey(), cacheKey.getValue());

Expand Down
19 changes: 11 additions & 8 deletions src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
Expand Up @@ -635,12 +635,19 @@ public String getFilename()
return dfile.path();
}

public void setupKeyCache()
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.
keyCache = CacheService.instance.keyCache;

// ensure secondary index compression metadata is linked to the parent metadata.
if (compression && metadata.isIndex())
{
getCompressionMetadata().parameters.setLiveMetadata(
Schema.instance.getCFMetaData(metadata.ksName, metadata.getParentColumnFamilyName()));
}
}

public boolean isKeyCacheSetup()
Expand Down Expand Up @@ -1287,13 +1294,7 @@ public CompressionMetadata getCompressionMetadata()
if (!compression)
throw new IllegalStateException(this + " is not compressed");

CompressionMetadata cmd = ((ICompressedFile) dfile).getMetadata();

// We need the parent cf metadata
String cfName = metadata.isIndex() ? metadata.getParentColumnFamilyName() : metadata.cfName;
cmd.parameters.setLiveMetadata(Schema.instance.getCFMetaData(metadata.ksName, cfName));

return cmd;
return ((ICompressedFile) dfile).getMetadata();
}

/**
Expand Down Expand Up @@ -2050,6 +2051,8 @@ void setup(boolean trackHotness)
{
tidy.setup(this, trackHotness);
this.readMeter = tidy.global.readMeter;
if (compression)
getCompressionMetadata().parameters.setLiveMetadata(metadata);
}

@VisibleForTesting
Expand Down

0 comments on commit 64e93a7

Please sign in to comment.