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=901207&r1=901206&r2=901207&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
 Wed Jan 20 13:57:49 2010
@@ -921,130 +921,12 @@
      * @param stopAt key to stop at, inclusive.  empty string = stop only when 
keys are exhausted.
      * @param maxResults
      * @return list of keys between startWith and stopAt
-     */
-    public RangeReply getKeyRange(final String startWith, final String stopAt, 
int maxResults)
-    throws IOException, ExecutionException, InterruptedException
-    {
-        final DecoratedKey startWithDK = partitioner.decorateKey(startWith);
-        final DecoratedKey stopAtDK = partitioner.decorateKey(stopAt);
-
-        // create a CollatedIterator that will return unique keys from 
different sources
-        // (current memtable, historical memtables, and SSTables) in the 
correct order.
-        List<Iterator<DecoratedKey>> iterators = new 
ArrayList<Iterator<DecoratedKey>>();
-
-        // we iterate through memtables with a priority queue to avoid more 
sorting than necessary.
-        // this predicate throws out the keys before the start of our range.
-        Predicate<DecoratedKey> p = new Predicate<DecoratedKey>()
-        {
-            public boolean apply(DecoratedKey key)
-            {
-                return startWithDK.compareTo(key) <= 0
-                       && (stopAt.isEmpty() || key.compareTo(stopAtDK) <= 0);
-            }
-        };
-
-        // current memtable keys.  have to go through the CFS api for locking.
-        iterators.add(Iterators.filter(memtableKeyIterator(), p));
-        // historical memtables
-        for (Memtable memtable : 
ColumnFamilyStore.getUnflushedMemtables(columnFamily_))
-        {
-            iterators.add(Iterators.filter(memtable.getKeyIterator(), p));
-        }
-
-        // sstables
-        for (SSTableReader sstable : ssTables_)
-        {
-            final SSTableScanner scanner = 
sstable.getScanner(KEY_RANGE_FILE_BUFFER_SIZE);
-            scanner.seekTo(startWithDK);
-            Iterator<DecoratedKey> iter = new CloseableIterator<DecoratedKey>()
-            {
-                public boolean hasNext()
-                {
-                    return scanner.hasNext();
-                }
-                public DecoratedKey next()
-                {
-                    return scanner.next().getKey();
-                }
-                public void remove()
-                {
-                    throw new UnsupportedOperationException();
-                }
-                public void close() throws IOException
-                {
-                    scanner.close();
-                }
-            };
-            assert iter instanceof Closeable; // otherwise we leak FDs
-            iterators.add(iter);
-        }
-
-        Iterator<DecoratedKey> collated = 
IteratorUtils.collatedIterator(DecoratedKey.comparator, iterators);
-        Iterable<DecoratedKey> reduced = new ReducingIterator<DecoratedKey, 
DecoratedKey>(collated) {
-            DecoratedKey current;
-
-            public void reduce(DecoratedKey current)
-            {
-                 this.current = current;
-            }
-
-            protected DecoratedKey getReduced()
-            {
-                return current;
-            }
-        };
-
-        try
-        {
-            // pull keys out of the CollatedIterator.  checking tombstone 
status is expensive,
-            // so we set an arbitrary limit on how many we'll do at once.
-            List<String> keys = new ArrayList<String>();
-            boolean rangeCompletedLocally = false;
-            for (DecoratedKey current : reduced)
-            {
-                if (!stopAt.isEmpty() && stopAtDK.compareTo(current) < 0)
-                {
-                    rangeCompletedLocally = true;
-                    break;
-                }
-                // make sure there is actually non-tombstone content 
associated w/ this key
-                // TODO record the key source(s) somehow and only check that 
source (e.g., memtable or sstable)
-                QueryFilter filter = new SliceQueryFilter(current.key, new 
QueryPath(columnFamily_), ArrayUtils.EMPTY_BYTE_ARRAY, 
ArrayUtils.EMPTY_BYTE_ARRAY, false, 1);
-                if (getColumnFamily(filter, Integer.MAX_VALUE) != null)
-                {
-                    keys.add(current.key);
-                }
-                if (keys.size() >= maxResults)
-                {
-                    rangeCompletedLocally = true;
-                    break;
-                }
-            }
-            return new RangeReply(keys, rangeCompletedLocally);
-        }
-        finally
-        {
-            for (Iterator iter : iterators)
-            {
-                if (iter instanceof Closeable)
-                {
-                    ((Closeable)iter).close();
-                }
-            }
-        }
-    }
-
-    /**
-     * @param startWith key to start with, inclusive.  empty string = start at 
beginning.
-     * @param stopAt key to stop at, inclusive.  empty string = stop only when 
keys are exhausted.
-     * @param maxResults
-     * @return list of keys between startWith and stopAt
 
        TODO refactor better.  this is just getKeyRange w/o the deletion check, 
for the benefit of
        range_slice.  still opens one randomaccessfile per key, which sucks.  
something like compactioniterator
        would be better.
      */
