Author: jbellis
Date: Thu Jul 15 03:02:21 2010
New Revision: 964289

URL: http://svn.apache.org/viewvc?rev=964289&view=rev
Log:
Remove convertToDiskFormat, and version-conditionally apply 
convertFromDiskFormat
patch by Stu Hood; reviewed by jbellis for CASSANDRA-1249

Modified:
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
    
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
    
cassandra/trunk/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
    cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java
    cassandra/trunk/src/java/org/apache/cassandra/dht/LocalPartitioner.java
    
cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
    cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
    cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
    cassandra/trunk/src/java/org/apache/cassandra/tools/SSTableExport.java
    cassandra/trunk/test/conf/cassandra.yaml
    cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
    
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
    
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableNamesIterator.java
 Thu Jul 15 03:02:21 2010
@@ -70,7 +70,9 @@ public class SSTableNamesIterator extend
                 file = ssTable.getFileDataInput(decoratedKey, 
DatabaseDescriptor.getIndexedReadBufferSizeInKB() * 1024);
                 if (file == null)
                     return;
-                DecoratedKey keyInDisk = 
ssTable.getPartitioner().convertFromDiskFormat(FBUtilities.readShortByteArray(file));
+                DecoratedKey keyInDisk = 
SSTableReader.decodeKey(ssTable.getPartitioner(),
+                                                                 
ssTable.getDescriptor(),
+                                                                 
FBUtilities.readShortByteArray(file));
                 assert keyInDisk.equals(decoratedKey)
                        : String.format("%s != %s in %s", keyInDisk, 
decoratedKey, file.getPath());
                 SSTableReader.readRowSize(file, ssTable.getDescriptor());

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/db/filter/SSTableSliceIterator.java
 Thu Jul 15 03:02:21 2010
@@ -88,7 +88,9 @@ class SSTableSliceIterator extends Abstr
                 return;
             try
             {
-                DecoratedKey keyInDisk = 
ssTable.getPartitioner().convertFromDiskFormat(FBUtilities.readShortByteArray(file));
+                DecoratedKey keyInDisk = 
SSTableReader.decodeKey(ssTable.getPartitioner(),
+                                                                 
ssTable.getDescriptor(),
+                                                                 
FBUtilities.readShortByteArray(file));
                 assert keyInDisk.equals(decoratedKey)
                        : String.format("%s != %s in %s", keyInDisk, 
decoratedKey, file.getPath());
                 SSTableReader.readRowSize(file, ssTable.getDescriptor());

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/dht/AbstractByteOrderedPartitioner.java
 Thu Jul 15 03:02:21 2010
@@ -47,11 +47,6 @@ public abstract class AbstractByteOrdere
         return new DecoratedKey<BytesToken>(getToken(key), key);
     }
 
