cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jbel...@apache.org
Subject svn commit: r764004 - in /incubator/cassandra/trunk: src/org/apache/cassandra/db/ColumnFamilyStore.java test/org/apache/cassandra/db/ColumnFamilyStoreTest.java
Date Fri, 10 Apr 2009 19:08:27 GMT
Author: jbellis
Date: Fri Apr 10 19:08:27 2009
New Revision: 764004

URL: http://svn.apache.org/viewvc?rev=764004&view=rev
Log:
fix getCompactionBuckets -- something is broken in NonBlockingHashMap when removing and re-adding
the same collection mid-iteration; it ends up with multiple references to that collection.
 going back to ConcurrentHashMap fixes the problem.  (any additional overhead from CHM is
negligible here.)  patch by jbellis; reviewed by Eric Evans for #57

Modified:
    incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyStore.java
    incubator/cassandra/trunk/test/org/apache/cassandra/db/ColumnFamilyStoreTest.java

Modified: incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyStore.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyStore.java?rev=764004&r1=764003&r2=764004&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyStore.java (original)
+++ incubator/cassandra/trunk/src/org/apache/cassandra/db/ColumnFamilyStore.java Fri Apr 10
19:08:27 2009
@@ -30,6 +30,7 @@
 import java.util.StringTokenizer;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.Future;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -50,7 +51,6 @@
 import org.apache.cassandra.utils.BloomFilter;
 import org.apache.cassandra.utils.FileUtils;
 import org.apache.cassandra.utils.LogUtil;
-import org.cliffc.high_scale_lib.NonBlockingHashMap;
 
 /**
  * Author : Avinash Lakshman ( alakshman@facebook.com) & Prashant Malik ( pmalik@facebook.com
)
@@ -694,7 +694,7 @@
      */
     static Set<List<String>> getCompactionBuckets(List<String> files, long
min)
     {
-    	Map<List<String>, Long> buckets = new NonBlockingHashMap<List<String>,
Long>();
+    	Map<List<String>, Long> buckets = new ConcurrentHashMap<List<String>,
Long>();
     	for(String fname : files)
     	{
     		File f = new File(fname);

Modified: incubator/cassandra/trunk/test/org/apache/cassandra/db/ColumnFamilyStoreTest.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/org/apache/cassandra/db/ColumnFamilyStoreTest.java?rev=764004&r1=764003&r2=764004&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/org/apache/cassandra/db/ColumnFamilyStoreTest.java (original)
+++ incubator/cassandra/trunk/test/org/apache/cassandra/db/ColumnFamilyStoreTest.java Fri
Apr 10 19:08:27 2009
@@ -15,6 +15,7 @@
 import java.util.concurrent.Future;
 
 import org.apache.commons.lang.ArrayUtils;
+import org.apache.commons.lang.StringUtils;
 
 import org.apache.cassandra.ServerTest;
 import org.testng.annotations.Test;
@@ -285,8 +286,23 @@
         }
 
         Set<List<String>> buckets = ColumnFamilyStore.getCompactionBuckets(all,
50);
-        assert buckets.contains(small);
-        assert buckets.contains(med);
+        assert buckets.size() == 2 : bucketString(buckets);
+        Iterator<List<String>> iter = buckets.iterator();
+        List<String> bucket1 = iter.next();
+        List<String> bucket2 = iter.next();
+        assert bucket1.size() + bucket2.size() == all.size() : bucketString(buckets) + "
does not match [" + StringUtils.join(all, ", ") + "]";
+        assert buckets.contains(small) : bucketString(buckets) + " does not contain {" +
StringUtils.join(small, ", ") + "}";
+        assert buckets.contains(med) : bucketString(buckets) + " does not contain {" + StringUtils.join(med,
", ") + "}";
+    }
+
+    private static String bucketString(Set<List<String>> buckets)
+    {
+        ArrayList<String> pieces = new ArrayList<String>();
+        for (List<String> bucket : buckets)
+        {
+            pieces.add("[" + StringUtils.join(bucket, ", ") + "]");
+        }
+        return "{" + StringUtils.join(pieces, ", ") + "}";
     }
 
     private String createFile(int nBytes) throws IOException



Mime
View raw message