-    public RangeReply getKeyRangeRaw(final DecoratedKey startWith, final 
DecoratedKey stopAt, int maxResults)
+    public RangeReply getKeyRange(final DecoratedKey startWith, final 
DecoratedKey stopAt, int maxResults)
     throws IOException, ExecutionException, InterruptedException
     {
         // create a CollatedIterator that will return unique keys from 
different sources
@@ -1161,7 +1043,7 @@
     public RangeSliceReply getRangeSlice(byte[] super_column, final 
DecoratedKey startKey, final DecoratedKey finishKey, int keyMax, SliceRange 
sliceRange, List<byte[]> columnNames)
     throws IOException, ExecutionException, InterruptedException
     {
-        RangeReply rr = getKeyRangeRaw(startKey, finishKey, keyMax);
+        RangeReply rr = getKeyRange(startKey, finishKey, keyMax);
         List<Row> rows = new ArrayList<Row>(rr.keys.size());
         final QueryPath queryPath =  new QueryPath(columnFamily_, 
super_column, null);
         final SortedSet<byte[]> columnNameSet = new 
TreeSet<byte[]>(getComparator());

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/CassandraServer.java
 Wed Jan 20 13:57:49 2010
@@ -625,6 +625,7 @@
 
         checkLoginDone();
 
+        ThriftValidation.validateColumnParent(keyspace, column_parent);
         ThriftValidation.validatePredicate(keyspace, column_parent, predicate);
         if (!StorageService.getPartitioner().preservesOrder())
         {
@@ -663,38 +664,6 @@
         return keySlices;
     }
 
-    public List<String> get_key_range(String tablename, String columnFamily, 
String startWith, String stopAt, int maxResults, ConsistencyLevel 
consistency_level)
-            throws InvalidRequestException, TException, UnavailableException, 
TimedOutException
-    {
-        if (logger.isDebugEnabled())
-            logger.debug("get_key_range");
-
-        checkLoginDone();
-
-        ThriftValidation.validateCommand(tablename, columnFamily);
-        if (!StorageService.getPartitioner().preservesOrder())
-        {
-            throw new InvalidRequestException("range queries may only be 
performed against an order-preserving partitioner");
-        }
-        if (maxResults <= 0)
-        {
-            throw new InvalidRequestException("maxResults must be positive");
-        }
-
-        try
-        {
-            return StorageProxy.getKeyRange(new RangeCommand(tablename, 
columnFamily, startWith, stopAt, maxResults));
-        }
-        catch (TimeoutException e)
-        {
-               throw new TimedOutException();
-        }
-        catch (IOException e)
-        {
-            throw new RuntimeException(e);
-        }
-    }
-
     @Override
     public void login(String keyspace, AuthenticationRequest auth_request) 
throws AuthenticationException, AuthorizationException, TException
     {

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageProxy.java
 Wed Jan 20 13:57:49 2010
@@ -592,53 +592,6 @@
         return results;
     }
 
-    static List<String> getKeyRange(RangeCommand command) throws IOException, 
UnavailableException, TimeoutException
-    {
-        long startTime = System.currentTimeMillis();
-        TokenMetadata tokenMetadata = 
StorageService.instance.getTokenMetadata();
-        Set<String> uniqueKeys = new HashSet<String>(command.maxResults);
-
-        InetAddress endPoint = 
StorageService.instance.findSuitableEndPoint(command.startWith);
-        InetAddress startEndpoint = endPoint;
-
-        do
-        {
-            Message message = command.getMessage();
-            if (logger.isDebugEnabled())
-                logger.debug("reading " + command + " from " + 
message.getMessageId() + "@" + endPoint);
-            IAsyncResult iar = MessagingService.instance().sendRR(message, 
endPoint);
-
-            // read response
-            byte[] responseBody;
-            responseBody = iar.get(DatabaseDescriptor.getRpcTimeout(), 
TimeUnit.MILLISECONDS);
-           
-            RangeReply rangeReply = RangeReply.read(responseBody);
-            uniqueKeys.addAll(rangeReply.keys);
-
-            if (uniqueKeys.size() >= command.maxResults || 
rangeReply.rangeCompletedLocally)
-            {
-                break;
-            }
-
-            // set up the next query --
-            // it's tempting to try to optimize this by starting with the last 
key seen for the next node,
-            // but that won't work when you have a replication factor of more 
than one--any node, not just
-            // the one holding the keys where the range wraps, could include 
both the smallest keys, and the largest,
-            // so starting with the largest in our scan of the next node means 
we'd never see keys from the middle.
-            do
-            {
-                endPoint = tokenMetadata.getSuccessor(endPoint);
-            } while (!FailureDetector.instance.isAlive(endPoint));
-        } while (!endPoint.equals(startEndpoint));
-
-        rangeStats.add(System.currentTimeMillis() - startTime);
-        List<String> allKeys = new ArrayList<String>(uniqueKeys);
-        Collections.sort(allKeys, keyComparator);
-        return (allKeys.size() > command.maxResults)
-               ? allKeys.subList(0, command.maxResults)
-               : allKeys;
-    }
-
     public double getReadLatency()
     {
         return readStats.mean();

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/StorageService.java
 Wed Jan 20 13:57:49 2010
@@ -83,7 +83,6 @@
     public final static String streamInitiateDoneVerbHandler_ = 
"BOOTSTRAP-INITIATE-DONE-VERB-HANDLER";
     public final static String streamFinishedVerbHandler_ = 
"BOOTSTRAP-TERMINATE-VERB-HANDLER";
     public final static String streamRequestVerbHandler_ = 
"BS-METADATA-VERB-HANDLER";
-    public final static String rangeVerbHandler_ = "RANGE-VERB-HANDLER";
     public final static String rangeSliceVerbHandler_ = 
"RANGE-SLICE-VERB-HANDLER";
     public final static String bootstrapTokenVerbHandler_ = 
"SPLITS-VERB-HANDLER";
 
@@ -193,7 +192,6 @@
         MessagingService.instance().registerVerbHandlers(mutationVerbHandler_, 
new RowMutationVerbHandler());
         
MessagingService.instance().registerVerbHandlers(readRepairVerbHandler_, new 
ReadRepairVerbHandler());
         MessagingService.instance().registerVerbHandlers(readVerbHandler_, new 
ReadVerbHandler());
-        MessagingService.instance().registerVerbHandlers(rangeVerbHandler_, 
new RangeVerbHandler());
         
MessagingService.instance().registerVerbHandlers(rangeSliceVerbHandler_, new 
RangeSliceVerbHandler());
         // see BootStrapper for a summary of how the bootstrap verbs interact
         
MessagingService.instance().registerVerbHandlers(bootstrapTokenVerbHandler_, 
new BootStrapper.BootstrapTokenVerbHandler());

Modified: 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ThriftValidation.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ThriftValidation.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ThriftValidation.java
 (original)
+++ 
incubator/cassandra/trunk/src/java/org/apache/cassandra/service/ThriftValidation.java
 Wed Jan 20 13:57:49 2010
@@ -44,18 +44,6 @@
         }
     }
 
-    static void validateCommand(String tablename, String... columnFamilyNames) 
throws KeyspaceNotDefinedException, ColumnFamilyNotDefinedException
-    {
-        validateTable(tablename);
-        for (String cfName : columnFamilyNames)
-        {
-            if (DatabaseDescriptor.getColumnType(tablename, cfName) == null)
-            {
-                throw new ColumnFamilyNotDefinedException("Column Family " + 
cfName + " is invalid.");
-            }
-        }
-    }
-
     private static void validateTable(String tablename) throws 
KeyspaceNotDefinedException
     {
         if (!DatabaseDescriptor.getTables().contains(tablename))

Modified: incubator/cassandra/trunk/test/system/test_server.py
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/system/test_server.py?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- incubator/cassandra/trunk/test/system/test_server.py (original)
+++ incubator/cassandra/trunk/test/system/test_server.py Wed Jan 20 13:57:49 
2010
@@ -505,7 +505,7 @@
         # get doesn't specify supercolumn name
         _expect_exception(lambda: client.get('Keyspace1', 'key1', 
ColumnPath('Super1'), ConsistencyLevel.ONE), InvalidRequestException)
         # invalid CF
-        _expect_exception(lambda: client.get_key_range('Keyspace1', 'S', '', 
'', 1000, ConsistencyLevel.ONE), InvalidRequestException)
+        _expect_exception(lambda: client.get_range_slice('Keyspace1', 
ColumnParent('S'), SlicePredicate(column_names=['', '']), '', '', 5, 
ConsistencyLevel.ONE), InvalidRequestException)
         # 'x' is not a valid Long
         _expect_exception(lambda: client.insert('Keyspace1', 'key1', 
ColumnPath('Super1', 'sc1', 'x'), 'value', 0, ConsistencyLevel.ONE), 
InvalidRequestException)
         # start is not a valid Long
@@ -672,49 +672,57 @@
 
 
     def test_empty_range(self):
-        assert client.get_key_range('Keyspace1', 'Standard1', '', '', 1000, 
ConsistencyLevel.ONE) == []
+        assert client.get_range_slice('Keyspace1', ColumnParent('Standard1'), 
SlicePredicate(column_names=['c1', 'c1']), '', '', 1000, ConsistencyLevel.ONE) 
== []
         _insert_simple()
-        assert client.get_key_range('Keyspace1', 'Super1', '', '', 1000, 
ConsistencyLevel.ONE) == []
+        assert client.get_range_slice('Keyspace1', ColumnParent('Super1'), 
SlicePredicate(column_names=['c1', 'c1']), '', '', 1000, ConsistencyLevel.ONE) 
== []
 
     def test_range_with_remove(self):
         _insert_simple()
-        assert client.get_key_range('Keyspace1', 'Standard1', 'key1', '', 
1000, ConsistencyLevel.ONE) == ['key1']
+        assert client.get_range_slice('Keyspace1', ColumnParent('Standard1'), 
SlicePredicate(column_names=['c1', 'c1']), 'key1', '', 1000, 
ConsistencyLevel.ONE)[0].key == 'key1'
 
         client.remove('Keyspace1', 'key1', ColumnPath('Standard1', 
column='c1'), 1, ConsistencyLevel.ONE)
         client.remove('Keyspace1', 'key1', ColumnPath('Standard1', 
column='c2'), 1, ConsistencyLevel.ONE)
-        actual = client.get_key_range('Keyspace1', 'Standard1', '', '', 1000, 
ConsistencyLevel.ONE)
-        assert actual == [], actual
+        actual = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['c1', 'c2']), '', '', 
1000, ConsistencyLevel.ONE)
+        assert actual == [KeySlice(columns=[], key='key1')], actual
 
     def test_range_with_remove_cf(self):
         _insert_simple()
-        assert client.get_key_range('Keyspace1', 'Standard1', 'key1', '', 
1000, ConsistencyLevel.ONE) == ['key1']
+        assert client.get_range_slice('Keyspace1', ColumnParent('Standard1'), 
SlicePredicate(column_names=['c1', 'c1']), 'key1', '', 1000, 
ConsistencyLevel.ONE)[0].key == 'key1'
 
         client.remove('Keyspace1', 'key1', ColumnPath('Standard1'), 1, 
ConsistencyLevel.ONE)
-        actual = client.get_key_range('Keyspace1', 'Standard1', '', '', 1000, 
ConsistencyLevel.ONE)
-        assert actual == [], actual
+        actual = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['c1', 'c1']), '', '', 
1000, ConsistencyLevel.ONE)
+        assert actual == [KeySlice(columns=[], key='key1')], actual
 
     def test_range_collation(self):
         for key in ['-a', '-b', 'a', 'b'] + [str(i) for i in xrange(100)]:
             client.insert('Keyspace1', key, ColumnPath('Standard1', 
column=key), 'v', 0, ConsistencyLevel.ONE)
-        L = client.get_key_range('Keyspace1', 'Standard1', '', '', 1000, 
ConsistencyLevel.ONE)
+        slices = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['-a', '-a']), '', '', 
1000, ConsistencyLevel.ONE)
         # note the collated ordering rather than ascii
