cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r1128020 - /cassandra/branches/cassandra-0.8.0/src/java/org/apache/cassandra/db/DataTracker.java
Date Thu, 26 May 2011 18:26:56 GMT
Author: jbellis
Date: Thu May 26 18:26:55 2011
New Revision: 1128020

URL: http://svn.apache.org/viewvc?rev=1128020&view=rev
Log:
fix potential stack overflow during compaction (take 2)
patch by jbellis; reviewed by brandonwilliams for CASSANDRA-2626

Modified:
    cassandra/branches/cassandra-0.8.0/src/java/org/apache/cassandra/db/DataTracker.java

Modified: cassandra/branches/cassandra-0.8.0/src/java/org/apache/cassandra/db/DataTracker.java
URL: http://svn.apache.org/viewvc/cassandra/branches/cassandra-0.8.0/src/java/org/apache/cassandra/db/DataTracker.java?rev=1128020&r1=1128019&r2=1128020&view=diff
==============================================================================
--- cassandra/branches/cassandra-0.8.0/src/java/org/apache/cassandra/db/DataTracker.java (original)
+++ cassandra/branches/cassandra-0.8.0/src/java/org/apache/cassandra/db/DataTracker.java Thu
May 26 18:26:55 2011
@@ -22,20 +22,17 @@ package org.apache.cassandra.db;
 import java.io.File;
 import java.io.IOError;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.*;
-import java.util.concurrent.ExecutionException;
 import java.util.concurrent.atomic.AtomicLong;
 import java.util.concurrent.atomic.AtomicReference;
 
-import com.google.common.collect.Iterables;
-import org.apache.commons.collections.set.UnmodifiableSet;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cache.AutoSavingCache;
 import org.apache.cassandra.config.DatabaseDescriptor;
-
 import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTableReader;
 import org.apache.cassandra.utils.Pair;
@@ -451,18 +448,17 @@ public class DataTracker
         public final Set<SSTableReader> sstables;
         public final Set<SSTableReader> compacting;
 
-        public View(Memtable memtable, Set<Memtable> pendingFlush, Set<SSTableReader>
sstables, Set<SSTableReader> compacting)
+        View(Memtable memtable, Set<Memtable> pendingFlush, Set<SSTableReader>
sstables, Set<SSTableReader> compacting)
         {
             this.memtable = memtable;
-            this.memtablesPendingFlush = pendingFlush instanceof UnmodifiableSet ? pendingFlush
: Collections.unmodifiableSet(pendingFlush);
-            this.sstables = sstables instanceof UnmodifiableSet ? sstables : Collections.unmodifiableSet(sstables);
-            this.compacting = compacting instanceof UnmodifiableSet ? compacting : Collections.unmodifiableSet(compacting);
+            this.memtablesPendingFlush = pendingFlush;
+            this.sstables = sstables;
+            this.compacting = compacting;
         }
 
         public View switchMemtable(Memtable newMemtable)
         {
-            Set<Memtable> newPending = new HashSet<Memtable>(memtablesPendingFlush);
-            newPending.add(memtable);
+            Set<Memtable> newPending = ImmutableSet.<Memtable>builder().addAll(memtablesPendingFlush).add(newMemtable).build();
             return new View(newMemtable, newPending, sstables, compacting);
         }
 
@@ -473,32 +469,27 @@ public class DataTracker
 
         public View replaceFlushed(Memtable flushedMemtable, SSTableReader newSSTable)
         {
-            Set<Memtable> newPendings = new HashSet<Memtable>(memtablesPendingFlush);
-            Set<SSTableReader> newSSTables = new HashSet<SSTableReader>(sstables);
-            newPendings.remove(flushedMemtable);
-            newSSTables.add(newSSTable);
-            return new View(memtable, newPendings, newSSTables, compacting);
+            Set<Memtable> newPending = ImmutableSet.copyOf(Sets.difference(memtablesPendingFlush,
Collections.singleton(flushedMemtable)));
+            Set<SSTableReader> newSSTables = ImmutableSet.<SSTableReader>builder().addAll(sstables).add(newSSTable).build();
+            return new View(memtable, newPending, newSSTables, compacting);
         }
 
         public View replace(Collection<SSTableReader> oldSSTables, Iterable<SSTableReader>
replacements)
         {
-            Set<SSTableReader> sstablesNew = new HashSet<SSTableReader>(sstables);
-            Iterables.addAll(sstablesNew, replacements);
-            sstablesNew.removeAll(oldSSTables);
-            return new View(memtable, memtablesPendingFlush, sstablesNew, compacting);
+            Sets.SetView<SSTableReader> remaining = Sets.difference(sstables, ImmutableSet.copyOf(oldSSTables));
+            Set<SSTableReader> newSSTables = ImmutableSet.<SSTableReader>builder().addAll(remaining).addAll(replacements).build();
+            return new View(memtable, memtablesPendingFlush, newSSTables, compacting);
         }
 
         public View markCompacting(Collection<SSTableReader> tomark)
         {
-            Set<SSTableReader> compactingNew = new HashSet<SSTableReader>(compacting);
-            compactingNew.addAll(tomark);
+            Set<SSTableReader> compactingNew = ImmutableSet.<SSTableReader>builder().addAll(sstables).addAll(tomark).build();
             return new View(memtable, memtablesPendingFlush, sstables, compactingNew);
         }
 
         public View unmarkCompacting(Collection<SSTableReader> tounmark)
         {
-            Set<SSTableReader> compactingNew = new HashSet<SSTableReader>(compacting);
-            compactingNew.removeAll(tounmark);
+            Set<SSTableReader> compactingNew = ImmutableSet.copyOf(Sets.difference(compacting,
ImmutableSet.copyOf(tounmark)));
             return new View(memtable, memtablesPendingFlush, sstables, compactingNew);
         }
     }



Mime
View raw message