Author: jbellis
Date: Tue Jan 26 23:21:05 2010
New Revision: 903486
URL: http://svn.apache.org/viewvc?rev=903486&view=rev
Log:
add SSTableTracker.replace & refactor. patch by jbellis
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableTracker.java
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java?rev=903486&r1=903485&r2=903486&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
Tue Jan 26 23:21:05 2010
@@ -616,11 +616,7 @@
void replaceCompactedSSTables(Collection<SSTableReader> sstables,
Iterable<SSTableReader> replacements)
throws IOException
{
- for (SSTableReader sstable : replacements)
- {
- ssTables_.add(sstable);
- }
- ssTables_.markCompacted(sstables);
+ ssTables_.replace(sstables, replacements);
}
public static List<Memtable> getUnflushedMemtables(String cfName)
Modified:
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableTracker.java
URL:
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableTracker.java?rev=903486&r1=903485&r2=903486&view=diff
==============================================================================
---
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableTracker.java
(original)
+++
incubator/cassandra/trunk/src/java/org/apache/cassandra/io/SSTableTracker.java
Tue Jan 26 23:21:05 2010
@@ -34,27 +34,43 @@
this.sstables = Collections.unmodifiableSet(new
HashSet<SSTableReader>(sstables));
}
- public synchronized void add(SSTableReader sstable)
+ public synchronized void replace(Collection<SSTableReader> oldSSTables,
Iterable<SSTableReader> replacements) throws IOException
{
- assert sstable != null;
- assert sstable.getIndexPositions() != null;
Set<SSTableReader> sstablesNew = new HashSet<SSTableReader>(sstables);
- sstablesNew.add(sstable);
- sstables = Collections.unmodifiableSet(sstablesNew);
- }
- // todo replace w/ compactionfinished for CASSANDRA-431
- public synchronized void markCompacted(Iterable<SSTableReader> compacted)
throws IOException
- {
- Set<SSTableReader> sstablesNew = new HashSet<SSTableReader>(sstables);
- for (SSTableReader sstable : compacted)
+ for (SSTableReader sstable : replacements)
+ {
+ assert sstable.getIndexPositions() != null;
+ sstablesNew.add(sstable);
+ }
+
+ for (SSTableReader sstable : oldSSTables)
{
sstablesNew.remove(sstable);
sstable.markCompacted();
}
+
sstables = Collections.unmodifiableSet(sstablesNew);
}
+ public synchronized void add(SSTableReader sstable)
+ {
+ assert sstable != null;
+ try
+ {
+ replace(Collections.<SSTableReader>emptyList(),
Arrays.asList(sstable));
+ }
+ catch (IOException e)
+ {
+ throw new AssertionError(e);
+ }
+ }
+
+ public synchronized void markCompacted(Collection<SSTableReader>
compacted) throws IOException
+ {
+ replace(compacted, Collections.<SSTableReader>emptyList());
+ }
+
// the modifiers create new, unmodifiable objects each time; the volatile
fences the assignment
// so we don't need any further synchronization for the common case here
public Set<SSTableReader> getSSTables()