-        assert L == ['0', '1', '10', '11', '12', '13', '14', '15', '16', '17', 
'18', '19', '2', '20', '21', '22', '23', '24', '25', '26', '27','28', '29', 
'3', '30', '31', '32', '33', '34', '35', '36', '37', '38', '39', '4', '40', 
'41', '42', '43', '44', '45', '46', '47', '48', '49', '5', '50', '51', '52', 
'53', '54', '55', '56', '57', '58', '59', '6', '60', '61', '62', '63', '64', 
'65', '66', '67', '68', '69', '7', '70', '71', '72', '73', '74', '75', '76', 
'77', '78', '79', '8', '80', '81', '82', '83', '84', '85', '86', '87', '88', 
'89', '9', '90', '91', '92', '93', '94', '95', '96', '97', '98', '99', 'a', 
'-a', 'b', '-b'], L
+        L = ['0', '1', '10', '11', '12', '13', '14', '15', '16', '17', '18', 
'19', '2', '20', '21', '22', '23', '24', '25', '26', '27','28', '29', '3', 
'30', '31', '32', '33', '34', '35', '36', '37', '38', '39', '4', '40', '41', 
'42', '43', '44', '45', '46', '47', '48', '49', '5', '50', '51', '52', '53', 
'54', '55', '56', '57', '58', '59', '6', '60', '61', '62', '63', '64', '65', 
'66', '67', '68', '69', '7', '70', '71', '72', '73', '74', '75', '76', '77', 
'78', '79', '8', '80', '81', '82', '83', '84', '85', '86', '87', '88', '89', 
'9', '90', '91', '92', '93', '94', '95', '96', '97', '98', '99', 'a', '-a', 
'b', '-b']
+        assert len(slices) == len(L)
+        for key, ks in zip(L, slices):
+            assert key == ks.key
 
     def test_range_partial(self):
         for key in ['-a', '-b', 'a', 'b'] + [str(i) for i in xrange(100)]:
             client.insert('Keyspace1', key, ColumnPath('Standard1', 
column=key), 'v', 0, ConsistencyLevel.ONE)
+        
+        def check_slices_against_keys(keyList, sliceList):
+            assert len(keyList) == len(sliceList)
+            for key, ks in zip(keyList, sliceList):
+                assert key == ks.key
+        
+        slices = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['-a', '-a']), 'a', '', 
1000, ConsistencyLevel.ONE)
+        check_slices_against_keys(['a', '-a', 'b', '-b'], slices)
+        
+        slices = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['-a', '-a']), '', '15', 
1000, ConsistencyLevel.ONE)
+        check_slices_against_keys(['0', '1', '10', '11', '12', '13', '14', 
'15'], slices)
 
