You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by jb...@apache.org on 2011/10/19 15:31:26 UTC
svn commit: r1186207 - in
/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra: db/
db/compaction/ db/index/ io/sstable/ streaming/
Author: jbellis
Date: Wed Oct 19 13:31:25 2011
New Revision: 1186207
URL: http://svn.apache.org/viewvc?rev=1186207&view=rev
Log:
cleanup for #3314
Modified:
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamInSession.java
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=1186207&r1=1186206&r2=1186207&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/ColumnFamilyStore.java Wed Oct 19 13:31:25 2011
@@ -886,11 +886,6 @@ public class ColumnFamilyStore implement
return false;
}
- public boolean isKeyExistenceExpensive(Set<? extends SSTable> sstablesToIgnore)
- {
- return compactionStrategy.isKeyExistenceExpensive(sstablesToIgnore);
- }
-
/*
* Called after a BinaryMemtable flushes its in-memory data, or we add a file
* via bootstrap. This information is cached in the ColumnFamilyStore.
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionManager.java?rev=1186207&r1=1186206&r2=1186207&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionManager.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/compaction/CompactionManager.java Wed Oct 19 13:31:25 2011
@@ -772,7 +772,7 @@ public class CompactionManager implement
}
// flush to ensure we don't lose the tombstones on a restart, since they are not commitlog'd
- cfs.indexManager.flushIndexes();
+ cfs.indexManager.flushIndexesBlocking();
cfs.replaceCompactedSSTables(Arrays.asList(sstable), results);
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java?rev=1186207&r1=1186206&r2=1186207&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexBuilder.java Wed Oct 19 13:31:25 2011
@@ -29,8 +29,7 @@ import org.apache.cassandra.db.compactio
import org.apache.cassandra.io.sstable.ReducingKeyIterator;
/**
- * Manages building an entire index from column family data
- * Sent to compaction manager
+ * Manages building an entire index from column family data. Runs on to compaction manager.
*/
public class SecondaryIndexBuilder implements CompactionInfo.Holder
{
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java?rev=1186207&r1=1186206&r2=1186207&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/db/index/SecondaryIndexManager.java Wed Oct 19 13:31:25 2011
@@ -97,8 +97,11 @@ public class SecondaryIndexManager
/**
- * Does a full rebuild of the indexes specified by columns from the sstables.
+ * Does a full, blocking rebuild of the indexes specified by columns from the sstables.
* Does nothing if columns is empty.
+ *
+ * Caller must acquire and release references to the sstables used here.
+ *
* @param sstables the data to build from
* @param columns the list of columns to index
* @throws IOException
@@ -116,7 +119,7 @@ public class SecondaryIndexManager
try
{
future.get();
- flushIndexes();
+ flushIndexesBlocking();
}
catch (InterruptedException e)
{
@@ -270,7 +273,7 @@ public class SecondaryIndexManager
* @throws ExecutionException
* @throws InterruptedException
*/
- public void flushIndexes() throws IOException
+ public void flushIndexesBlocking() throws IOException
{
for (Map.Entry<ByteBuffer, SecondaryIndex> entry : indexesByColumn.entrySet())
entry.getValue().forceBlockingFlush();
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java?rev=1186207&r1=1186206&r2=1186207&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/io/sstable/ReducingKeyIterator.java Wed Oct 19 13:31:25 2011
@@ -30,6 +30,9 @@ import org.apache.cassandra.utils.Closea
import org.apache.cassandra.utils.IMergeIterator;
import org.apache.cassandra.utils.MergeIterator;
+/**
+ * Caller must acquire and release references to the sstables used here.
+ */
public class ReducingKeyIterator implements CloseableIterator<DecoratedKey>
{
private final IMergeIterator<DecoratedKey,DecoratedKey> mi;
Modified: cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamInSession.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamInSession.java?rev=1186207&r1=1186206&r2=1186207&view=diff
==============================================================================
--- cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamInSession.java (original)
+++ cassandra/branches/cassandra-1.0/src/java/org/apache/cassandra/streaming/StreamInSession.java Wed Oct 19 13:31:25 2011
@@ -126,18 +126,16 @@ public class StreamInSession
if (files.isEmpty())
{
HashMap <ColumnFamilyStore, List<SSTableReader>> cfstores = new HashMap<ColumnFamilyStore, List<SSTableReader>>();
- List<SSTableReader> referenced = new LinkedList<SSTableReader>();
try
{
for (SSTableReader sstable : readers)
{
assert sstable.getTableName().equals(table);
- // Acquiring the reference (for secondary index building) before adding it makes sure we don't have to care about races
+ // Acquire the reference (for secondary index building) before submitting the index build,
+ // so it can't get compacted out of existence in between
if (!sstable.acquireReference())
- throw new RuntimeException("We shouldn't fail acquiring a reference on a sstable that has just been transfered");
-
- referenced.add(sstable);
+ throw new AssertionError("We shouldn't fail acquiring a reference on a sstable that has just been transfered");
ColumnFamilyStore cfs = Table.open(sstable.getTableName()).getColumnFamilyStore(sstable.getColumnFamilyName());
cfs.addSSTable(sstable);
@@ -155,7 +153,8 @@ public class StreamInSession
}
finally
{
- SSTableReader.releaseReferences(referenced);
+ for (List<SSTableReader> referenced : cfstores.values())
+ SSTableReader.releaseReferences(referenced);
}
// send reply to source that we're done