-    public byte[] convertToDiskFormat(DecoratedKey<BytesToken> key)
-    {
-        return key.key;
-    }
-
     public BytesToken midpoint(BytesToken ltoken, BytesToken rtoken)
     {
         int sigbytes = Math.max(ltoken.token.length, rtoken.token.length);

Modified: cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/IPartitioner.java Thu Jul 
15 03:02:21 2010
@@ -25,6 +25,8 @@ import org.apache.cassandra.db.Decorated
 public interface IPartitioner<T extends Token>
 {
     /**
+     * @Deprecated: Used by SSTables before version 'e'.
+     *
      * Convert the on disk representation to a DecoratedKey object
      * @param key On disk representation 
      * @return DecoratedKey object
@@ -32,14 +34,6 @@ public interface IPartitioner<T extends 
     public DecoratedKey<T> convertFromDiskFormat(byte[] key);
     
     /**
-     * Convert the DecoratedKey to the on disk format used for
-     * this partitioner.
-     * @param key The DecoratedKey in question
-     * @return
-     */
-    public byte[] convertToDiskFormat(DecoratedKey<T> key);    
-    
-    /**
      * Transform key to object representation of the on-disk format.
      *
      * @param key the raw, client-facing key

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/dht/LocalPartitioner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/LocalPartitioner.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/LocalPartitioner.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/LocalPartitioner.java Thu 
Jul 15 03:02:21 2010
@@ -19,11 +19,6 @@ public class LocalPartitioner implements
         return decorateKey(key);
     }
 
-    public byte[] convertToDiskFormat(DecoratedKey<LocalToken> key)
-    {
-        return key.token.token;
-    }
-
     public DecoratedKey<LocalToken> decorateKey(byte[] key)
     {
         return new DecoratedKey<LocalToken>(getToken(key), key);

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
 (original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
 Thu Jul 15 03:02:21 2010
@@ -46,11 +46,6 @@ public class OrderPreservingPartitioner 
         return new DecoratedKey<StringToken>(getToken(key), key);
     }
 
-    public byte[] convertToDiskFormat(DecoratedKey<StringToken> key)
-    {
-        return key.key;
-    }
-
     public StringToken midpoint(StringToken ltoken, StringToken rtoken)
     {
         int sigchars = Math.max(ltoken.token.length(), rtoken.token.length());

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/dht/RandomPartitioner.java 
Thu Jul 15 03:02:21 2010
@@ -64,21 +64,6 @@ public class RandomPartitioner implement
         return new DecoratedKey<BigIntegerToken>(new BigIntegerToken(token), 
key);
     }
 
-    public byte[] convertToDiskFormat(DecoratedKey<BigIntegerToken> key)
-    {
-        // encode token prefix and calculate final length (with delimiter)
-        byte[] prefix = key.token.toString().getBytes(UTF_8);
-        int length = prefix.length + 1 + key.key.length;
-        assert length <= FBUtilities.MAX_UNSIGNED_SHORT;
-
-        // copy into output bytes
-        byte[] todisk = new byte[length];
-        System.arraycopy(prefix, 0, todisk, 0, prefix.length);
-        todisk[prefix.length] = DELIMITER_BYTE;
-        System.arraycopy(key.key, 0, todisk, prefix.length + 1, 
key.key.length);
-        return todisk;
-    }
-
     public BigIntegerToken midpoint(BigIntegerToken ltoken, BigIntegerToken 
rtoken)
     {
         Pair<BigInteger,Boolean> midpair = FBUtilities.midpoint(ltoken.token, 
rtoken.token, 127);

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/Descriptor.java 
Thu Jul 15 03:02:21 2010
@@ -15,7 +15,7 @@ import com.google.common.base.Objects;
 public class Descriptor
 {
     public static final String LEGACY_VERSION = "a";
-    public static final String CURRENT_VERSION = "d";
+    public static final String CURRENT_VERSION = "e";
 
     public final File directory;
     public final String version;
@@ -160,6 +160,11 @@ public class Descriptor
         return version.compareTo("d") < 0;
     }
 
+    public boolean hasEncodedKeys()
+    {
+        return version.compareTo("e") < 0;
+    }
+
     public boolean isLatestVersion()
     {
         return version.compareTo(CURRENT_VERSION) == 0;

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableReader.java 
Thu Jul 15 03:02:21 2010
@@ -266,7 +266,7 @@ public class SSTableReader extends SSTab
                 if (indexPosition == indexSize)
                     break;
 
-                DecoratedKey decoratedKey = 
partitioner.convertFromDiskFormat(FBUtilities.readShortByteArray(input));
+                DecoratedKey decoratedKey = decodeKey(partitioner, desc, 
FBUtilities.readShortByteArray(input));
                 if (recreatebloom)
                     bf.add(decoratedKey.key);
                 long dataPosition = input.readLong();
@@ -414,7 +414,7 @@ public class SSTableReader extends SSTab
                 while (!input.isEOF())
                 {
                     // read key & data position from index entry
-                    DecoratedKey indexDecoratedKey = 
partitioner.convertFromDiskFormat(FBUtilities.readShortByteArray(input));
+                    DecoratedKey indexDecoratedKey = decodeKey(partitioner, 
desc, FBUtilities.readShortByteArray(input));
                     long dataPosition = input.readLong();
 
                     int comparison = indexDecoratedKey.compareTo(decoratedKey);
@@ -557,6 +557,16 @@ public class SSTableReader extends SSTab
     }
 
     /**
+     * Conditionally use the deprecated 'IPartitioner.convertFromDiskFormat' 
method.
+     */
+    public static DecoratedKey decodeKey(IPartitioner p, Descriptor d, byte[] 
bytes)
+    {
+        if (d.hasEncodedKeys())
+            return p.convertFromDiskFormat(bytes);
+        return p.decorateKey(bytes);
+    }
+
+    /**
      * TODO: Move someplace reusable
      */
     public abstract static class Operator

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java 
(original)
+++ 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableScanner.java 
Thu Jul 15 03:02:21 2010
@@ -168,7 +168,9 @@ public class SSTableScanner implements I
                     file.seek(finishedAt);
                 assert !file.isEOF();
 
-                DecoratedKey key = 
StorageService.getPartitioner().convertFromDiskFormat(FBUtilities.readShortByteArray(file));
+                DecoratedKey key = 
SSTableReader.decodeKey(sstable.getPartitioner(),
+                                                           
sstable.getDescriptor(),
+                                                           
FBUtilities.readShortByteArray(file));
                 long dataSize = SSTableReader.readRowSize(file, 
sstable.getDescriptor());
                 dataStart = file.getFilePointer();
                 finishedAt = dataStart + dataSize;

Modified: 
cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/io/sstable/SSTableWriter.java 
Thu Jul 15 03:02:21 2010
@@ -99,7 +99,7 @@ public class SSTableWriter extends SSTab
     public void append(AbstractCompactedRow row) throws IOException
     {
         long currentPosition = beforeAppend(row.key);
-        
FBUtilities.writeShortByteArray(partitioner.convertToDiskFormat(row.key), 
dataFile);
+        FBUtilities.writeShortByteArray(row.key.key, dataFile);
         row.write(dataFile);
         afterAppend(row.key, currentPosition);
     }
@@ -107,7 +107,7 @@ public class SSTableWriter extends SSTab
     public void append(DecoratedKey decoratedKey, ColumnFamily cf) throws 
IOException
     {
         long startPosition = beforeAppend(decoratedKey);
-        
FBUtilities.writeShortByteArray(partitioner.convertToDiskFormat(decoratedKey), 
dataFile);
+        FBUtilities.writeShortByteArray(decoratedKey.key, dataFile);
         // write placeholder for the row size, since we don't know it yet
         long sizePosition = dataFile.getFilePointer();
         dataFile.writeLong(-1);
@@ -125,7 +125,7 @@ public class SSTableWriter extends SSTab
     public void append(DecoratedKey decoratedKey, byte[] value) throws 
IOException
     {
         long currentPosition = beforeAppend(decoratedKey);
-        
FBUtilities.writeShortByteArray(partitioner.convertToDiskFormat(decoratedKey), 
dataFile);
+        FBUtilities.writeShortByteArray(decoratedKey.key, dataFile);
         assert value.length > 0;
         dataFile.writeLong(value.length);
         dataFile.write(value);
@@ -237,7 +237,7 @@ public class SSTableWriter extends SSTab
             long dataPosition = 0;
             while (dataPosition < dfile.length())
             {
-                key = 
StorageService.getPartitioner().convertFromDiskFormat(FBUtilities.readShortByteArray(dfile));
+                key = SSTableReader.decodeKey(StorageService.getPartitioner(), 
desc, FBUtilities.readShortByteArray(dfile));
                 long dataSize = SSTableReader.readRowSize(dfile, desc);
                 iwriter.afterAppend(key, dataPosition);
                 dataPosition = dfile.getFilePointer() + dataSize;
@@ -301,7 +301,7 @@ public class SSTableWriter extends SSTab
         {
             bf.add(key.key);
             long indexPosition = indexFile.getFilePointer();
-            
FBUtilities.writeShortByteArray(partitioner.convertToDiskFormat(key), 
indexFile);
+            FBUtilities.writeShortByteArray(key.key, indexFile);
             indexFile.writeLong(dataPosition);
             if (logger.isTraceEnabled())
                 logger.trace("wrote index of " + key + " at " + indexPosition);

Modified: cassandra/trunk/src/java/org/apache/cassandra/tools/SSTableExport.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/src/java/org/apache/cassandra/tools/SSTableExport.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/src/java/org/apache/cassandra/tools/SSTableExport.java 
(original)
+++ cassandra/trunk/src/java/org/apache/cassandra/tools/SSTableExport.java Thu 
Jul 15 03:02:21 2010
@@ -31,6 +31,7 @@ import org.apache.cassandra.db.IColumn;
 import org.apache.cassandra.db.TimestampClock;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.io.sstable.Descriptor;
 import org.apache.cassandra.io.sstable.SSTable;
 import org.apache.cassandra.io.sstable.SSTableIdentityIterator;
 import org.apache.cassandra.io.sstable.SSTableReader;
@@ -157,10 +158,13 @@ public class SSTableExport
     throws IOException
     {
         IPartitioner partitioner = StorageService.getPartitioner();
+        Descriptor desc = Descriptor.fromFilename(ssTableFile);
         BufferedRandomAccessFile input = new 
BufferedRandomAccessFile(SSTable.indexFilename(ssTableFile), "r");
         while (!input.isEOF())
         {
-            DecoratedKey decoratedKey = 
partitioner.convertFromDiskFormat(FBUtilities.readShortByteArray(input));
+            DecoratedKey decoratedKey = SSTableReader.decodeKey(partitioner,
+                                                                desc,
+                                                                
FBUtilities.readShortByteArray(input));
             long dataPosition = input.readLong();
             outs.println(bytesToHex(decoratedKey.key));
         }

Modified: cassandra/trunk/test/conf/cassandra.yaml
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/conf/cassandra.yaml?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/test/conf/cassandra.yaml (original)
+++ cassandra/trunk/test/conf/cassandra.yaml Thu Jul 15 03:02:21 2010
@@ -1,3 +1,7 @@
+#
+# Warning!
+# Consider the effects on 'o.a.c.i.s.LegacySSTableTest' before changing 
schemas in this file.
+#
 cluster_name: Test Cluster
 in_memory_compaction_limit_in_mb: 1
 commitlog_sync: batch

Modified: 
cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java 
(original)
+++ cassandra/trunk/test/unit/org/apache/cassandra/dht/PartitionerTestCase.java 
Thu Jul 15 03:02:21 2010
@@ -101,15 +101,6 @@ public abstract class PartitionerTestCas
     }
     
     @Test
-    public void testDiskFormat()
-    {
-        byte[] key = "key".getBytes();
-        DecoratedKey<T> decKey = partitioner.decorateKey(key);
-        DecoratedKey<T> result = 
partitioner.convertFromDiskFormat(partitioner.convertToDiskFormat(decKey));
-        assertEquals(decKey, result);
-    }
-    
-    @Test
     public void testTokenFactoryBytes()
     {
         Token.TokenFactory factory = partitioner.getTokenFactory();

Modified: 
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
 (original)
+++ 
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
 Thu Jul 15 03:02:21 2010
@@ -48,7 +48,7 @@ public class LegacySSTableTest extends C
     {
         String scp = System.getProperty(LEGACY_SSTABLE_PROP);
         assert scp != null;
-        LEGACY_SSTABLE_ROOT = new File(scp);
+        LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
         assert LEGACY_SSTABLE_ROOT.isDirectory();
 
         TEST_DATA = new HashMap<byte[],byte[]>();
@@ -75,7 +75,7 @@ public class LegacySSTableTest extends C
     public void buildTestSSTable() throws IOException
     {
         // write the output in a version specific directory
-        SSTable.Descriptor dest = 
getDescriptor(SSTable.Descriptor.CURRENT_VERSION);
+        Descriptor dest = getDescriptor(Descriptor.CURRENT_VERSION);
         assert dest.directory.mkdirs() : "Could not create " + dest.directory 
+ ". Might it already exist?";
 
         SSTableReader ssTable = SSTableUtils.writeRawSSTable(new 
File(dest.filenameFor(SSTable.COMPONENT_DATA)),
@@ -88,22 +88,33 @@ public class LegacySSTableTest extends C
     }
     */
 
-    /**
-     * Between version b and c, on disk bloom filters became incompatible, and 
needed to be regenerated.
-     */
     @Test
-    public void testVerB() throws IOException
+    public void testVersions() throws IOException
     {
-        SSTableReader reader = SSTableReader.open(getDescriptor("b"));
+        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
+            testVersion(version.getName());
+    }
 
-        List<byte[]> keys = new ArrayList<byte[]>(TEST_DATA.keySet());
-        Collections.shuffle(keys);
-        BufferedRandomAccessFile file = new 
BufferedRandomAccessFile(reader.getFilename(), "r");
-        for (byte[] key : keys)
+    public void testVersion(String version)
+    {
+        try
+        {
+            SSTableReader reader = SSTableReader.open(getDescriptor(version));
+
+            List<byte[]> keys = new ArrayList<byte[]>(TEST_DATA.keySet());
+            Collections.shuffle(keys);
+            BufferedRandomAccessFile file = new 
BufferedRandomAccessFile(reader.getFilename(), "r");
+            for (byte[] key : keys)
+            {
+                // confirm that the bloom filter does not reject any keys
+                
file.seek(reader.getPosition(reader.partitioner.decorateKey(key), 
SSTableReader.Operator.EQ));
+                assert Arrays.equals(key, 
FBUtilities.readShortByteArray(file));
+            }
+        }
+        catch (Throwable e)
         {
-            // confirm that the bloom filter does not reject any keys
-            file.seek(reader.getPosition(reader.partitioner.decorateKey(key), 
SSTableReader.Operator.EQ));
-            assert Arrays.equals(key, FBUtilities.readShortByteArray(file));
+            System.err.println("Failed to read " + version);
+            e.printStackTrace(System.err);
         }
     }
 }

Modified: 
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
URL: 
http://svn.apache.org/viewvc/cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java?rev=964289&r1=964288&r2=964289&view=diff
==============================================================================
--- 
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
 (original)
+++ 
cassandra/trunk/test/unit/org/apache/cassandra/io/sstable/SSTableReaderTest.java
 Thu Jul 15 03:02:21 2010
@@ -97,7 +97,9 @@ public class SSTableReaderTest extends C
         {
             DecoratedKey dk = Util.dk(String.valueOf(j));
             FileDataInput file = sstable.getFileDataInput(dk, 
DatabaseDescriptor.getIndexedReadBufferSizeInKB() * 1024);
-            DecoratedKey keyInDisk = 
sstable.getPartitioner().convertFromDiskFormat(FBUtilities.readShortByteArray(file));
+            DecoratedKey keyInDisk = 
SSTableReader.decodeKey(sstable.getPartitioner(),
+                                                             
sstable.getDescriptor(),
+                                                             
FBUtilities.readShortByteArray(file));
             assert keyInDisk.equals(dk) : String.format("%s != %s in %s", 
keyInDisk, dk, file.getPath());
         }
 


Reply via email to