-        L = client.get_key_range('Keyspace1', 'Standard1', 'a', '', 1000, 
ConsistencyLevel.ONE)
-        assert L == ['a', '-a', 'b', '-b'], L
-
-        L = client.get_key_range('Keyspace1', 'Standard1', '', '15', 1000, 
ConsistencyLevel.ONE)
-        assert L == ['0', '1', '10', '11', '12', '13', '14', '15'], L
-
-        L = client.get_key_range('Keyspace1', 'Standard1', '50', '51', 1000, 
ConsistencyLevel.ONE)
-        assert L == ['50', '51'], L
-    
-        L = client.get_key_range('Keyspace1', 'Standard1', '1', '', 10, 
ConsistencyLevel.ONE)
-        assert L == ['1', '10', '11', '12', '13', '14', '15', '16', '17', 
'18'], L
+        slices = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['-a', '-a']), '50', 
'51', 1000, ConsistencyLevel.ONE)
+        check_slices_against_keys(['50', '51'], slices)
+        
+        slices = client.get_range_slice('Keyspace1', 
ColumnParent('Standard1'), SlicePredicate(column_names=['-a', '-a']), '1', '', 
10, ConsistencyLevel.ONE)
+        check_slices_against_keys(['1', '10', '11', '12', '13', '14', '15', 
'16', '17', '18'], slices)
 
     def test_get_slice_range(self):
         _insert_range()

