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 2009/05/09 03:40:17 UTC

svn commit: r773152 - in /incubator/cassandra/trunk/src/java/org/apache/cassandra: db/Memtable.java db/Table.java service/RangeVerbHandler.java

Author: jbellis
Date: Sat May  9 01:40:16 2009
New Revision: 773152

URL: http://svn.apache.org/viewvc?rev=773152&view=rev
Log:
fix CME during range queries -- keyset call needs to go on the memtable exector.
patch by jbellis; tested by Tv for CASSANDRA-153

Modified:
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
    incubator/cassandra/trunk/src/java/org/apache/cassandra/service/RangeVerbHandler.java

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java?rev=773152&r1=773151&r2=773152&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Memtable.java Sat May  9 01:40:16 2009
@@ -399,9 +399,16 @@
         }
     }
 
-    public Iterator<String> sortedKeyIterator()
+    public Iterator<String> sortedKeyIterator() throws ExecutionException, InterruptedException
     {
-        Set<String> keys = columnFamilies_.keySet();
+        Callable<Set<String>> callable = new Callable<Set<String>>()
+        {
+            public Set<String> call() throws Exception
+            {
+                return columnFamilies_.keySet();
+            }
+        };
+        Set<String> keys = executor_.submit(callable).get();
         if (keys.size() == 0)
         {
             // cannot create a PQ of size zero (wtf?)

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java?rev=773152&r1=773151&r2=773152&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/db/Table.java Sat May  9 01:40:16 2009
@@ -26,6 +26,7 @@
 import java.io.FileNotFoundException;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.commons.collections.IteratorUtils;
 import org.apache.commons.collections.Predicate;
@@ -870,7 +871,7 @@
      * @param maxResults
      * @return list of keys between startWith and stopAt
      */
-    public List<String> getKeyRange(final String startWith, final String stopAt, int maxResults) throws IOException
+    public List<String> getKeyRange(final String startWith, final String stopAt, int maxResults) throws IOException, ExecutionException, InterruptedException
     {
         // (OPP key decoration is a no-op so using the "decorated" comparator against raw keys is fine)
         final Comparator<String> comparator = StorageService.getPartitioner().getDecoratedKeyComparator();

Modified: incubator/cassandra/trunk/src/java/org/apache/cassandra/service/RangeVerbHandler.java
URL: http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/RangeVerbHandler.java?rev=773152&r1=773151&r2=773152&view=diff
==============================================================================
--- incubator/cassandra/trunk/src/java/org/apache/cassandra/service/RangeVerbHandler.java (original)
+++ incubator/cassandra/trunk/src/java/org/apache/cassandra/service/RangeVerbHandler.java Sat May  9 01:40:16 2009
@@ -22,7 +22,7 @@
             Table table = Table.open(command.table);
             keys = table.getKeyRange(command.startWith, command.stopAt, command.maxResults);
         }
-        catch (IOException e)
+        catch (Exception e)
         {
             throw new RuntimeException(e);
         }