Modified: 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
 (original)
+++ 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/CompactionsTest.java
 Wed Jan 20 13:57:49 2010
@@ -27,6 +27,7 @@
 import java.util.HashSet;
 import java.util.Collection;
 
+import org.apache.cassandra.service.StorageService;
 import org.junit.Test;
 
 import org.apache.cassandra.io.SSTableReader;
@@ -50,6 +51,7 @@
         // this test does enough rows to force multiple block indexes to be 
used
         Table table = Table.open(TABLE1);
         ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        DecoratedKey emptyKey = 
StorageService.getPartitioner().decorateKey("");
 
         final int ROWS_PER_SSTABLE = 10;
         Set<String> inserted = new HashSet<String>();
@@ -62,7 +64,7 @@
                 inserted.add(key);
             }
             store.forceBlockingFlush();
-            assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange("", "", 10000).keys.size());
+            assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange(emptyKey, emptyKey, 
10000).keys.size());
         }
         while (true)
         {
@@ -74,7 +76,7 @@
         {
             CompactionManager.instance.submitMajor(store).get();
         }
-        assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange("", "", 10000).keys.size());
+        assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange(emptyKey, emptyKey, 
10000).keys.size());
     }
 
     @Test
@@ -84,6 +86,7 @@
 
         Table table = Table.open(TABLE2);
         ColumnFamilyStore store = table.getColumnFamilyStore("Standard1");
+        DecoratedKey emptyKey = 
StorageService.getPartitioner().decorateKey("");
 
         final int ROWS_PER_SSTABLE = 10;
         Set<String> inserted = new HashSet<String>();
@@ -96,13 +99,13 @@
                 inserted.add(key);
             }
             store.forceBlockingFlush();
-            assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange("", "", 10000).keys.size());
+            assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange(emptyKey, emptyKey, 
10000).keys.size());
         }
 
         // perform readonly compaction and confirm that no sstables changed
         ArrayList<SSTableReader> oldsstables = new 
ArrayList<SSTableReader>(store.getSSTables());
         CompactionManager.instance.submitReadonly(store, LOCAL).get();
         assertEquals(oldsstables, new 
ArrayList<SSTableReader>(store.getSSTables()));
-        assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange("", "", 10000).keys.size());
+        assertEquals(inserted.size(), 
table.getColumnFamilyStore("Standard1").getKeyRange(emptyKey, emptyKey, 
10000).keys.size());
     }
 }

Modified: 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
 (original)
+++ 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/OneCompactionTest.java
 Wed Jan 20 13:57:49 2010
@@ -20,10 +20,10 @@
 
 import java.io.IOException;
 import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Future;
 import java.util.Set;
 import java.util.HashSet;
 
+import org.apache.cassandra.service.StorageService;
 import org.junit.Test;
 
 import static junit.framework.Assert.assertEquals;
@@ -38,6 +38,7 @@
 
         Table table = Table.open("Keyspace1");
         ColumnFamilyStore store = table.getColumnFamilyStore(columnFamilyName);
+        DecoratedKey emptyKey = 
StorageService.getPartitioner().decorateKey("");
 
         Set<String> inserted = new HashSet<String>();
         for (int j = 0; j < insertsPerTable; j++) {
@@ -47,11 +48,10 @@
             rm.apply();
             inserted.add(key);
             store.forceBlockingFlush();
-            assertEquals(inserted.size(), 
table.getColumnFamilyStore(columnFamilyName).getKeyRange("", "", 
10000).keys.size());
+            assertEquals(inserted.size(), 
table.getColumnFamilyStore(columnFamilyName).getKeyRange(emptyKey, emptyKey, 
10000).keys.size());
         }
         CompactionManager.instance.submitMajor(store).get();
         assertEquals(1, store.getSSTables().size());
-        
assertEquals(table.getColumnFamilyStore(columnFamilyName).getKeyRange("", "", 
10000).keys.size(), inserted.size());
     }
 
     @Test

Modified: 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
 (original)
+++ 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/db/RecoveryManager2Test.java
 Wed Jan 20 13:57:49 2010
@@ -27,6 +27,7 @@
 import java.util.HashSet;
 import java.util.Collections;
 
+import org.apache.cassandra.service.StorageService;
 import org.junit.Test;
 
 import org.apache.cassandra.CleanupHelper;
@@ -38,6 +39,7 @@
     public void testWithFlush() throws IOException, ExecutionException, 
InterruptedException
     {
         CompactionManager.instance.disableAutoCompaction();
+        DecoratedKey emptyKey = 
StorageService.getPartitioner().decorateKey("");
 
         for (int i = 0; i < 100; i++)
         {
@@ -52,7 +54,7 @@
         cfs.clearUnsafe();
         RecoveryManager.doRecovery(); // this is a no-op. is testing this 
useful?
 
-        Set<String> foundKeys = new HashSet<String>(cfs.getKeyRange("", "", 
1000).keys);
+        Set<String> foundKeys = new HashSet<String>(cfs.getKeyRange(emptyKey, 
emptyKey, 1000).keys);
         assert foundKeys.equals(Collections.emptySet());
     }
 

Modified: 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/StreamingTest.java
URL: 
http://svn.apache.org/viewvc/incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/StreamingTest.java?rev=901207&r1=901206&r2=901207&view=diff
==============================================================================
--- 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/StreamingTest.java 
(original)
+++ 
incubator/cassandra/trunk/test/unit/org/apache/cassandra/io/StreamingTest.java 
Wed Jan 20 13:57:49 2010
@@ -19,14 +19,13 @@
 package org.apache.cassandra.io;
 
 import static junit.framework.Assert.assertEquals;
-import static org.junit.Assert.*;
 
 import java.net.InetAddress;
-import java.io.IOException;
 import java.util.*;
 
 import org.apache.cassandra.CleanupHelper;
 import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.db.RangeReply;
 import org.apache.cassandra.db.Table;
 import org.apache.cassandra.io.SSTableUtils;
@@ -44,6 +43,7 @@
     public void testTransferTable() throws Exception
     {
         StorageService.instance.initServer();
+        DecoratedKey emptyKey = 
StorageService.getPartitioner().decorateKey("");
 
         // write a temporary SSTable, but don't register it
         Set<String> content = new HashSet<String>();
@@ -57,7 +57,7 @@
 
         // confirm that the SSTable was transferred and registered
         ColumnFamilyStore cfstore = 
Table.open(tablename).getColumnFamilyStore(cfname);
-        RangeReply rr = cfstore.getKeyRange("", "", 2);
+        RangeReply rr = cfstore.getKeyRange(emptyKey, emptyKey, 2);
         assert rr.keys.size() == 1;
         assert rr.keys.contains("key");
     }


Reply via email to