You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by yu...@apache.org on 2016/03/17 17:06:57 UTC

[01/12] cassandra git commit: Support streaming of older version sstables in 3.0

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-3.0 7f1339cc9 -> e8651b662
  refs/heads/cassandra-3.5 e3716ee36 -> 587773fa4
  refs/heads/trunk 704374408 -> 68e6a8333


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 501f4ae..3a1f348 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -25,7 +25,6 @@ import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
-import org.apache.cassandra.io.util.NIODataInputStream;
 import org.apache.cassandra.net.MessagingService;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 8732881..f864bbc 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -243,46 +243,76 @@ public class DirectoriesTest
         for (CFMetaData cfm : CFM)
         {
             Directories directories = new Directories(cfm);
-            Directories.SSTableLister lister;
-            Set<File> listed;
+            checkFiles(cfm, directories);
+        }
+    }
 
-            // List all but no snapshot, backup
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+    private void checkFiles(CFMetaData cfm, Directories directories)
+    {
+        Directories.SSTableLister lister;
+        Set<File> listed;// List all but no snapshot, backup
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
+        }
 
-            // List all but including backup (but no snapshot)
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+        // List all but including backup (but no snapshot)
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
+        }
 
-            // Skip temporary and compacted
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else if (f.getName().contains("tmp-"))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+        // Skip temporary and compacted
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else if (f.getName().contains("tmp-"))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
         }
     }
 
+    @Test
+    public void testTemporaryFile() throws IOException
+    {
+        for (CFMetaData cfm : CFM)
+        {
+            Directories directories = new Directories(cfm);
+
+            File tempDir = directories.getTemporaryWriteableDirectoryAsFile(10);
+            tempDir.mkdir();
+            File tempFile = new File(tempDir, "tempFile");
+            tempFile.createNewFile();
+
+            assertTrue(tempDir.exists());
+            assertTrue(tempFile.exists());
+
+            //make sure temp dir/file will not affect existing sstable listing
+            checkFiles(cfm, directories);
+
+            directories.removeTemporaryDirectories();
+
+            //make sure temp dir/file deletion will not affect existing sstable listing
+            checkFiles(cfm, directories);
+
+            assertFalse(tempDir.exists());
+            assertFalse(tempFile.exists());
+        }
+    }
 
     @Test
     public void testDiskFailurePolicy_best_effort()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/gms/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/SerializationsTest.java b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
index 8b7ad1f..e50b461 100644
--- a/test/unit/org/apache/cassandra/gms/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.gms;
 
 import org.apache.cassandra.AbstractSerializationsTester;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 9154d79..74b5c74 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.MmappedRegions;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriter;
@@ -84,7 +84,7 @@ public class CompressedRandomAccessReaderTest
                 for (int i = 0; i < 20; i++)
                     writer.write("x".getBytes());
 
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
                 // write enough garbage to create new chunks:
                 for (int i = 0; i < 40; ++i)
                     writer.write("y".getBytes());
@@ -126,7 +126,7 @@ public class CompressedRandomAccessReaderTest
                 : SequentialWriter.open(f))
             {
                 writer.write("The quick ".getBytes());
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
                 writer.write("blue fox jumps over the lazy dog".getBytes());
 
                 // write enough to be sure to change chunk

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index 56c83da..9b09f0b 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriterTest;
 import org.apache.cassandra.schema.CompressionParams;
@@ -104,7 +104,7 @@ public class CompressedSequentialWriterTest extends SequentialWriterTest
                 dataPost.flip();
 
                 writer.write(dataPre);
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
 
                 // Write enough garbage to transition chunk
                 for (int i = 0; i < CompressionParams.DEFAULT_CHUNK_LENGTH; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 00727b8..4b9a769 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -21,14 +21,13 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
-import java.util.Set;
 
+import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -37,16 +36,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.rows.SliceableUnfilteredRowIterator;
-import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -55,7 +48,6 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
-import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamPlan;
@@ -63,8 +55,6 @@ import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
 /**
  * Tests backwards compatibility for SSTables
  */
@@ -73,10 +63,7 @@ public class LegacySSTableTest
     private static final Logger logger = LoggerFactory.getLogger(LegacySSTableTest.class);
 
     public static final String LEGACY_SSTABLE_PROP = "legacy-sstable-root";
-    public static final String KSNAME = "Keyspace1";
-    public static final String CFNAME = "Standard1";
 
-    public static Set<String> TEST_DATA;
     public static File LEGACY_SSTABLE_ROOT;
 
     /**
@@ -104,58 +91,78 @@ public class LegacySSTableTest
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
-
-        CFMetaData metadata = CFMetaData.Builder.createDense(KSNAME, CFNAME, false, false)
-                                                .addPartitionKey("key", BytesType.instance)
-                                                .addClusteringColumn("column", BytesType.instance)
-                                                .addRegularColumn("value", BytesType.instance)
-                                                .build();
-
-        SchemaLoader.createKeyspace(KSNAME,
-                                    KeyspaceParams.simple(1),
-                                    metadata);
-        beforeClass();
-    }
-
-    public static void beforeClass()
-    {
+        StorageService.instance.initServer();
         Keyspace.setInitialized();
+        createKeyspace();
+        for (String legacyVersion : legacyVersions)
+        {
+            createTables(legacyVersion);
+        }
         String scp = System.getProperty(LEGACY_SSTABLE_PROP);
         assert scp != null;
         LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
         assert LEGACY_SSTABLE_ROOT.isDirectory();
+    }
 
-        TEST_DATA = new HashSet<String>();
-        for (int i = 100; i < 1000; ++i)
-            TEST_DATA.add(Integer.toString(i));
+    @After
+    public void tearDown()
+    {
+        for (String legacyVersion : legacyVersions)
+        {
+            truncateTables(legacyVersion);
+        }
     }
 
     /**
      * Get a descriptor for the legacy sstable at the given version.
      */
-    protected Descriptor getDescriptor(String ver)
+    protected Descriptor getDescriptor(String legacyVersion, String table)
     {
-        File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
-        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, SSTableFormat.Type.LEGACY);
+        return new Descriptor(legacyVersion, getTableDir(legacyVersion, table), "legacy_tables", table, 1,
+                              BigFormat.instance.getVersion(legacyVersion).hasNewFileName()?
+                              SSTableFormat.Type.BIG :SSTableFormat.Type.LEGACY);
     }
 
     @Test
-    public void testStreaming() throws Throwable
+    public void testLoadLegacyCqlTables() throws Exception
     {
-        StorageService.instance.initServer();
+        for (String legacyVersion : legacyVersions)
+        {
+            logger.info("Loading legacy version: {}", legacyVersion);
+            loadLegacyTables(legacyVersion);
+            CacheService.instance.invalidateKeyCache();
+            long startCount = CacheService.instance.keyCache.size();
+            verifyReads(legacyVersion);
+            verifyCache(legacyVersion, startCount);
+        }
+    }
+
+    @Test
+    public void testStreamLegacyCqlTables() throws Exception
+    {
+        for (String legacyVersion : legacyVersions)
+        {
+            streamLegacyTables(legacyVersion);
+            verifyReads(legacyVersion);
+        }
+    }
 
-        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
+    private void streamLegacyTables(String legacyVersion) throws Exception
+    {
+        for (int compact = 0; compact <= 1; compact++)
         {
-            if (!new File(LEGACY_SSTABLE_ROOT + File.separator + version.getName() + File.separator + KSNAME).isDirectory())
-                continue;
-            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatibleForStreaming())
-                testStreaming(version.getName());
+            logger.info("Streaming legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
         }
     }
 
-    private void testStreaming(String version) throws Exception
+    private void streamLegacyTable(String tablePattern, String legacyVersion, String compactNameSuffix) throws Exception
     {
-        SSTableReader sstable = SSTableReader.open(getDescriptor(version));
+        String table = String.format(tablePattern, legacyVersion, compactNameSuffix);
+        SSTableReader sstable = SSTableReader.open(getDescriptor(legacyVersion, table));
         IPartitioner p = sstable.getPartitioner();
         List<Range<Token>> ranges = new ArrayList<>();
         ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("100"))));
@@ -166,94 +173,41 @@ public class LegacySSTableTest
                                                                sstable.estimatedKeysForRanges(ranges), sstable.getSSTableMetadata().repairedAt));
         new StreamPlan("LegacyStreamingTest").transferFiles(FBUtilities.getBroadcastAddress(), details)
                                              .execute().get();
-
-        ColumnFamilyStore cfs = Keyspace.open(KSNAME).getColumnFamilyStore(CFNAME);
-        assert cfs.getLiveSSTables().size() == 1;
-        sstable = cfs.getLiveSSTables().iterator().next();
-        for (String keystring : TEST_DATA)
-        {
-            ByteBuffer key = bytes(keystring);
-
-            SliceableUnfilteredRowIterator iter = sstable.iterator(Util.dk(key), ColumnFilter.selectionBuilder().add(cfs.metadata.getColumnDefinition(bytes("name"))).build(), false, false);
-
-            // check not deleted (CASSANDRA-6527)
-            assert iter.partitionLevelDeletion().equals(DeletionTime.LIVE);
-            assert iter.next().clustering().get(0).equals(key);
-        }
-        sstable.selfRef().release();
-    }
-
-    @Test
-    public void testVersions() throws Throwable
-    {
-        boolean notSkipped = false;
-
-        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
-        {
-            if (!new File(LEGACY_SSTABLE_ROOT + File.separator + version.getName() + File.separator + KSNAME).isDirectory())
-                continue;
-            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatible())
-            {
-                notSkipped = true;
-                testVersion(version.getName());
-            }
-        }
-
-        assert notSkipped;
     }
 
-    public void testVersion(String version) throws Throwable
+    private static void loadLegacyTables(String legacyVersion) throws Exception
     {
-        try
-        {
-            ColumnFamilyStore cfs = Keyspace.open(KSNAME).getColumnFamilyStore(CFNAME);
-
-
-            SSTableReader reader = SSTableReader.open(getDescriptor(version));
-            for (String keystring : TEST_DATA)
-            {
-
-                ByteBuffer key = bytes(keystring);
-
-                SliceableUnfilteredRowIterator iter = reader.iterator(Util.dk(key), ColumnFilter.selection(cfs.metadata.partitionColumns()), false, false);
-
-                // check not deleted (CASSANDRA-6527)
-                assert iter.partitionLevelDeletion().equals(DeletionTime.LIVE);
-                assert iter.next().clustering().get(0).equals(key);
-            }
-
-            // TODO actually test some reads
-        }
-        catch (Throwable e)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            System.err.println("Failed to read " + version);
-            throw e;
+            logger.info("Preparing legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
         }
     }
 
-    @Test
-    public void testLegacyCqlTables() throws Exception
+    private static void verifyCache(String legacyVersion, long startCount) throws InterruptedException, java.util.concurrent.ExecutionException
     {
-        createKeyspace();
-
-        loadLegacyTables();
+        //For https://issues.apache.org/jira/browse/CASSANDRA-10778
+        //Validate whether the key cache successfully saves in the presence of old keys as
+        //well as loads the correct number of keys
+        long endCount = CacheService.instance.keyCache.size();
+        Assert.assertTrue(endCount > startCount);
+        CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
+        CacheService.instance.invalidateKeyCache();
+        Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
+        CacheService.instance.keyCache.loadSaved();
+        if (BigFormat.instance.getVersion(legacyVersion).storeRows())
+            Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
+        else
+            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
     }
 
-    private static void loadLegacyTables() throws Exception
+    private static void verifyReads(String legacyVersion)
     {
-        for (String legacyVersion : legacyVersions)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            logger.info("Preparing legacy version {}", legacyVersion);
-
-            createTables(legacyVersion);
-
-            loadLegacyTable("legacy_%s_simple", legacyVersion);
-            loadLegacyTable("legacy_%s_simple_counter", legacyVersion);
-            loadLegacyTable("legacy_%s_clust", legacyVersion);
-            loadLegacyTable("legacy_%s_clust_counter", legacyVersion);
-
-            CacheService.instance.invalidateKeyCache();
-            long startCount = CacheService.instance.keyCache.size();
             for (int ck = 0; ck < 50; ck++)
             {
                 String ckValue = Integer.toString(ck) + longString;
@@ -265,58 +219,94 @@ public class LegacySSTableTest
                     UntypedResultSet rs;
                     if (ck == 0)
                     {
-                        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple WHERE pk=?", legacyVersion), pkValue);
-                        Assert.assertNotNull(rs);
-                        Assert.assertEquals(1, rs.size());
-                        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
-                        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter WHERE pk=?", legacyVersion), pkValue);
-                        Assert.assertNotNull(rs);
-                        Assert.assertEquals(1, rs.size());
-                        Assert.assertEquals(1L, rs.one().getLong("val"));
+                        readSimpleTable(legacyVersion, getCompactNameSuffix(compact),  pkValue);
+                        readSimpleCounterTable(legacyVersion, getCompactNameSuffix(compact), pkValue);
                     }
 
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
-                    assertLegacyClustRows(1, rs);
-
-                    String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
-                    String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion), pkValue, ckValue, ckValue2, ckValue3);
-                    assertLegacyClustRows(3, rs);
-
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
-                    Assert.assertNotNull(rs);
-                    Assert.assertEquals(1, rs.size());
-                    Assert.assertEquals(1L, rs.one().getLong("val"));
+                    readClusteringTable(legacyVersion, getCompactNameSuffix(compact), ck, ckValue, pkValue);
+                    readClusteringCounterTable(legacyVersion, getCompactNameSuffix(compact), ckValue, pkValue);
                 }
             }
-
-            //For https://issues.apache.org/jira/browse/CASSANDRA-10778
-            //Validate whether the key cache successfully saves in the presence of old keys as
-            //well as loads the correct number of keys
-            long endCount = CacheService.instance.keyCache.size();
-            Assert.assertTrue(endCount > startCount);
-            CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
-            CacheService.instance.invalidateKeyCache();
-            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
-            CacheService.instance.keyCache.loadSaved();
-            if (BigFormat.instance.getVersion(legacyVersion).storeRows())
-                Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
-            else
-                Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
         }
     }
 
-    private void createKeyspace()
+    private static void readClusteringCounterTable(String legacyVersion, String compactSuffix, String ckValue, String pkValue)
+    {
+        logger.debug("Read legacy_{}_clust_counter{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals(1L, rs.one().getLong("val"));
+    }
+
+    private static void readClusteringTable(String legacyVersion, String compactSuffix, int ck, String ckValue, String pkValue)
+    {
+        logger.debug("Read legacy_{}_clust{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        assertLegacyClustRows(1, rs);
+
+        String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
+        String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion, compactSuffix), pkValue, ckValue, ckValue2, ckValue3);
+        assertLegacyClustRows(3, rs);
+    }
+
+    private static void readSimpleCounterTable(String legacyVersion, String compactSuffix, String pkValue)
+    {
+        logger.debug("Read legacy_{}_simple_counter{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals(1L, rs.one().getLong("val"));
+    }
+
+    private static void readSimpleTable(String legacyVersion, String compactSuffix, String pkValue)
+    {
+        logger.debug("Read simple: legacy_{}_simple{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
+    }
+
+    private static void createKeyspace()
     {
         QueryProcessor.executeInternal("CREATE KEYSPACE legacy_tables WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
     }
 
     private static void createTables(String legacyVersion)
     {
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple (pk text PRIMARY KEY, val text)", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter (pk text PRIMARY KEY, val counter)", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter (pk text, ck text, val counter, PRIMARY KEY (pk, ck))", legacyVersion));
+        for (int i=0; i<=1; i++)
+        {
+            String compactSuffix = getCompactNameSuffix(i);
+            String tableSuffix = i == 0? "" : " WITH COMPACT STORAGE";
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple%s (pk text PRIMARY KEY, val text)%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter%s (pk text PRIMARY KEY, val counter)%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust%s (pk text, ck text, val text, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter%s (pk text, ck text, val counter, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
+        }
+    }
+
+    private static String getCompactNameSuffix(int i)
+    {
+        return i == 0? "" : "_compact";
+    }
+
+    private static void truncateTables(String legacyVersion)
+    {
+        for (int compact = 0; compact <= 1; compact++)
+        {
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact)));
+        }
+        CacheService.instance.invalidateCounterCache();
+        CacheService.instance.invalidateKeyCache();
     }
 
     private static void assertLegacyClustRows(int count, UntypedResultSet rs)
@@ -332,9 +322,9 @@ public class LegacySSTableTest
         }
     }
 
-    private static void loadLegacyTable(String tablePattern, String legacyVersion) throws IOException
+    private static void loadLegacyTable(String tablePattern, String legacyVersion, String compactSuffix) throws IOException
     {
-        String table = String.format(tablePattern, legacyVersion);
+        String table = String.format(tablePattern, legacyVersion, compactSuffix);
 
         logger.info("Loading legacy table {}", table);
 
@@ -349,7 +339,7 @@ public class LegacySSTableTest
     }
 
     /**
-     * Generates sstables for 4 CQL tables (see {@link #createTables(String)}) in <i>current</i>
+     * Generates sstables for 8 CQL tables (see {@link #createTables(String)}) in <i>current</i>
      * sstable format (version) into {@code test/data/legacy-sstables/VERSION}, where
      * {@code VERSION} matches {@link Version#getVersion() BigFormat.latestVersion.getVersion()}.
      * <p>
@@ -357,13 +347,10 @@ public class LegacySSTableTest
      * during development. I.e. remove the {@code @Ignore} annotation temporarily.
      * </p>
      */
-    @Test
     @Ignore
+    @Test
     public void testGenerateSstables() throws Throwable
     {
-        createKeyspace();
-        createTables(BigFormat.latestVersion.getVersion());
-
         Random rand = new Random();
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < 128; i++)
@@ -372,25 +359,28 @@ public class LegacySSTableTest
         }
         String randomString = sb.toString();
 
-        for (int pk = 0; pk < 5; pk++)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            String valPk = Integer.toString(pk);
-            QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple (pk, val) VALUES ('%s', '%s')",
-                                                         BigFormat.latestVersion, valPk, "foo bar baz"));
+            for (int pk = 0; pk < 5; pk++)
+            {
+                String valPk = Integer.toString(pk);
+                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple%s (pk, val) VALUES ('%s', '%s')",
+                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, "foo bar baz"));
 
-            QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter SET val = val + 1 WHERE pk = '%s'",
-                                                         BigFormat.latestVersion, valPk));
+                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter%s SET val = val + 1 WHERE pk = '%s'",
+                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk));
 
-            for (int ck = 0; ck < 50; ck++)
-            {
-                String valCk = Integer.toString(ck);
+                for (int ck = 0; ck < 50; ck++)
+                {
+                    String valCk = Integer.toString(ck);
 
-                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust (pk, ck, val) VALUES ('%s', '%s', '%s')",
-                                                             BigFormat.latestVersion, valPk, valCk + longString, randomString));
+                    QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust%s (pk, ck, val) VALUES ('%s', '%s', '%s')",
+                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString, randomString));
 
-                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
-                                                             BigFormat.latestVersion, valPk, valCk + longString));
+                    QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter%s SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
+                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString));
 
+                }
             }
         }
 
@@ -398,10 +388,13 @@ public class LegacySSTableTest
 
         File ksDir = new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables", BigFormat.latestVersion));
         ksDir.mkdirs();
-        copySstablesFromTestData(String.format("legacy_%s_simple", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_simple_counter", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_clust", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_clust_counter", BigFormat.latestVersion), ksDir);
+        for (int compact = 0; compact <= 1; compact++)
+        {
+            copySstablesFromTestData(String.format("legacy_%s_simple%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_simple_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_clust%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_clust_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+        }
     }
 
     private void copySstablesFromTestData(String table, File ksDir) throws IOException
@@ -420,12 +413,17 @@ public class LegacySSTableTest
 
     private static void copySstablesToTestData(String legacyVersion, String table, File cfDir) throws IOException
     {
-        for (File file : new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table)).listFiles())
+        for (File file : getTableDir(legacyVersion, table).listFiles())
         {
             copyFile(cfDir, file);
         }
     }
 
+    private static File getTableDir(String legacyVersion, String table)
+    {
+        return new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table));
+    }
+
     private static void copyFile(File cfDir, File file) throws IOException
     {
         byte[] buf = new byte[65536];

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index c3a4539..360d262 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -406,7 +406,7 @@ public class BufferedRandomAccessFileTest
             assert tmpFile.getPath().equals(r.getPath());
 
             // Create a mark and move the rw there.
-            final FileMark mark = r.mark();
+            final DataPosition mark = r.mark();
             r.reset(mark);
 
             // Expect this call to succeed.
@@ -457,7 +457,7 @@ public class BufferedRandomAccessFileTest
         RandomAccessReader file = RandomAccessReader.open(channel);
 
         file.seek(10);
-        FileMark mark = file.mark();
+        DataPosition mark = file.mark();
 
         file.seek(file.length());
         assertTrue(file.isEOF());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
index f0d4383..e83c015 100644
--- a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
@@ -356,7 +356,7 @@ public class RandomAccessReaderTest
             assertFalse(reader.isEOF());
             assertEquals((numIterations - 1) * expected.length(), reader.bytesRemaining());
 
-            FileMark mark = reader.mark();
+            DataPosition mark = reader.mark();
             assertEquals(0, reader.bytesPastMark());
             assertEquals(0, reader.bytesPastMark(mark));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
new file mode 100644
index 0000000..175ab53
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
@@ -0,0 +1,539 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class RewindableDataInputStreamPlusTest
+{
+
+    private final int INITIAL_BUFFER_SIZE = 1;
+
+    private File file;
+
+    @Before
+    public void setup() throws Exception
+    {
+        this.file = new File(System.getProperty("java.io.tmpdir"), "subdir/test.buffer");
+    }
+
+    @Test
+    public void testMarkAndResetSimple() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+            // long
+            out.writeLong(1L);
+            // float
+            out.writeFloat(1.0f);
+            // double
+            out.writeDouble(1.0d);
+
+            // String
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+        }
+
+        for (int memCapacity = 0; memCapacity <= 16; memCapacity++)
+        {
+            int diskCapacity = 16 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                try {
+                    //should mark before resetting
+                    reader.reset(null);
+                    fail("Should have thrown IOException");
+                } catch (IOException e) {}
+
+                assertTrue(reader.readBoolean());
+
+                reader.mark();
+
+                try {
+                    //cannot mark already marked stream
+                    reader.mark();
+                    fail("Should have thrown IllegalStateException");
+                } catch (IllegalStateException e) {}
+
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(3, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                try {
+                    //cannot mark when reading from cache
+                    reader.mark();
+                    fail("Should have thrown IllegalStateException");
+                } catch (IllegalStateException e) {}
+
+                //read again previous sequence
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                //finish reading again previous sequence
+                assertEquals(1, reader.readShort());
+
+                reader.mark();
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(16, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                //read again previous sequence
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                assertEquals(1.0f, reader.readFloat(), 0);
+                //finish reading again previous sequence
+
+                //mark again
+                reader.mark();
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals(8, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                //read again previous sequence
+                assertEquals(1.0d, reader.readDouble(), 0);
+                //finish reading again previous sequence
+
+                //mark and reset
+                reader.mark();
+                reader.reset(null);
+
+                assertEquals("abc", reader.readUTF());
+
+                //check max file size
+                assertEquals(diskCapacity, file.length());
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+    @Test
+    public void testVeryLargeCapacity() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+        }
+
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, Integer.MAX_VALUE, file,
+                                                                                      Integer.MAX_VALUE))
+        {
+            reader.mark();
+            assertEquals("abc", reader.readUTF());
+            reader.reset();
+            assertEquals("abc", reader.readUTF());
+        }
+        assertFalse(file.exists());
+
+
+        baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            out.writeBoolean(true);
+            out.writeBoolean(true);
+            testData = baos.toByteArray();
+        }
+    }
+
+    @Test
+    public void testMarkAndResetBigBuffer() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+            // long
+            out.writeLong(1L);
+            // float
+            out.writeFloat(1.0f);
+            // double
+            out.writeDouble(1.0d);
+
+            // String
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+
+            // 1 (boolean) + 1 (byte) + 2 (char) + 2 (short) + 4 (int) + 8 (long)
+            // + 4 (float) + 8 (double) + 5 bytes (utf string)
+        }
+
+        for (int memCapacity = 0; memCapacity <= 18; memCapacity++)
+        {
+            int diskCapacity = 18 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                //read a big amount before resetting
+                reader.mark();
+                assertTrue(reader.readBoolean());
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(1, reader.readShort());
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                reader.reset();
+
+                //read from buffer
+                assertTrue(reader.readBoolean());
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(1, reader.readShort());
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+
+                assertEquals(17, reader.available());
+
+                //mark again
+                reader.mark();
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals("abc", reader.readUTF());
+                reader.reset();
+
+                assertEquals(17, reader.available());
+
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals("abc", reader.readUTF());
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+
+    @Test
+    public void testCircularSpillFile() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+
+            // String
+            out.writeUTF("ab");
+            testData = baos.toByteArray();
+
+            // 1 (boolean) + 1 (byte) + 2 (char) + 2 (short) + 4 (int) + 4 bytes (utf string)
+        }
+
+        //read at most 4 bytes multiple times (and then check file size)
+        int MEM_SIZE = 0;
+        int DISK_SIZE = 4;
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, MEM_SIZE, file,
+                                                                                      DISK_SIZE))
+        {
+            //read 2 bytes and reset
+            reader.mark();
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            assertEquals(2, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            //finish reading again previous sequence
+
+            //read 4 bytes and reset
+            reader.mark();
+            assertEquals('a', reader.readChar());
+            assertEquals(1, reader.readShort());
+            assertEquals(4, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertEquals('a', reader.readChar());
+            assertEquals(1, reader.readShort());
+            //finish reading again previous sequence
+
+            //read 4 bytes and reset
+            reader.mark();
+            assertEquals(1, reader.readInt());
+            assertEquals(4, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertEquals(1, reader.readInt());
+
+            //check max file size
+            assertEquals(DISK_SIZE, file.length());
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testExhaustCapacity() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            testData = baos.toByteArray();
+        }
+
+        //test capacity exhausted when reading more than 4 bytes
+        testCapacityExhausted(testData, 0, 2);
+        testCapacityExhausted(testData, 2, 0);
+        testCapacityExhausted(testData, 1, 1);
+    }
+
+    private void testCapacityExhausted(byte[] testData, int memSize, int diskSize) throws IOException
+    {
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, memSize, file,
+                                                                                      diskSize))
+        {
+            //read 2 bytes and reset
+            reader.mark();
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            assertEquals(2, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            //finish reading again previous sequence
+
+            reader.mark();
+            //read 3 bytes - OK
+            assertEquals('a', reader.readChar());
+            //read 1 more bytes - CAPACITY will exhaust when trying to reset :(
+            assertEquals(1, reader.readShort());
+
+            try
+            {
+                reader.reset();
+                fail("Should have thrown IOException");
+            }
+            catch (IOException e) {}
+
+            //check max file size
+            assertEquals(diskSize, file.length());
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testMarkAndResetUnsignedRead() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // byte
+            out.writeByte(0x1);
+            // short
+            out.writeShort(2);
+            testData = baos.toByteArray();
+        }
+
+        for (int memCapacity = 0; memCapacity <= 1; memCapacity++)
+        {
+            int diskCapacity = 1 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                assertEquals(1, reader.readUnsignedByte());
+                reader.reset();
+                assertEquals(1, reader.readUnsignedByte());
+
+                //will read first byte of short 2
+                reader.mark();
+                assertEquals(0, reader.readUnsignedByte());
+                reader.reset();
+
+                assertEquals(2, reader.readUnsignedShort());
+
+                reader.mark();
+                reader.reset();
+                assertEquals(0, reader.available());
+            }
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testMarkAndResetSkipBytes() throws Exception
+    {
+        String testStr = "1234567890";
+        byte[] testData = testStr.getBytes();
+
+        for (int memCapacity = 0; memCapacity <= 7; memCapacity++)
+        {
+            int diskCapacity = 7 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                // read first 5 bytes and rewind
+                byte[] out = new byte[5];
+                reader.readFully(out, 0, 5);
+                assertEquals("12345", new String(out));
+
+                // then skip 2 bytes (67)
+                reader.skipBytes(2);
+
+                assertEquals(7, reader.bytesPastMark(null));
+                reader.reset();
+
+                //now read part of the previously skipped bytes
+                out = new byte[5];
+                reader.readFully(out);
+                assertEquals("12345", new String(out));
+
+                //skip 3 bytes (2 from cache, 1 from stream)
+                reader.skip(3);
+
+                // mark and read 2 more bytes
+                reader.mark();
+                out = new byte[2];
+                reader.readFully(out);
+                assertEquals("90", new String(out));
+                assertEquals(0, reader.available());
+                reader.reset();
+
+                //reset and read only the next byte "9" in the third position
+                reader.readFully(out, 1, 1);
+                assertEquals("99", new String(out));
+
+                //now we read the remainder via readline
+                assertEquals(1, reader.available());
+                assertEquals("0", reader.readLine());
+
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+    @Test
+    public void testMarkAndResetReadFully() throws Exception
+    {
+        String testStr = "1234567890";
+        byte[] testData = testStr.getBytes();
+
+        for (int memCapacity = 0; memCapacity <= 5; memCapacity++)
+        {
+            int diskCapacity = 5 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                // read first 5 bytes and rewind
+                byte[] out = new byte[5];
+                reader.readFully(out, 0, 5);
+                assertEquals("12345", new String(out));
+                reader.reset();
+
+                // read half from cache, half from parent stream
+                out = new byte[7];
+                reader.readFully(out);
+                assertEquals("1234567", new String(out));
+
+                // mark and read 3 more bytes
+                reader.mark();
+                out = new byte[3];
+                reader.readFully(out);
+                assertEquals("890", new String(out));
+                assertEquals(0, reader.available());
+                reader.reset();
+
+                //reset and read only the next byte "8" in the third position
+                reader.readFully(out, 2, 1);
+                assertEquals("898", new String(out));
+
+                //now we read the remainder via readline
+                assertEquals(2, reader.available());
+                assertEquals("90", reader.readLine());
+            }
+            assertFalse(file.exists());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java b/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
index 221e55c..7693b45 100644
--- a/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
+++ b/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
@@ -18,6 +18,10 @@
  */
 package org.apache.cassandra.utils;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -25,8 +29,10 @@ import java.io.DataOutputStream;
 
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.apache.cassandra.io.util.BytesReadTracker;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
+import org.apache.cassandra.io.util.TrackedInputStream;
 
 public class BytesReadTrackerTest
 {
@@ -34,6 +40,33 @@ public class BytesReadTrackerTest
     @Test
     public void testBytesRead() throws Exception
     {
+        internalTestBytesRead(true);
+        internalTestBytesRead(false);
+    }
+
+    @Test
+    public void testUnsignedRead() throws Exception
+    {
+        internalTestUnsignedRead(true);
+        internalTestUnsignedRead(false);
+    }
+
+    @Test
+    public void testSkipBytesAndReadFully() throws Exception
+    {
+        internalTestSkipBytesAndReadFully(true);
+        internalTestSkipBytesAndReadFully(false);
+    }
+
+    @Test
+    public void testReadLine() throws Exception
+    {
+        internalTestReadLine(true);
+        internalTestReadLine(false);
+    }
+
+    public void internalTestBytesRead(boolean inputStream) throws Exception
+    {
         byte[] testData;
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -66,45 +99,46 @@ public class BytesReadTrackerTest
             out.close();
         }
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // boolean = 1byte
-            boolean bool = tracker.readBoolean();
+            boolean bool = reader.readBoolean();
             assertTrue(bool);
             assertEquals(1, tracker.getBytesRead());
             // byte = 1byte
-            byte b = tracker.readByte();
+            byte b = reader.readByte();
             assertEquals(b, 0x1);
             assertEquals(2, tracker.getBytesRead());
             // char = 2byte
-            char c = tracker.readChar();
+            char c = reader.readChar();
             assertEquals('a', c);
             assertEquals(4, tracker.getBytesRead());
             // short = 2bytes
-            short s = tracker.readShort();
+            short s = reader.readShort();
             assertEquals(1, s);
             assertEquals((short) 6, tracker.getBytesRead());
             // int = 4bytes
-            int i = tracker.readInt();
+            int i = reader.readInt();
             assertEquals(1, i);
             assertEquals(10, tracker.getBytesRead());
             // long = 8bytes
-            long l = tracker.readLong();
+            long l = reader.readLong();
             assertEquals(1L, l);
             assertEquals(18, tracker.getBytesRead());
             // float = 4bytes
-            float f = tracker.readFloat();
+            float f = reader.readFloat();
             assertEquals(1.0f, f, 0);
             assertEquals(22, tracker.getBytesRead());
             // double = 8bytes
-            double d = tracker.readDouble();
+            double d = reader.readDouble();
             assertEquals(1.0d, d, 0);
             assertEquals(30, tracker.getBytesRead());
             // String("abc") = 2(string size) + 3 = 5 bytes
-            String str = tracker.readUTF();
+            String str = reader.readUTF();
             assertEquals("abc", str);
             assertEquals(35, tracker.getBytesRead());
 
@@ -119,8 +153,7 @@ public class BytesReadTrackerTest
         assertEquals(0, tracker.getBytesRead());
     }
 
-    @Test
-    public void testUnsignedRead() throws Exception
+    public void internalTestUnsignedRead(boolean inputStream) throws Exception
     {
         byte[] testData;
 
@@ -139,17 +172,18 @@ public class BytesReadTrackerTest
             out.close();
         }
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // byte = 1byte
-            int b = tracker.readUnsignedByte();
+            int b = reader.readUnsignedByte();
             assertEquals(b, 1);
             assertEquals(1, tracker.getBytesRead());
             // short = 2bytes
-            int s = tracker.readUnsignedShort();
+            int s = reader.readUnsignedShort();
             assertEquals(1, s);
             assertEquals(3, tracker.getBytesRead());
 
@@ -161,30 +195,30 @@ public class BytesReadTrackerTest
         }
     }
 
-    @Test
-    public void testSkipBytesAndReadFully() throws Exception
+    public void internalTestSkipBytesAndReadFully(boolean inputStream) throws Exception
     {
         String testStr = "1234567890";
         byte[] testData = testStr.getBytes();
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // read first 5 bytes
             byte[] out = new byte[5];
-            tracker.readFully(out, 0, 5);
+            reader.readFully(out, 0, 5);
             assertEquals("12345", new String(out));
             assertEquals(5, tracker.getBytesRead());
 
             // then skip 2 bytes
-            tracker.skipBytes(2);
+            reader.skipBytes(2);
             assertEquals(7, tracker.getBytesRead());
 
             // and read the rest
             out = new byte[3];
-            tracker.readFully(out);
+            reader.readFully(out);
             assertEquals("890", new String(out));
             assertEquals(10, tracker.getBytesRead());
 
@@ -196,16 +230,24 @@ public class BytesReadTrackerTest
         }
     }
 
-    @Test(expected = UnsupportedOperationException.class)
-    public void testReadLine() throws Exception
+    public void internalTestReadLine(boolean inputStream) throws Exception
     {
         DataInputStream in = new DataInputStream(new ByteArrayInputStream("1".getBytes()));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
-            // throws UnsupportedOperationException
-            tracker.readLine();
+            String line = reader.readLine();
+            if (inputStream)
+                assertEquals(line, "1");
+            else
+                fail("Should have thrown UnsupportedOperationException");
+        }
+        catch (UnsupportedOperationException e)
+        {
+            if (inputStream)
+                fail("Should have not thrown UnsupportedOperationException");
         }
         finally
         {


[05/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/utils/BytesReadTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BytesReadTracker.java b/src/java/org/apache/cassandra/utils/BytesReadTracker.java
deleted file mode 100644
index 5e98b25..0000000
--- a/src/java/org/apache/cassandra/utils/BytesReadTracker.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.utils;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-
-/**
- * This class is to track bytes read from given DataInput
- */
-public class BytesReadTracker implements DataInputPlus
-{
-
-    private long bytesRead;
-    final DataInput source;
-
-    public BytesReadTracker(DataInput source)
-    {
-        this.source = source;
-    }
-
-    public long getBytesRead()
-    {
-        return bytesRead;
-    }
-
-    /**
-     * reset counter to @param count
-     */
-    public void reset(long count)
-    {
-        bytesRead = count;
-    }
-
-    public boolean readBoolean() throws IOException
-    {
-        boolean bool = source.readBoolean();
-        bytesRead += 1;
-        return bool;
-    }
-
-    public byte readByte() throws IOException
-    {
-        byte b = source.readByte();
-        bytesRead += 1;
-        return b;
-    }
-
-    public char readChar() throws IOException
-    {
-        char c = source.readChar();
-        bytesRead += 2;
-        return c;
-    }
-
-    public double readDouble() throws IOException
-    {
-        double d = source.readDouble();
-        bytesRead += 8;
-        return d;
-    }
-
-    public float readFloat() throws IOException
-    {
-        float f = source.readFloat();
-        bytesRead += 4;
-        return f;
-    }
-
-    public void readFully(byte[] b, int off, int len) throws IOException
-    {
-        source.readFully(b, off, len);
-        bytesRead += len;
-    }
-
-    public void readFully(byte[] b) throws IOException
-    {
-        source.readFully(b);
-        bytesRead += b.length;
-    }
-
-    public int readInt() throws IOException
-    {
-        int i = source.readInt();
-        bytesRead += 4;
-        return i;
-    }
-
-    public String readLine() throws IOException
-    {
-        // since this method is deprecated and cannot track bytes read
-        // just throw exception
-        throw new UnsupportedOperationException();
-    }
-
-    public long readLong() throws IOException
-    {
-        long l = source.readLong();
-        bytesRead += 8;
-        return l;
-    }
-
-    public short readShort() throws IOException
-    {
-        short s = source.readShort();
-        bytesRead += 2;
-        return s;
-    }
-
-    public String readUTF() throws IOException
-    {
-        return DataInputStream.readUTF(this);
-    }
-
-    public int readUnsignedByte() throws IOException
-    {
-        int i = source.readUnsignedByte();
-        bytesRead += 1;
-        return i;
-    }
-
-    public int readUnsignedShort() throws IOException
-    {
-        int i = source.readUnsignedShort();
-        bytesRead += 2;
-        return i;
-    }
-
-    public int skipBytes(int n) throws IOException
-    {
-        int skipped = source.skipBytes(n);
-        bytesRead += skipped;
-        return skipped;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/utils/CloseableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CloseableIterator.java b/src/java/org/apache/cassandra/utils/CloseableIterator.java
index a7c4300..57034ae 100644
--- a/src/java/org/apache/cassandra/utils/CloseableIterator.java
+++ b/src/java/org/apache/cassandra/utils/CloseableIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.Closeable;
 import java.util.Iterator;
 
 // so we can instantiate anonymous classes implementing both interfaces

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..5eddda7
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db
new file mode 100644
index 0000000..61ef270
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..ab83acc
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db
new file mode 100644
index 0000000..896a529
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..34d459d
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db
new file mode 100644
index 0000000..b511d30
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..aa3c757
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db
new file mode 100644
index 0000000..896a529
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db
new file mode 100644
index 0000000..c7e8586
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..792e733
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db
new file mode 100644
index 0000000..af5e781
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db
new file mode 100644
index 0000000..f545b04
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..8657050
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db
new file mode 100644
index 0000000..af5e781
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..654094e
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db
new file mode 100644
index 0000000..4c87e07
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..4690757
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1331331706
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..ab55258
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db
new file mode 100644
index 0000000..774cbd1
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..e3b71a4
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db
new file mode 100644
index 0000000..90d42a5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..52e6552
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2793875907
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..8360ed5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db
new file mode 100644
index 0000000..774cbd1
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db
new file mode 100644
index 0000000..6a38c52
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..be8e5fb
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+606280675
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..e3fd855
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db
new file mode 100644
index 0000000..af8ad8b
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db
new file mode 100644
index 0000000..5f4a7db
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..a71f766
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+616768162
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..4a6e940
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db
new file mode 100644
index 0000000..af8ad8b
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2a72f70
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db
new file mode 100644
index 0000000..6bc08d2
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..943dd1e
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+1372047449
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..b2fd408
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..d4dec70
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db
new file mode 100644
index 0000000..63ee721
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..577407e
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2583914481
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..2bfc59d
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db
new file mode 100644
index 0000000..2e912a1
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..c07a57f
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+278403976
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..a81e03e
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db
new file mode 100644
index 0000000..323ff37
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..92237e7
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2048991053
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..37324a7
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..11e69f9
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..9af7ba2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..46e277c
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+231958969
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..10fce6d
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..70a08ed
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2c92e35
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..f8dadef
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..b7dbadc
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3570926375
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..5697f6f
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..45dbcfe
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..adb7fc4
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..69aec90
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..da73bd8
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3668325305
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..56f29df
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..d4b841a
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..56c95a8
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..b30b1c4
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..18ba1ff
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3769133549
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..d094f73
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..b81ea33
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db differ


[03/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
Support streaming of older version sstables in 3.0

patch by Paulo Motta; reviewed by Yuki Morishita for CASSANDRA-10990


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e8651b66
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e8651b66
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e8651b66

Branch: refs/heads/cassandra-3.0
Commit: e8651b6625c7f6260852f2a9c45fb189c63ab528
Parents: 7f1339c
Author: Paulo Motta <pa...@gmail.com>
Authored: Fri Feb 5 12:38:39 2016 -0300
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 17 10:04:44 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   2 +
 .../org/apache/cassandra/db/Directories.java    |  30 +
 .../cassandra/db/SerializationHeader.java       |   5 +
 .../org/apache/cassandra/db/Serializers.java    | 114 ++--
 .../columniterator/AbstractSSTableIterator.java |   4 +-
 .../cassandra/hints/ChecksummedDataInput.java   |   8 +-
 .../org/apache/cassandra/hints/HintMessage.java |   4 +-
 .../io/compress/CompressedSequentialWriter.java |   8 +-
 .../io/sstable/SSTableSimpleIterator.java       |  11 +-
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../io/sstable/format/SSTableWriter.java        |   2 +-
 .../io/sstable/format/big/BigTableWriter.java   |   4 +-
 .../cassandra/io/util/BytesReadTracker.java     |  30 +
 .../apache/cassandra/io/util/DataPosition.java  |  21 +
 .../apache/cassandra/io/util/FileDataInput.java |   8 +-
 .../org/apache/cassandra/io/util/FileMark.java  |  20 -
 .../io/util/FileSegmentInputStream.java         |   6 +-
 .../cassandra/io/util/RandomAccessReader.java   |   8 +-
 .../cassandra/io/util/RewindableDataInput.java  |  30 +
 .../io/util/RewindableDataInputStreamPlus.java  | 569 +++++++++++++++++++
 .../cassandra/io/util/SequentialWriter.java     |   6 +-
 .../cassandra/io/util/TrackedDataInputPlus.java | 150 +++++
 .../cassandra/io/util/TrackedInputStream.java   |  76 +++
 .../cassandra/service/StorageService.java       |   1 +
 .../cassandra/streaming/StreamReader.java       |  84 ++-
 .../compress/CompressedStreamReader.java        |  18 +-
 .../streaming/messages/FileMessageHeader.java   |   4 +-
 .../apache/cassandra/tools/nodetool/Repair.java |   2 +-
 .../cassandra/utils/BytesReadTracker.java       | 153 -----
 .../cassandra/utils/CloseableIterator.java      |   1 -
 ...acy_jb_clust_compact-jb-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-Data.db | Bin 0 -> 5270 bytes
 ...ables-legacy_jb_clust_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_jb_clust_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_jb_clust_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...bles-legacy_jb_clust_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-TOC.txt |   7 +
 ...lust_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_jb_clust_counter_compact-jb-1-Data.db | Bin 0 -> 4228 bytes
 ...gacy_jb_clust_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_jb_clust_counter_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ..._jb_clust_counter_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...acy_jb_clust_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...legacy_jb_clust_counter_compact-jb-1-TOC.txt |   7 +
 ...cy_jb_simple_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_jb_simple_compact-jb-1-Data.db | Bin 0 -> 108 bytes
 ...bles-legacy_jb_simple_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_jb_simple_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_jb_simple_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...les-legacy_jb_simple_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...tables-legacy_jb_simple_compact-jb-1-TOC.txt |   7 +
 ...mple_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_jb_simple_counter_compact-jb-1-Data.db | Bin 0 -> 118 bytes
 ...acy_jb_simple_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_jb_simple_counter_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...jb_simple_counter_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...cy_jb_simple_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...egacy_jb_simple_counter_compact-jb-1-TOC.txt |   7 +
 ...acy_ka_clust_compact-ka-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-Data.db | Bin 0 -> 5277 bytes
 ...les-legacy_ka_clust_compact-ka-1-Digest.sha1 |   1 +
 ...ables-legacy_ka_clust_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_ka_clust_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_ka_clust_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...bles-legacy_ka_clust_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-TOC.txt |   8 +
 ...lust_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_ka_clust_counter_compact-ka-1-Data.db | Bin 0 -> 4527 bytes
 ...cy_ka_clust_counter_compact-ka-1-Digest.sha1 |   1 +
 ...gacy_ka_clust_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_ka_clust_counter_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ..._ka_clust_counter_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...acy_ka_clust_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...legacy_ka_clust_counter_compact-ka-1-TOC.txt |   8 +
 ...cy_ka_simple_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_ka_simple_compact-ka-1-Data.db | Bin 0 -> 105 bytes
 ...es-legacy_ka_simple_compact-ka-1-Digest.sha1 |   1 +
 ...bles-legacy_ka_simple_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_ka_simple_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_ka_simple_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...les-legacy_ka_simple_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...tables-legacy_ka_simple_compact-ka-1-TOC.txt |   8 +
 ...mple_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_ka_simple_counter_compact-ka-1-Data.db | Bin 0 -> 124 bytes
 ...y_ka_simple_counter_compact-ka-1-Digest.sha1 |   1 +
 ...acy_ka_simple_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_ka_simple_counter_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...ka_simple_counter_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...cy_ka_simple_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...egacy_ka_simple_counter_compact-ka-1-TOC.txt |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_la_clust_compact/la-1-big-Data.db    | Bin 0 -> 5286 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_clust_compact/la-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_la_clust_compact/la-1-big-Index.db   | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../legacy_la_clust_compact/la-1-big-Summary.db | Bin 0 -> 75 bytes
 .../legacy_la_clust_compact/la-1-big-TOC.txt    |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 4527 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_la_simple_compact/la-1-big-Data.db   | Bin 0 -> 106 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_simple_compact/la-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_la_simple_compact/la-1-big-Index.db  | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../legacy_la_simple_compact/la-1-big-TOC.txt   |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 124 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_ma_clust_compact/ma-1-big-Data.db    | Bin 0 -> 5393 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_clust_compact/ma-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_ma_clust_compact/ma-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7046 bytes
 .../legacy_ma_clust_compact/ma-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_ma_clust_compact/ma-1-big-TOC.txt    |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 4606 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7055 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_ma_simple_compact/ma-1-big-Data.db   | Bin 0 -> 91 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_simple_compact/ma-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_ma_simple_compact/ma-1-big-Index.db  | Bin 0 -> 26 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4640 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_ma_simple_compact/ma-1-big-TOC.txt   |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4649 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../cassandra/AbstractSerializationsTester.java |   1 -
 .../apache/cassandra/db/DirectoriesTest.java    |  98 ++--
 .../cassandra/gms/SerializationsTest.java       |   1 -
 .../CompressedRandomAccessReaderTest.java       |   6 +-
 .../CompressedSequentialWriterTest.java         |   4 +-
 .../cassandra/io/sstable/LegacySSTableTest.java | 368 ++++++------
 .../io/util/BufferedRandomAccessFileTest.java   |   4 +-
 .../io/util/RandomAccessReaderTest.java         |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java | 539 ++++++++++++++++++
 .../cassandra/utils/BytesReadTrackerTest.java   | 104 +++-
 165 files changed, 2101 insertions(+), 544 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index defc25a..51cfc16 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.5
+ * Support streaming pre-3.0 sstables (CASSANDRA-10990)
  * Add backpressure to compressed commit log (CASSANDRA-10971)
  * SSTableExport supports secondary index tables (CASSANDRA-11330)
  * Fix sstabledump to include missing info in debug output (CASSANDRA-11321)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c564d8d..f28df1c 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -571,6 +571,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
          // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
         clearEphemeralSnapshots(directories);
 
+        directories.removeTemporaryDirectories();
+
         logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.cfName);
         LifecycleTransaction.removeUnfinishedLeftovers(metadata);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 8744d43..83321ac 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -94,9 +94,11 @@ public class Directories
 
     public static final String BACKUPS_SUBDIR = "backups";
     public static final String SNAPSHOT_SUBDIR = "snapshots";
+    public static final String TMP_SUBDIR = "tmp";
     public static final String SECONDARY_INDEX_NAME_SEPARATOR = ".";
 
     public static final DataDirectory[] dataDirectories;
+
     static
     {
         String[] locations = DatabaseDescriptor.getAllDataFileLocations();
@@ -322,6 +324,34 @@ public class Directories
     }
 
     /**
+     * Returns a temporary subdirectory on non-blacklisted data directory
+     * that _currently_ has {@code writeSize} bytes as usable space.
+     * This method does not create the temporary directory.
+     *
+     * @throws IOError if all directories are blacklisted.
+     */
+    public File getTemporaryWriteableDirectoryAsFile(long writeSize)
+    {
+        File location = getLocationForDisk(getWriteableLocation(writeSize));
+        if (location == null)
+            return null;
+        return new File(location, TMP_SUBDIR);
+    }
+
+    public void removeTemporaryDirectories()
+    {
+        for (File dataDir : dataPaths)
+        {
+            File tmpDir = new File(dataDir, TMP_SUBDIR);
+            if (tmpDir.exists())
+            {
+                logger.debug("Removing temporary directory {}", tmpDir);
+                FileUtils.deleteRecursive(tmpDir);
+            }
+        }
+    }
+
+    /**
      * Returns a non-blacklisted data directory that _currently_ has {@code writeSize} bytes as usable space.
      *
      * @throws IOError if all directories are blacklisted.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/SerializationHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index 6e03756..0fd1281 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -70,6 +70,11 @@ public class SerializationHeader
         this.typeMap = typeMap;
     }
 
+    public static SerializationHeader makeWithoutStats(CFMetaData metadata)
+    {
+        return new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
+    }
+
     public static SerializationHeader forKeyCache(CFMetaData metadata)
     {
         // We don't save type information in the key cache (we could change

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/Serializers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Serializers.java b/src/java/org/apache/cassandra/db/Serializers.java
index 9b29d89..348fda3 100644
--- a/src/java/org/apache/cassandra/db/Serializers.java
+++ b/src/java/org/apache/cassandra/db/Serializers.java
@@ -29,8 +29,6 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
-
 /**
  * Holds references on serializers that depend on the table definition.
  */
@@ -48,62 +46,77 @@ public class Serializers
     // unecessary (since IndexInfo.Serializer won't depend on the metadata either).
     public ISerializer<ClusteringPrefix> indexEntryClusteringPrefixSerializer(final Version version, final SerializationHeader header)
     {
-        if (!version.storeRows())
+        if (!version.storeRows() || header ==  null) //null header indicates streaming from pre-3.0 sstables
         {
-            return new ISerializer<ClusteringPrefix>()
+            return oldFormatSerializer(version);
+        }
+
+        return newFormatSerializer(version, header);
+    }
+
+    private ISerializer<ClusteringPrefix> oldFormatSerializer(final Version version)
+    {
+        return new ISerializer<ClusteringPrefix>()
+        {
+            SerializationHeader newHeader = SerializationHeader.makeWithoutStats(metadata);
+
+            public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
             {
-                public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
-                {
-                    // We should only use this for reading old sstable, never write new ones.
-                    throw new UnsupportedOperationException();
-                }
+                //we deserialize in the old format and serialize in the new format
+                ClusteringPrefix.serializer.serialize(clustering, out,
+                                                      version.correspondingMessagingVersion(),
+                                                      newHeader.clusteringTypes());
+            }
+
+            public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+            {
+                // We're reading the old cellname/composite
+                ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
+                assert bb.hasRemaining(); // empty cellnames were invalid
+
+                int clusteringSize = metadata.clusteringColumns().size();
+                // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
+                if (clusteringSize == 0)
+                    return Clustering.EMPTY;
 
-                public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+                if (!metadata.isCompound())
+                    return new Clustering(bb);
+
+                List<ByteBuffer> components = CompositeType.splitName(bb);
+                byte eoc = CompositeType.lastEOC(bb);
+
+                if (eoc == 0 || components.size() >= clusteringSize)
                 {
-                    // We're reading the old cellname/composite
-                    ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
-                    assert bb.hasRemaining(); // empty cellnames were invalid
-
-                    int clusteringSize = metadata.clusteringColumns().size();
-                    // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
-                    if (clusteringSize == 0)
-                        return Clustering.EMPTY;
-
-                    if (!metadata.isCompound())
-                        return new Clustering(bb);
-
-                    List<ByteBuffer> components = CompositeType.splitName(bb);
-                    byte eoc = CompositeType.lastEOC(bb);
-
-                    if (eoc == 0 || components.size() >= clusteringSize)
-                    {
-                        // That's a clustering.
-                        if (components.size() > clusteringSize)
-                            components = components.subList(0, clusteringSize);
-
-                        return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
-                    }
-                    else
-                    {
-                        // It's a range tombstone bound. It is a start since that's the only part we've ever included
-                        // in the index entries.
-                        Slice.Bound.Kind boundKind = eoc > 0
-                                                   ? Slice.Bound.Kind.EXCL_START_BOUND
-                                                   : Slice.Bound.Kind.INCL_START_BOUND;
-
-                        return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
-                    }
-                }
+                    // That's a clustering.
+                    if (components.size() > clusteringSize)
+                        components = components.subList(0, clusteringSize);
 
-                public long serializedSize(ClusteringPrefix clustering)
+                    return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
+                }
+                else
                 {
-                    // We should only use this for reading old sstable, never write new ones.
-                    throw new UnsupportedOperationException();
+                    // It's a range tombstone bound. It is a start since that's the only part we've ever included
+                    // in the index entries.
+                    Slice.Bound.Kind boundKind = eoc > 0
+                                                 ? Slice.Bound.Kind.EXCL_START_BOUND
+                                                 : Slice.Bound.Kind.INCL_START_BOUND;
+
+                    return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
                 }
-            };
-        }
+            }
 
-        return new ISerializer<ClusteringPrefix>()
+            public long serializedSize(ClusteringPrefix clustering)
+            {
+                return ClusteringPrefix.serializer.serializedSize(clustering, version.correspondingMessagingVersion(),
+                                                                  newHeader.clusteringTypes());
+            }
+        };
+    }
+
+
+    private ISerializer<ClusteringPrefix> newFormatSerializer(final Version version, final SerializationHeader header)
+    {
+        return new ISerializer<ClusteringPrefix>() //Reading and writing from/to the new sstable format
         {
             public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
             {
@@ -121,4 +134,5 @@ public class Serializers
             }
         };
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index 8ac3dcb..0e2012e 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
@@ -401,7 +401,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         private int currentIndexIdx;
 
         // Marks the beginning of the block corresponding to currentIndexIdx.
-        private FileMark mark;
+        private DataPosition mark;
 
         public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
index 1dc6d1e..095d7f4 100644
--- a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
+++ b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
@@ -22,13 +22,9 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.zip.CRC32;
 
-import org.apache.cassandra.io.FSError;
-import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.compress.ICompressor;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.schema.CompressionParams;
 
 /**
  * A {@link RandomAccessReader} wrapper that calctulates the CRC in place.
@@ -48,7 +44,7 @@ public class ChecksummedDataInput extends RandomAccessReader.RandomAccessReaderW
     private boolean crcUpdateDisabled;
 
     private long limit;
-    private FileMark limitMark;
+    private DataPosition limitMark;
 
     protected ChecksummedDataInput(Builder builder)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/hints/HintMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintMessage.java b/src/java/org/apache/cassandra/hints/HintMessage.java
index e78738d..723ab6d 100644
--- a/src/java/org/apache/cassandra/hints/HintMessage.java
+++ b/src/java/org/apache/cassandra/hints/HintMessage.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 /**
@@ -117,7 +117,7 @@ public final class HintMessage
             UUID hostId = UUIDSerializer.serializer.deserialize(in, version);
 
             long hintSize = in.readUnsignedVInt();
-            BytesReadTracker countingIn = new BytesReadTracker(in);
+            TrackedDataInputPlus countingIn = new TrackedDataInputPlus(in);
             try
             {
                 return new HintMessage(hostId, Hint.serializer.deserialize(countingIn, version));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 14f1ba7..9bd1145 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.SequentialWriter;
 import org.apache.cassandra.schema.CompressionParams;
@@ -153,7 +153,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     }
 
     @Override
-    public FileMark mark()
+    public DataPosition mark()
     {
         if (!buffer.hasRemaining())
             doFlush(0);
@@ -161,7 +161,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     }
 
     @Override
-    public synchronized void resetAndTruncate(FileMark mark)
+    public synchronized void resetAndTruncate(DataPosition mark)
     {
         assert mark instanceof CompressedFileWriterMark;
 
@@ -306,7 +306,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class CompressedFileWriterMark implements FileMark
+    protected static class CompressedFileWriterMark implements DataPosition
     {
         // chunk offset in the compressed file
         final long chunkOffset;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
index 365d469..f82db4e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.IOError;
 import java.util.Iterator;
 
+import org.apache.cassandra.io.util.RewindableDataInput;
 import org.apache.cassandra.utils.AbstractIterator;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -29,7 +30,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.net.MessagingService;
 
 /**
@@ -113,11 +114,9 @@ public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered>
                 // need to extract them. Which imply 2 passes (one to extract the static, then one for other value).
                 if (metadata.isStaticCompactTable())
                 {
-                    // Because we don't support streaming from old file version, the only case we should get there is for compaction,
-                    // where the DataInput should be a file based one.
-                    assert in instanceof FileDataInput;
-                    FileDataInput file = (FileDataInput)in;
-                    FileMark mark = file.mark();
+                    assert in instanceof RewindableDataInput;
+                    RewindableDataInput file = (RewindableDataInput)in;
+                    DataPosition mark = file.mark();
                     Row staticRow = LegacyLayout.extractStaticColumns(metadata, file, metadata.partitionColumns().statics);
                     file.reset(mark);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 8a778b7..b9561ec 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -427,7 +427,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                                              System.currentTimeMillis(),
                                              statsMetadata,
                                              OpenReason.NORMAL,
-                                             header.toHeader(metadata));
+                                             header == null? null : header.toHeader(metadata));
 
         // special implementation of load to use non-pooled SegmentedFile builders
         try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 4cbbd70..5f35029 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -80,7 +80,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
         this.keyCount = keyCount;
         this.repairedAt = repairedAt;
         this.metadataCollector = metadataCollector;
-        this.header = header;
+        this.header = header != null ? header : SerializationHeader.makeWithoutStats(metadata); //null header indicates streaming from pre-3.0 sstable
         this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, descriptor.version, header);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index 2335e47..d3630d7 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -55,7 +55,7 @@ public class BigTableWriter extends SSTableWriter
     private final SegmentedFile.Builder dbuilder;
     protected final SequentialWriter dataFile;
     private DecoratedKey lastWrittenKey;
-    private FileMark dataMark;
+    private DataPosition dataMark;
 
     public BigTableWriter(Descriptor descriptor, 
                           Long keyCount, 
@@ -368,7 +368,7 @@ public class BigTableWriter extends SSTableWriter
         public final SegmentedFile.Builder builder;
         public final IndexSummaryBuilder summary;
         public final IFilter bf;
-        private FileMark mark;
+        private DataPosition mark;
 
         IndexWriter(long keyCount, final SequentialWriter dataFile)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BytesReadTracker.java b/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
new file mode 100644
index 0000000..fc83856
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+public interface BytesReadTracker
+{
+    public long getBytesRead();
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count);
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/DataPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/DataPosition.java b/src/java/org/apache/cassandra/io/util/DataPosition.java
new file mode 100644
index 0000000..e106dae
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/DataPosition.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+public interface DataPosition
+{}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileDataInput.java b/src/java/org/apache/cassandra/io/util/FileDataInput.java
index f56193b..1059b01 100644
--- a/src/java/org/apache/cassandra/io/util/FileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/FileDataInput.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.io.util;
 import java.io.Closeable;
 import java.io.IOException;
 
-public interface FileDataInput extends DataInputPlus, Closeable
+public interface FileDataInput extends RewindableDataInput, Closeable
 {
     String getPath();
 
@@ -30,11 +30,5 @@ public interface FileDataInput extends DataInputPlus, Closeable
 
     void seek(long pos) throws IOException;
 
-    FileMark mark();
-
-    void reset(FileMark mark) throws IOException;
-
-    long bytesPastMark(FileMark mark);
-
     long getFilePointer();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileMark.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileMark.java b/src/java/org/apache/cassandra/io/util/FileMark.java
deleted file mode 100644
index 781bc1e..0000000
--- a/src/java/org/apache/cassandra/io/util/FileMark.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.io.util;
-
-public interface FileMark {}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java b/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
index 425c7d6..a585215 100644
--- a/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
+++ b/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
@@ -74,17 +74,17 @@ public class FileSegmentInputStream extends DataInputBuffer implements FileDataI
         return false;
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         throw new UnsupportedOperationException();
     }
 
-    public void reset(FileMark mark)
+    public void reset(DataPosition mark)
     {
         throw new UnsupportedOperationException();
     }
 
-    public long bytesPastMark(FileMark mark)
+    public long bytesPastMark(DataPosition mark)
     {
         return 0;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index b495bf0..1943773 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -200,19 +200,19 @@ public class RandomAccessReader extends RebufferingInputStream implements FileDa
         return bytes;
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         markedPointer = current();
         return new BufferedRandomAccessFileMark(markedPointer);
     }
 
-    public void reset(FileMark mark)
+    public void reset(DataPosition mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
         seek(((BufferedRandomAccessFileMark) mark).pointer);
     }
 
-    public long bytesPastMark(FileMark mark)
+    public long bytesPastMark(DataPosition mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
         long bytes = current() - ((BufferedRandomAccessFileMark) mark).pointer;
@@ -262,7 +262,7 @@ public class RandomAccessReader extends RebufferingInputStream implements FileDa
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class BufferedRandomAccessFileMark implements FileMark
+    protected static class BufferedRandomAccessFileMark implements DataPosition
     {
         final long pointer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
new file mode 100644
index 0000000..c202f60
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.IOException;
+
+public interface RewindableDataInput extends DataInputPlus
+{
+    DataPosition mark();
+
+    void reset(DataPosition mark) throws IOException;
+
+    long bytesPastMark(DataPosition mark);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
new file mode 100644
index 0000000..3a680f4
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
@@ -0,0 +1,569 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * Adds mark/reset functionality to another input stream by caching read bytes to a memory buffer and
+ * spilling to disk if necessary.
+ *
+ * When the stream is marked via {@link this#mark()} or {@link this#mark(int)}, up to
+ * <code>maxMemBufferSize</code> will be cached in memory (heap). If more than
+ * <code>maxMemBufferSize</code> bytes are read while the stream is marked, the
+ * following bytes are cached on the <code>spillFile</code> for up to <code>maxDiskBufferSize</code>.
+ *
+ * Please note that successive calls to {@link this#mark()} and {@link this#reset()} will write
+ * sequentially to the same <code>spillFile</code> until <code>maxDiskBufferSize</code> is reached.
+ * At this point, if less than <code>maxDiskBufferSize</code> bytes are currently cached on the
+ * <code>spillFile</code>, the remaining bytes are written to the beginning of the file,
+ * treating the <code>spillFile</code> as a circular buffer.
+ *
+ * If more than <code>maxMemBufferSize + maxDiskBufferSize</code> are cached while the stream is marked,
+ * the following {@link this#reset()} invocation will throw a {@link IllegalStateException}.
+ *
+ */
+public class RewindableDataInputStreamPlus extends FilterInputStream implements RewindableDataInput, Closeable
+{
+    private boolean marked = false;
+    private boolean exhausted = false;
+    private AtomicBoolean closed = new AtomicBoolean(false);
+
+    protected int memAvailable = 0;
+    protected int diskTailAvailable = 0;
+    protected int diskHeadAvailable = 0;
+
+    private final File spillFile;
+    private final int initialMemBufferSize;
+    private final int maxMemBufferSize;
+    private final int maxDiskBufferSize;
+
+    private volatile byte memBuffer[];
+    private int memBufferSize;
+    private RandomAccessFile spillBuffer;
+
+    private final DataInputPlus dataReader;
+
+    public RewindableDataInputStreamPlus(InputStream in, int initialMemBufferSize, int maxMemBufferSize,
+                                         File spillFile, int maxDiskBufferSize)
+    {
+        super(in);
+        dataReader = new DataInputStreamPlus(this);
+        this.initialMemBufferSize = initialMemBufferSize;
+        this.maxMemBufferSize = maxMemBufferSize;
+        this.spillFile = spillFile;
+        this.maxDiskBufferSize = maxDiskBufferSize;
+    }
+
+    /* RewindableDataInput methods */
+
+    /**
+     * Marks the current position of a stream to return to this position later via the {@link this#reset(DataPosition)} method.
+     * @return An empty @link{DataPosition} object
+     */
+    public DataPosition mark()
+    {
+        mark(0);
+        return new RewindableDataInputPlusMark();
+    }
+
+    /**
+     * Rewinds to the previously marked position via the {@link this#mark()} method.
+     * @param mark it's not possible to return to a custom position, so this parameter is ignored.
+     * @throws IOException if an error ocurs while resetting
+     */
+    public void reset(DataPosition mark) throws IOException
+    {
+        reset();
+    }
+
+    public long bytesPastMark(DataPosition mark)
+    {
+        return maxMemBufferSize - memAvailable + (diskTailAvailable == -1? 0 : maxDiskBufferSize - diskHeadAvailable - diskTailAvailable);
+    }
+
+
+    protected static class RewindableDataInputPlusMark implements DataPosition
+    {
+    }
+
+    /* InputStream methods */
+
+    public boolean markSupported()
+    {
+        return true;
+    }
+
+    /**
+     * Marks the current position of a stream to return to this position
+     * later via the {@link this#reset()} method.
+     * @param readlimit the maximum amount of bytes to cache
+     */
+    public synchronized void mark(int readlimit)
+    {
+        if (marked)
+            throw new IllegalStateException("Cannot mark already marked stream.");
+
+        if (memAvailable > 0 || diskHeadAvailable > 0 || diskTailAvailable > 0)
+            throw new IllegalStateException("Can only mark stream after reading previously marked data.");
+
+        marked = true;
+        memAvailable = maxMemBufferSize;
+        diskHeadAvailable = -1;
+        diskTailAvailable = -1;
+    }
+
+    public synchronized void reset() throws IOException
+    {
+        if (!marked)
+            throw new IOException("Must call mark() before calling reset().");
+
+        if (exhausted)
+            throw new IOException(String.format("Read more than capacity: %d bytes.", maxMemBufferSize + maxDiskBufferSize));
+
+        memAvailable = maxMemBufferSize - memAvailable;
+        memBufferSize = memAvailable;
+
+        if (diskTailAvailable == -1)
+        {
+            diskHeadAvailable = 0;
+            diskTailAvailable = 0;
+        }
+        else
+        {
+            int initialPos = diskTailAvailable > 0 ? 0 : (int)getIfNotClosed(spillBuffer).getFilePointer();
+            int diskMarkpos = initialPos + diskHeadAvailable;
+            getIfNotClosed(spillBuffer).seek(diskMarkpos);
+
+            diskHeadAvailable = diskMarkpos - diskHeadAvailable;
+            diskTailAvailable = (maxDiskBufferSize - diskTailAvailable) - diskMarkpos;
+        }
+
+        marked = false;
+    }
+
+    public int available() throws IOException
+    {
+
+        return super.available() + (marked? 0 : memAvailable + diskHeadAvailable + diskTailAvailable);
+    }
+
+    public int read() throws IOException
+    {
+        int read = readOne();
+        if (read == -1)
+            return read;
+
+        if (marked)
+        {
+            //mark exhausted
+            if (isExhausted(1))
+            {
+                exhausted = true;
+                return read;
+            }
+
+            writeOne(read);
+        }
+
+        return read;
+    }
+
+    public int read(byte[] b, int off, int len) throws IOException
+    {
+        int readBytes = readMulti(b, off, len);
+        if (readBytes == -1)
+            return readBytes;
+
+        if (marked)
+        {
+            //check we have space on buffer
+            if (isExhausted(readBytes))
+            {
+                exhausted = true;
+                return readBytes;
+            }
+
+            writeMulti(b, off, readBytes);
+        }
+
+        return readBytes;
+    }
+
+    private void maybeCreateDiskBuffer() throws IOException
+    {
+        if (spillBuffer == null)
+        {
+            if (!spillFile.getParentFile().exists())
+                spillFile.getParentFile().mkdirs();
+            spillFile.createNewFile();
+
+            this.spillBuffer = new RandomAccessFile(spillFile, "rw");
+        }
+    }
+
+
+    private int readOne() throws IOException
+    {
+        if (!marked)
+        {
+            if (memAvailable > 0)
+            {
+                int pos = memBufferSize - memAvailable;
+                memAvailable--;
+                return getIfNotClosed(memBuffer)[pos] & 0xff;
+            }
+
+            if (diskTailAvailable > 0 || diskHeadAvailable > 0)
+            {
+                int read = getIfNotClosed(spillBuffer).read();
+                if (diskTailAvailable > 0)
+                    diskTailAvailable--;
+                else if (diskHeadAvailable > 0)
+                    diskHeadAvailable++;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+                return read;
+            }
+        }
+
+        return getIfNotClosed(in).read();
+    }
+
+    private boolean isExhausted(int readBytes)
+    {
+        return exhausted || readBytes > memAvailable + (long)(diskTailAvailable == -1? maxDiskBufferSize : diskTailAvailable + diskHeadAvailable);
+    }
+
+    private int readMulti(byte[] b, int off, int len) throws IOException
+    {
+        int readBytes = 0;
+        if (!marked)
+        {
+            if (memAvailable > 0)
+            {
+                readBytes += memAvailable < len ? memAvailable : len;
+                int pos = memBufferSize - memAvailable;
+                System.arraycopy(memBuffer, pos, b, off, readBytes);
+                memAvailable -= readBytes;
+                off += readBytes;
+                len -= readBytes;
+            }
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int readFromTail = diskTailAvailable < len? diskTailAvailable : len;
+                getIfNotClosed(spillBuffer).read(b, off, readFromTail);
+                readBytes += readFromTail;
+                diskTailAvailable -= readFromTail;
+                off += readFromTail;
+                len -= readFromTail;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+            }
+            if (len > 0 && diskHeadAvailable > 0)
+            {
+                int readFromHead = diskHeadAvailable < len? diskHeadAvailable : len;
+                getIfNotClosed(spillBuffer).read(b, off, readFromHead);
+                readBytes += readFromHead;
+                diskHeadAvailable -= readFromHead;
+                off += readFromHead;
+                len -= readFromHead;
+            }
+        }
+
+        if (len > 0)
+            readBytes += getIfNotClosed(in).read(b, off, len);
+
+        return readBytes;
+    }
+
+    private void writeMulti(byte[] b, int off, int len) throws IOException
+    {
+        if (memAvailable > 0)
+        {
+            if (memBuffer == null)
+                memBuffer = new byte[initialMemBufferSize];
+            int pos = maxMemBufferSize - memAvailable;
+            int memWritten = memAvailable < len? memAvailable : len;
+            if (pos + memWritten >= getIfNotClosed(memBuffer).length)
+                growMemBuffer(pos, memWritten);
+            System.arraycopy(b, off, memBuffer, pos, memWritten);
+            off += memWritten;
+            len -= memWritten;
+            memAvailable -= memWritten;
+        }
+
+        if (len > 0)
+        {
+            if (diskTailAvailable == -1)
+            {
+                maybeCreateDiskBuffer();
+                diskHeadAvailable = (int)spillBuffer.getFilePointer();
+                diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
+            }
+
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int diskTailWritten = diskTailAvailable < len? diskTailAvailable : len;
+                getIfNotClosed(spillBuffer).write(b, off, diskTailWritten);
+                off += diskTailWritten;
+                len -= diskTailWritten;
+                diskTailAvailable -= diskTailWritten;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+            }
+
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int diskHeadWritten = diskHeadAvailable < len? diskHeadAvailable : len;
+                getIfNotClosed(spillBuffer).write(b, off, diskHeadWritten);
+            }
+        }
+    }
+
+    private void writeOne(int value) throws IOException
+    {
+        if (memAvailable > 0)
+        {
+            if (memBuffer == null)
+                memBuffer = new byte[initialMemBufferSize];
+            int pos = maxMemBufferSize - memAvailable;
+            if (pos == getIfNotClosed(memBuffer).length)
+                growMemBuffer(pos, 1);
+            getIfNotClosed(memBuffer)[pos] = (byte)value;
+            memAvailable--;
+            return;
+        }
+
+        if (diskTailAvailable == -1)
+        {
+            maybeCreateDiskBuffer();
+            diskHeadAvailable = (int)spillBuffer.getFilePointer();
+            diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
+        }
+
+        if (diskTailAvailable > 0 || diskHeadAvailable > 0)
+        {
+            getIfNotClosed(spillBuffer).write(value);
+            if (diskTailAvailable > 0)
+                diskTailAvailable--;
+            else if (diskHeadAvailable > 0)
+                diskHeadAvailable--;
+            if (diskTailAvailable == 0)
+                spillBuffer.seek(0);
+            return;
+        }
+    }
+
+    public int read(byte[] b) throws IOException
+    {
+        return read(b, 0, b.length);
+    }
+
+    private void growMemBuffer(int pos, int writeSize)
+    {
+        int newSize = Math.min(2 * (pos + writeSize), maxMemBufferSize);
+        byte newBuffer[] = new byte[newSize];
+        System.arraycopy(memBuffer, 0, newBuffer, 0, (int)pos);
+        memBuffer = newBuffer;
+    }
+
+    public long skip(long n) throws IOException
+    {
+        long skipped = 0;
+
+        if (marked)
+        {
+            //if marked, we need to cache skipped bytes
+            while (n-- > 0 && read() != -1)
+            {
+                skipped++;
+            }
+            return skipped;
+        }
+
+        if (memAvailable > 0)
+        {
+            skipped += memAvailable < n ? memAvailable : n;
+            memAvailable -= skipped;
+            n -= skipped;
+        }
+        if (n > 0 && diskTailAvailable > 0)
+        {
+            int skipFromTail = diskTailAvailable < n? diskTailAvailable : (int)n;
+            getIfNotClosed(spillBuffer).skipBytes(skipFromTail);
+            diskTailAvailable -= skipFromTail;
+            skipped += skipFromTail;
+            n -= skipFromTail;
+            if (diskTailAvailable == 0)
+                spillBuffer.seek(0);
+        }
+        if (n > 0 && diskHeadAvailable > 0)
+        {
+            int skipFromHead = diskHeadAvailable < n? diskHeadAvailable : (int)n;
+            getIfNotClosed(spillBuffer).skipBytes(skipFromHead);
+            diskHeadAvailable -= skipFromHead;
+            skipped += skipFromHead;
+            n -= skipFromHead;
+        }
+
+        if (n > 0)
+            skipped += getIfNotClosed(in).skip(n);
+
+        return skipped;
+    }
+
+    private <T> T getIfNotClosed(T in) throws IOException {
+        if (closed.get())
+            throw new IOException("Stream closed");
+        return in;
+    }
+
+    public void close() throws IOException
+    {
+        close(true);
+    }
+
+    public void close(boolean closeUnderlying) throws IOException
+    {
+        if (closed.compareAndSet(false, true))
+        {
+            Throwable fail = null;
+            if (closeUnderlying)
+            {
+                try
+                {
+                    super.close();
+                }
+                catch (IOException e)
+                {
+                    fail = merge(fail, e);
+                }
+            }
+            try
+            {
+                if (spillBuffer != null)
+                {
+                    this.spillBuffer.close();
+                    this.spillBuffer = null;
+                }
+            } catch (IOException e)
+            {
+                fail = merge(fail, e);
+            }
+            try {
+                if (spillFile.exists())
+                {
+                    spillFile.delete();
+                }
+            }
+            catch (Throwable e)
+            {
+                fail = merge(fail, e);
+            }
+            maybeFail(fail, IOException.class);
+        }
+    }
+
+    /* DataInputPlus methods */
+
+    public void readFully(byte[] b) throws IOException
+    {
+        dataReader.readFully(b);
+    }
+
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        dataReader.readFully(b, off, len);
+    }
+
+    public int skipBytes(int n) throws IOException
+    {
+        return dataReader.skipBytes(n);
+    }
+
+    public boolean readBoolean() throws IOException
+    {
+        return dataReader.readBoolean();
+    }
+
+    public byte readByte() throws IOException
+    {
+        return dataReader.readByte();
+    }
+
+    public int readUnsignedByte() throws IOException
+    {
+        return dataReader.readUnsignedByte();
+    }
+
+    public short readShort() throws IOException
+    {
+        return dataReader.readShort();
+    }
+
+    public int readUnsignedShort() throws IOException
+    {
+        return dataReader.readUnsignedShort();
+    }
+
+    public char readChar() throws IOException
+    {
+        return dataReader.readChar();
+    }
+
+    public int readInt() throws IOException
+    {
+        return dataReader.readInt();
+    }
+
+    public long readLong() throws IOException
+    {
+        return dataReader.readLong();
+    }
+
+    public float readFloat() throws IOException
+    {
+        return dataReader.readFloat();
+    }
+
+    public double readDouble() throws IOException
+    {
+        return dataReader.readDouble();
+    }
+
+    public String readLine() throws IOException
+    {
+        return dataReader.readLine();
+    }
+
+    public String readUTF() throws IOException
+    {
+        return dataReader.readUTF();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index dd49868..26316a2 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -297,7 +297,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
         return bufferOffset + (buffer == null ? 0 : buffer.position());
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         return new BufferedFileWriterMark(current());
     }
@@ -306,7 +306,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
      * Drops all buffered data that's past the limits of our new file mark + buffer capacity, or syncs and truncates
      * the underlying file to the marked position
      */
-    public void resetAndTruncate(FileMark mark)
+    public void resetAndTruncate(DataPosition mark)
     {
         assert mark instanceof BufferedFileWriterMark;
 
@@ -404,7 +404,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class BufferedFileWriterMark implements FileMark
+    protected static class BufferedFileWriterMark implements DataPosition
     {
         final long pointer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java b/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
new file mode 100644
index 0000000..dc5bbb6
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * This class is to track bytes read from given DataInput
+ */
+public class TrackedDataInputPlus implements DataInputPlus, BytesReadTracker
+{
+    private long bytesRead;
+    final DataInput source;
+
+    public TrackedDataInputPlus(DataInput source)
+    {
+        this.source = source;
+    }
+
+    public long getBytesRead()
+    {
+        return bytesRead;
+    }
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count)
+    {
+        bytesRead = count;
+    }
+
+    public boolean readBoolean() throws IOException
+    {
+        boolean bool = source.readBoolean();
+        bytesRead += 1;
+        return bool;
+    }
+
+    public byte readByte() throws IOException
+    {
+        byte b = source.readByte();
+        bytesRead += 1;
+        return b;
+    }
+
+    public char readChar() throws IOException
+    {
+        char c = source.readChar();
+        bytesRead += 2;
+        return c;
+    }
+
+    public double readDouble() throws IOException
+    {
+        double d = source.readDouble();
+        bytesRead += 8;
+        return d;
+    }
+
+    public float readFloat() throws IOException
+    {
+        float f = source.readFloat();
+        bytesRead += 4;
+        return f;
+    }
+
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        source.readFully(b, off, len);
+        bytesRead += len;
+    }
+
+    public void readFully(byte[] b) throws IOException
+    {
+        source.readFully(b);
+        bytesRead += b.length;
+    }
+
+    public int readInt() throws IOException
+    {
+        int i = source.readInt();
+        bytesRead += 4;
+        return i;
+    }
+
+    public String readLine() throws IOException
+    {
+        // since this method is deprecated and cannot track bytes read
+        // just throw exception
+        throw new UnsupportedOperationException();
+    }
+
+    public long readLong() throws IOException
+    {
+        long l = source.readLong();
+        bytesRead += 8;
+        return l;
+    }
+
+    public short readShort() throws IOException
+    {
+        short s = source.readShort();
+        bytesRead += 2;
+        return s;
+    }
+
+    public String readUTF() throws IOException
+    {
+        return DataInputStream.readUTF(this);
+    }
+
+    public int readUnsignedByte() throws IOException
+    {
+        int i = source.readUnsignedByte();
+        bytesRead += 1;
+        return i;
+    }
+
+    public int readUnsignedShort() throws IOException
+    {
+        int i = source.readUnsignedShort();
+        bytesRead += 2;
+        return i;
+    }
+
+    public int skipBytes(int n) throws IOException
+    {
+        int skipped = source.skipBytes(n);
+        bytesRead += skipped;
+        return skipped;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/TrackedInputStream.java b/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
new file mode 100644
index 0000000..f398d30
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * This class is to track bytes read from given DataInputStream
+ */
+public class TrackedInputStream extends FilterInputStream implements BytesReadTracker
+{
+    private long bytesRead;
+
+    public TrackedInputStream(InputStream source)
+    {
+        super(source);
+    }
+
+    public long getBytesRead()
+    {
+        return bytesRead;
+    }
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count)
+    {
+        bytesRead = count;
+    }
+
+    public int read() throws IOException
+    {
+        int read = super.read();
+        bytesRead += 1;
+        return read;
+    }
+
+    public int read(byte[] b, int off, int len) throws IOException
+    {
+        int read = super.read(b, off, len);
+        bytesRead += read;
+        return read;
+    }
+
+    public int read(byte[] b) throws IOException
+    {
+        int read = super.read(b);
+        bytesRead += read;
+        return read;
+    }
+
+    public long skip(long n) throws IOException
+    {
+        long skip = super.skip(n);
+        bytesRead += skip;
+        return skip;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f445e25..fd7f4b6 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4221,6 +4221,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return Collections.unmodifiableList(keyspaceNamesList);
     }
 
+
     public List<String> getNonSystemKeyspaces()
     {
         List<String> keyspaceNamesList = new ArrayList<>(Schema.instance.getNonSystemKeyspaces());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index 268f974..f8db26b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -40,10 +40,13 @@ import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.util.RewindableDataInputStreamPlus;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedInputStream;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 
@@ -105,9 +108,9 @@ public class StreamReader
                      session.planId(), fileSeqNum, session.peer, repairedAt, totalSize, cfs.keyspace.getName(),
                      cfs.getColumnFamilyName());
 
-        DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
-        BytesReadTracker in = new BytesReadTracker(dis);
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, header.toHeader(cfs.metadata));
+        TrackedInputStream in = new TrackedInputStream(new LZFInputStream(Channels.newInputStream(channel)));
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+                                                                 totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
         {
@@ -131,12 +134,22 @@ public class StreamReader
             {
                 writer.abort(e);
             }
-            drain(dis, in.getBytesRead());
+            drain(in, in.getBytesRead());
             if (e instanceof IOException)
                 throw (IOException) e;
             else
                 throw Throwables.propagate(e);
         }
+        finally
+        {
+            if (deserializer != null)
+                deserializer.cleanup();
+        }
+    }
+
+    protected SerializationHeader getHeader(CFMetaData metadata)
+    {
+        return header != null? header.toHeader(metadata) : null; //pre-3.0 sstable have no SerializationHeader
     }
 
     protected SSTableMultiWriter createWriter(ColumnFamilyStore cfs, long totalSize, long repairedAt, SSTableFormat.Type format) throws IOException
@@ -146,8 +159,7 @@ public class StreamReader
             throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
         desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
 
-
-        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), session.getTransaction(cfId));
+        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, getHeader(cfs.metadata), session.getTransaction(cfId));
     }
 
     protected void drain(InputStream dis, long bytesRead) throws IOException
@@ -185,6 +197,13 @@ public class StreamReader
 
     public static class StreamDeserializer extends UnmodifiableIterator<Unfiltered> implements UnfilteredRowIterator
     {
+        public static final int INITIAL_MEM_BUFFER_SIZE = Integer.getInteger("cassandra.streamdes.initial_mem_buffer_size", 32768);
+        public static final int MAX_MEM_BUFFER_SIZE = Integer.getInteger("cassandra.streamdes.max_mem_buffer_size", 1048576);
+        public static final int MAX_SPILL_FILE_SIZE = Integer.getInteger("cassandra.streamdes.max_spill_file_size", Integer.MAX_VALUE);
+
+        public static final String BUFFER_FILE_PREFIX = "buf";
+        public static final String BUFFER_FILE_SUFFIX = "dat";
+
         private final CFMetaData metadata;
         private final DataInputPlus in;
         private final SerializationHeader header;
@@ -196,11 +215,20 @@ public class StreamReader
         private Row staticRow;
         private IOException exception;
 
-        public StreamDeserializer(CFMetaData metadata, DataInputPlus in, Version version, SerializationHeader header)
+        public StreamDeserializer(CFMetaData metadata, InputStream in, Version version, SerializationHeader header,
+                                  long totalSize, UUID sessionId) throws IOException
         {
-            assert version.storeRows() : "We don't allow streaming from pre-3.0 nodes";
             this.metadata = metadata;
-            this.in = in;
+            // streaming pre-3.0 sstables require mark/reset support from source stream
+            if (version.correspondingMessagingVersion() < MessagingService.VERSION_30)
+            {
+                logger.trace("Initializing rewindable input stream for reading legacy sstable with {} bytes with following " +
+                             "parameters: initial_mem_buffer_size={}, max_mem_buffer_size={}, max_spill_file_size={}.",
+                             totalSize, INITIAL_MEM_BUFFER_SIZE, MAX_MEM_BUFFER_SIZE, MAX_SPILL_FILE_SIZE);
+                File bufferFile = getTempBufferFile(metadata, totalSize, sessionId);
+                this.in = new RewindableDataInputStreamPlus(in, INITIAL_MEM_BUFFER_SIZE, MAX_MEM_BUFFER_SIZE, bufferFile, MAX_SPILL_FILE_SIZE);
+            } else
+                this.in = new DataInputPlus.DataInputStreamPlus(in);
             this.helper = new SerializationHelper(metadata, version.correspondingMessagingVersion(), SerializationHelper.Flag.PRESERVE_SIZE);
             this.header = header;
         }
@@ -292,5 +320,41 @@ public class StreamReader
         public void close()
         {
         }
+
+        /* We have a separate cleanup method because sometimes close is called before exhausting the
+           StreamDeserializer (for instance, when enclosed in an try-with-resources wrapper, such as in
+           BigTableWriter.append()).
+         */
+        public void cleanup()
+        {
+            if (in instanceof RewindableDataInputStreamPlus)
+            {
+                try
+                {
+                    ((RewindableDataInputStreamPlus) in).close(false);
+                }
+                catch (IOException e)
+                {
+                    logger.warn("Error while closing RewindableDataInputStreamPlus.", e);
+                }
+            }
+        }
+
+        private static File getTempBufferFile(CFMetaData metadata, long totalSize, UUID sessionId) throws IOException
+        {
+            ColumnFamilyStore cfs = Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName);
+            if (cfs == null)
+            {
+                // schema was dropped during streaming
+                throw new RuntimeException(String.format("CF %s.%s was dropped during streaming", metadata.ksName, metadata.cfName));
+            }
+
+            long maxSize = Math.min(MAX_SPILL_FILE_SIZE, totalSize);
+            File tmpDir = cfs.getDirectories().getTemporaryWriteableDirectoryAsFile(maxSize);
+            if (tmpDir == null)
+                throw new IOException(String.format("No sufficient disk space to stream legacy sstable from {}.{}. " +
+                                                         "Required disk space: %s.", FBUtilities.prettyPrintMemory(maxSize)));
+            return new File(tmpDir, String.format("%s-%s.%s", BUFFER_FILE_PREFIX, sessionId, BUFFER_FILE_SUFFIX));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index 5210d5b..9719587 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -24,7 +24,6 @@ import java.nio.channels.ReadableByteChannel;
 
 import com.google.common.base.Throwables;
 
-import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 
 import org.slf4j.Logger;
@@ -38,7 +37,7 @@ import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamReader;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedInputStream;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -83,8 +82,10 @@ public class CompressedStreamReader extends StreamReader
 
         CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo,
                                                               inputVersion.compressedChecksumType(), cfs::getCrcCheckChance);
-        BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, header.toHeader(cfs.metadata));
+        TrackedInputStream in = new TrackedInputStream(cis);
+
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+                                                                 totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
         {
@@ -115,17 +116,22 @@ public class CompressedStreamReader extends StreamReader
         {
             if (deserializer != null)
                 logger.warn("[Stream {}] Error while reading partition {} from stream on ks='{}' and table='{}'.",
-                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getColumnFamilyName());
+                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getTableName());
             if (writer != null)
             {
                 writer.abort(e);
             }
-            drain(cis, in.getBytesRead());
+            drain(in, in.getBytesRead());
             if (e instanceof IOException)
                 throw (IOException) e;
             else
                 throw Throwables.propagate(e);
         }
+        finally
+        {
+            if (deserializer != null)
+                deserializer.cleanup();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
index e1e13b7..2b5047d 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -204,7 +204,7 @@ public class FileMessageHeader
             out.writeLong(header.repairedAt);
             out.writeInt(header.sstableLevel);
 
-            if (version >= StreamMessage.VERSION_30)
+            if (version >= StreamMessage.VERSION_30 && header.version.storeRows())
                 SerializationHeader.serializer.serialize(header.version, header.header, out);
             return compressionInfo;
         }
@@ -227,7 +227,7 @@ public class FileMessageHeader
             CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, MessagingService.current_version);
             long repairedAt = in.readLong();
             int sstableLevel = in.readInt();
-            SerializationHeader.Component header = version >= StreamMessage.VERSION_30
+            SerializationHeader.Component header = version >= StreamMessage.VERSION_30 && sstableVersion.storeRows()
                                                  ? SerializationHeader.serializer.deserialize(sstableVersion, in)
                                                  : null;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/tools/nodetool/Repair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
index 84b463c..bd1a916 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Repair.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
@@ -56,7 +56,7 @@ public class Repair extends NodeToolCmd
     private boolean localDC = false;
 
     @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
-    private List<String> specificDataCenters = new ArrayList<>();
+    private List<String> specificDataCenters = new ArrayList<>();;
 
     @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
     private List<String> specificHosts = new ArrayList<>();


[12/12] cassandra git commit: Merge branch 'cassandra-3.5' into trunk

Posted by yu...@apache.org.
Merge branch 'cassandra-3.5' into trunk


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/68e6a833
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/68e6a833
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/68e6a833

Branch: refs/heads/trunk
Commit: 68e6a8333279cf2207ad35c68b8929ce4d153cf2
Parents: 7043744 587773f
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Mar 17 11:03:25 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 17 11:03:25 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   2 +
 .../org/apache/cassandra/db/Directories.java    |  30 +
 .../cassandra/db/SerializationHeader.java       |   5 +
 .../org/apache/cassandra/db/Serializers.java    | 114 ++--
 .../columniterator/AbstractSSTableIterator.java |   4 +-
 .../EncryptedFileSegmentInputStream.java        |   4 +-
 .../cassandra/hints/ChecksummedDataInput.java   |   8 +-
 .../org/apache/cassandra/hints/HintMessage.java |   4 +-
 .../io/compress/CompressedSequentialWriter.java |   8 +-
 .../io/sstable/SSTableSimpleIterator.java       |  11 +-
 .../sstable/format/RangeAwareSSTableWriter.java |   8 +-
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../io/sstable/format/SSTableWriter.java        |   2 +-
 .../io/sstable/format/big/BigTableWriter.java   |   4 +-
 .../cassandra/io/util/BytesReadTracker.java     |  30 +
 .../apache/cassandra/io/util/DataPosition.java  |  21 +
 .../apache/cassandra/io/util/FileDataInput.java |   8 +-
 .../org/apache/cassandra/io/util/FileMark.java  |  20 -
 .../io/util/FileSegmentInputStream.java         |  12 +-
 .../cassandra/io/util/RandomAccessReader.java   |   8 +-
 .../cassandra/io/util/RewindableDataInput.java  |  30 +
 .../io/util/RewindableDataInputStreamPlus.java  | 569 +++++++++++++++++++
 .../cassandra/io/util/SequentialWriter.java     |   6 +-
 .../cassandra/io/util/TrackedDataInputPlus.java | 150 +++++
 .../cassandra/io/util/TrackedInputStream.java   |  76 +++
 .../cassandra/service/StorageService.java       |   1 +
 .../cassandra/streaming/StreamReader.java       |  85 ++-
 .../compress/CompressedStreamReader.java        |  18 +-
 .../streaming/messages/FileMessageHeader.java   |   4 +-
 .../apache/cassandra/tools/nodetool/Repair.java |   2 +-
 .../cassandra/utils/BytesReadTracker.java       | 153 -----
 .../cassandra/utils/CloseableIterator.java      |   1 -
 ...acy_jb_clust_compact-jb-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-Data.db | Bin 0 -> 5270 bytes
 ...ables-legacy_jb_clust_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_jb_clust_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_jb_clust_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...bles-legacy_jb_clust_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-TOC.txt |   7 +
 ...lust_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_jb_clust_counter_compact-jb-1-Data.db | Bin 0 -> 4228 bytes
 ...gacy_jb_clust_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_jb_clust_counter_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ..._jb_clust_counter_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...acy_jb_clust_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...legacy_jb_clust_counter_compact-jb-1-TOC.txt |   7 +
 ...cy_jb_simple_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_jb_simple_compact-jb-1-Data.db | Bin 0 -> 108 bytes
 ...bles-legacy_jb_simple_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_jb_simple_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_jb_simple_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...les-legacy_jb_simple_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...tables-legacy_jb_simple_compact-jb-1-TOC.txt |   7 +
 ...mple_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_jb_simple_counter_compact-jb-1-Data.db | Bin 0 -> 118 bytes
 ...acy_jb_simple_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_jb_simple_counter_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...jb_simple_counter_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...cy_jb_simple_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...egacy_jb_simple_counter_compact-jb-1-TOC.txt |   7 +
 ...acy_ka_clust_compact-ka-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-Data.db | Bin 0 -> 5277 bytes
 ...les-legacy_ka_clust_compact-ka-1-Digest.sha1 |   1 +
 ...ables-legacy_ka_clust_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_ka_clust_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_ka_clust_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...bles-legacy_ka_clust_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-TOC.txt |   8 +
 ...lust_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_ka_clust_counter_compact-ka-1-Data.db | Bin 0 -> 4527 bytes
 ...cy_ka_clust_counter_compact-ka-1-Digest.sha1 |   1 +
 ...gacy_ka_clust_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_ka_clust_counter_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ..._ka_clust_counter_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...acy_ka_clust_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...legacy_ka_clust_counter_compact-ka-1-TOC.txt |   8 +
 ...cy_ka_simple_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_ka_simple_compact-ka-1-Data.db | Bin 0 -> 105 bytes
 ...es-legacy_ka_simple_compact-ka-1-Digest.sha1 |   1 +
 ...bles-legacy_ka_simple_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_ka_simple_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_ka_simple_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...les-legacy_ka_simple_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...tables-legacy_ka_simple_compact-ka-1-TOC.txt |   8 +
 ...mple_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_ka_simple_counter_compact-ka-1-Data.db | Bin 0 -> 124 bytes
 ...y_ka_simple_counter_compact-ka-1-Digest.sha1 |   1 +
 ...acy_ka_simple_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_ka_simple_counter_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...ka_simple_counter_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...cy_ka_simple_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...egacy_ka_simple_counter_compact-ka-1-TOC.txt |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_la_clust_compact/la-1-big-Data.db    | Bin 0 -> 5286 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_clust_compact/la-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_la_clust_compact/la-1-big-Index.db   | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../legacy_la_clust_compact/la-1-big-Summary.db | Bin 0 -> 75 bytes
 .../legacy_la_clust_compact/la-1-big-TOC.txt    |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 4527 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_la_simple_compact/la-1-big-Data.db   | Bin 0 -> 106 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_simple_compact/la-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_la_simple_compact/la-1-big-Index.db  | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../legacy_la_simple_compact/la-1-big-TOC.txt   |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 124 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_ma_clust_compact/ma-1-big-Data.db    | Bin 0 -> 5393 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_clust_compact/ma-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_ma_clust_compact/ma-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7046 bytes
 .../legacy_ma_clust_compact/ma-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_ma_clust_compact/ma-1-big-TOC.txt    |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 4606 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7055 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_ma_simple_compact/ma-1-big-Data.db   | Bin 0 -> 91 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_simple_compact/ma-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_ma_simple_compact/ma-1-big-Index.db  | Bin 0 -> 26 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4640 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_ma_simple_compact/ma-1-big-TOC.txt   |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4649 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../cassandra/AbstractSerializationsTester.java |   1 -
 .../apache/cassandra/db/DirectoriesTest.java    |  98 ++--
 .../cassandra/gms/SerializationsTest.java       |   1 -
 .../CompressedRandomAccessReaderTest.java       |   6 +-
 .../CompressedSequentialWriterTest.java         |   4 +-
 .../cassandra/io/sstable/LegacySSTableTest.java | 369 ++++++------
 .../io/util/BufferedRandomAccessFileTest.java   |   4 +-
 .../io/util/RandomAccessReaderTest.java         |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java | 539 ++++++++++++++++++
 .../cassandra/utils/BytesReadTrackerTest.java   | 104 +++-
 167 files changed, 2115 insertions(+), 550 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/68e6a833/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 149b1d1,53dd292..a5819e6
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,22 -1,6 +1,23 @@@
 +3.6
 + * Stress should exit with non-zero status after failure (CASSANDRA-10340)
 + * Add client to cqlsh SHOW_SESSION (CASSANDRA-8958)
 + * Fix nodetool tablestats keyspace level metrics (CASSANDRA-11226)
 + * Store repair options in parent_repair_history (CASSANDRA-11244)
 + * Print current leveling in sstableofflinerelevel (CASSANDRA-9588)
 + * Change repair message for keyspaces with RF 1 (CASSANDRA-11203)
 + * Remove hard-coded SSL cipher suites and protocols (CASSANDRA-10508)
 + * Improve concurrency in CompactionStrategyManager (CASSANDRA-10099)
 + * (cqlsh) interpret CQL type for formatting blobs (CASSANDRA-11274)
 + * Refuse to start and print txn log information in case of disk
 +   corruption (CASSANDRA-10112)
 + * Resolve some eclipse-warnings (CASSANDRA-11086)
 + * (cqlsh) Show static columns in a different color (CASSANDRA-11059)
 + * Allow to remove TTLs on table with default_time_to_live (CASSANDRA-11207)
 +
 +
  3.5
  Merged from 3.0:
+  * Support streaming pre-3.0 sstables (CASSANDRA-10990)
   * Add backpressure to compressed or encrypted commit log (CASSANDRA-10971)
   * SSTableExport supports secondary index tables (CASSANDRA-11330)
   * Fix sstabledump to include missing info in debug output (CASSANDRA-11321)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/68e6a833/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 12a5f62,fbfb7ee..a2102f4
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -589,13 -588,10 +589,15 @@@ public class ColumnFamilyStore implemen
           // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
          clearEphemeralSnapshots(directories);
  
+         directories.removeTemporaryDirectories();
+ 
          logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.cfName);
 -        LifecycleTransaction.removeUnfinishedLeftovers(metadata);
 +        if (!LifecycleTransaction.removeUnfinishedLeftovers(metadata))
 +            throw new StartupException(StartupException.ERR_WRONG_DISK_STATE,
 +                                       String.format("Cannot remove temporary or obsoleted files for %s.%s due to a problem with transaction " +
 +                                                     "log files. Please check records with problems in the log messages above and fix them. " +
 +                                                     "Refer to the 3.0 upgrading instructions in NEWS.txt " +
 +                                                     "for a description of transaction log files.", metadata.ksName, metadata.cfName));
  
          logger.trace("Further extra check for orphan sstable files for {}", metadata.cfName);
          for (Map.Entry<Descriptor,Set<Component>> sstableFiles : directories.sstableLister(Directories.OnTxnErr.IGNORE).list().entrySet())


[07/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 501f4ae..3a1f348 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -25,7 +25,6 @@ import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
-import org.apache.cassandra.io.util.NIODataInputStream;
 import org.apache.cassandra.net.MessagingService;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 8732881..f864bbc 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -243,46 +243,76 @@ public class DirectoriesTest
         for (CFMetaData cfm : CFM)
         {
             Directories directories = new Directories(cfm);
-            Directories.SSTableLister lister;
-            Set<File> listed;
+            checkFiles(cfm, directories);
+        }
+    }
 
-            // List all but no snapshot, backup
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+    private void checkFiles(CFMetaData cfm, Directories directories)
+    {
+        Directories.SSTableLister lister;
+        Set<File> listed;// List all but no snapshot, backup
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
+        }
 
-            // List all but including backup (but no snapshot)
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+        // List all but including backup (but no snapshot)
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
+        }
 
-            // Skip temporary and compacted
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else if (f.getName().contains("tmp-"))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+        // Skip temporary and compacted
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else if (f.getName().contains("tmp-"))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
         }
     }
 
+    @Test
+    public void testTemporaryFile() throws IOException
+    {
+        for (CFMetaData cfm : CFM)
+        {
+            Directories directories = new Directories(cfm);
+
+            File tempDir = directories.getTemporaryWriteableDirectoryAsFile(10);
+            tempDir.mkdir();
+            File tempFile = new File(tempDir, "tempFile");
+            tempFile.createNewFile();
+
+            assertTrue(tempDir.exists());
+            assertTrue(tempFile.exists());
+
+            //make sure temp dir/file will not affect existing sstable listing
+            checkFiles(cfm, directories);
+
+            directories.removeTemporaryDirectories();
+
+            //make sure temp dir/file deletion will not affect existing sstable listing
+            checkFiles(cfm, directories);
+
+            assertFalse(tempDir.exists());
+            assertFalse(tempFile.exists());
+        }
+    }
 
     @Test
     public void testDiskFailurePolicy_best_effort()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/gms/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/SerializationsTest.java b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
index 8b7ad1f..e50b461 100644
--- a/test/unit/org/apache/cassandra/gms/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.gms;
 
 import org.apache.cassandra.AbstractSerializationsTester;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 9154d79..74b5c74 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.MmappedRegions;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriter;
@@ -84,7 +84,7 @@ public class CompressedRandomAccessReaderTest
                 for (int i = 0; i < 20; i++)
                     writer.write("x".getBytes());
 
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
                 // write enough garbage to create new chunks:
                 for (int i = 0; i < 40; ++i)
                     writer.write("y".getBytes());
@@ -126,7 +126,7 @@ public class CompressedRandomAccessReaderTest
                 : SequentialWriter.open(f))
             {
                 writer.write("The quick ".getBytes());
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
                 writer.write("blue fox jumps over the lazy dog".getBytes());
 
                 // write enough to be sure to change chunk

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index 56c83da..9b09f0b 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriterTest;
 import org.apache.cassandra.schema.CompressionParams;
@@ -104,7 +104,7 @@ public class CompressedSequentialWriterTest extends SequentialWriterTest
                 dataPost.flip();
 
                 writer.write(dataPre);
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
 
                 // Write enough garbage to transition chunk
                 for (int i = 0; i < CompressionParams.DEFAULT_CHUNK_LENGTH; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 00727b8..4b9a769 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -21,14 +21,13 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
-import java.util.Set;
 
+import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -37,16 +36,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.rows.SliceableUnfilteredRowIterator;
-import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -55,7 +48,6 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
-import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamPlan;
@@ -63,8 +55,6 @@ import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
 /**
  * Tests backwards compatibility for SSTables
  */
@@ -73,10 +63,7 @@ public class LegacySSTableTest
     private static final Logger logger = LoggerFactory.getLogger(LegacySSTableTest.class);
 
     public static final String LEGACY_SSTABLE_PROP = "legacy-sstable-root";
-    public static final String KSNAME = "Keyspace1";
-    public static final String CFNAME = "Standard1";
 
-    public static Set<String> TEST_DATA;
     public static File LEGACY_SSTABLE_ROOT;
 
     /**
@@ -104,58 +91,78 @@ public class LegacySSTableTest
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
-
-        CFMetaData metadata = CFMetaData.Builder.createDense(KSNAME, CFNAME, false, false)
-                                                .addPartitionKey("key", BytesType.instance)
-                                                .addClusteringColumn("column", BytesType.instance)
-                                                .addRegularColumn("value", BytesType.instance)
-                                                .build();
-
-        SchemaLoader.createKeyspace(KSNAME,
-                                    KeyspaceParams.simple(1),
-                                    metadata);
-        beforeClass();
-    }
-
-    public static void beforeClass()
-    {
+        StorageService.instance.initServer();
         Keyspace.setInitialized();
+        createKeyspace();
+        for (String legacyVersion : legacyVersions)
+        {
+            createTables(legacyVersion);
+        }
         String scp = System.getProperty(LEGACY_SSTABLE_PROP);
         assert scp != null;
         LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
         assert LEGACY_SSTABLE_ROOT.isDirectory();
+    }
 
-        TEST_DATA = new HashSet<String>();
-        for (int i = 100; i < 1000; ++i)
-            TEST_DATA.add(Integer.toString(i));
+    @After
+    public void tearDown()
+    {
+        for (String legacyVersion : legacyVersions)
+        {
+            truncateTables(legacyVersion);
+        }
     }
 
     /**
      * Get a descriptor for the legacy sstable at the given version.
      */
-    protected Descriptor getDescriptor(String ver)
+    protected Descriptor getDescriptor(String legacyVersion, String table)
     {
-        File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
-        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, SSTableFormat.Type.LEGACY);
+        return new Descriptor(legacyVersion, getTableDir(legacyVersion, table), "legacy_tables", table, 1,
+                              BigFormat.instance.getVersion(legacyVersion).hasNewFileName()?
+                              SSTableFormat.Type.BIG :SSTableFormat.Type.LEGACY);
     }
 
     @Test
-    public void testStreaming() throws Throwable
+    public void testLoadLegacyCqlTables() throws Exception
     {
-        StorageService.instance.initServer();
+        for (String legacyVersion : legacyVersions)
+        {
+            logger.info("Loading legacy version: {}", legacyVersion);
+            loadLegacyTables(legacyVersion);
+            CacheService.instance.invalidateKeyCache();
+            long startCount = CacheService.instance.keyCache.size();
+            verifyReads(legacyVersion);
+            verifyCache(legacyVersion, startCount);
+        }
+    }
+
+    @Test
+    public void testStreamLegacyCqlTables() throws Exception
+    {
+        for (String legacyVersion : legacyVersions)
+        {
+            streamLegacyTables(legacyVersion);
+            verifyReads(legacyVersion);
+        }
+    }
 
-        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
+    private void streamLegacyTables(String legacyVersion) throws Exception
+    {
+        for (int compact = 0; compact <= 1; compact++)
         {
-            if (!new File(LEGACY_SSTABLE_ROOT + File.separator + version.getName() + File.separator + KSNAME).isDirectory())
-                continue;
-            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatibleForStreaming())
-                testStreaming(version.getName());
+            logger.info("Streaming legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
         }
     }
 
-    private void testStreaming(String version) throws Exception
+    private void streamLegacyTable(String tablePattern, String legacyVersion, String compactNameSuffix) throws Exception
     {
-        SSTableReader sstable = SSTableReader.open(getDescriptor(version));
+        String table = String.format(tablePattern, legacyVersion, compactNameSuffix);
+        SSTableReader sstable = SSTableReader.open(getDescriptor(legacyVersion, table));
         IPartitioner p = sstable.getPartitioner();
         List<Range<Token>> ranges = new ArrayList<>();
         ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("100"))));
@@ -166,94 +173,41 @@ public class LegacySSTableTest
                                                                sstable.estimatedKeysForRanges(ranges), sstable.getSSTableMetadata().repairedAt));
         new StreamPlan("LegacyStreamingTest").transferFiles(FBUtilities.getBroadcastAddress(), details)
                                              .execute().get();
-
-        ColumnFamilyStore cfs = Keyspace.open(KSNAME).getColumnFamilyStore(CFNAME);
-        assert cfs.getLiveSSTables().size() == 1;
-        sstable = cfs.getLiveSSTables().iterator().next();
-        for (String keystring : TEST_DATA)
-        {
-            ByteBuffer key = bytes(keystring);
-
-            SliceableUnfilteredRowIterator iter = sstable.iterator(Util.dk(key), ColumnFilter.selectionBuilder().add(cfs.metadata.getColumnDefinition(bytes("name"))).build(), false, false);
-
-            // check not deleted (CASSANDRA-6527)
-            assert iter.partitionLevelDeletion().equals(DeletionTime.LIVE);
-            assert iter.next().clustering().get(0).equals(key);
-        }
-        sstable.selfRef().release();
-    }
-
-    @Test
-    public void testVersions() throws Throwable
-    {
-        boolean notSkipped = false;
-
-        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
-        {
-            if (!new File(LEGACY_SSTABLE_ROOT + File.separator + version.getName() + File.separator + KSNAME).isDirectory())
-                continue;
-            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatible())
-            {
-                notSkipped = true;
-                testVersion(version.getName());
-            }
-        }
-
-        assert notSkipped;
     }
 
-    public void testVersion(String version) throws Throwable
+    private static void loadLegacyTables(String legacyVersion) throws Exception
     {
-        try
-        {
-            ColumnFamilyStore cfs = Keyspace.open(KSNAME).getColumnFamilyStore(CFNAME);
-
-
-            SSTableReader reader = SSTableReader.open(getDescriptor(version));
-            for (String keystring : TEST_DATA)
-            {
-
-                ByteBuffer key = bytes(keystring);
-
-                SliceableUnfilteredRowIterator iter = reader.iterator(Util.dk(key), ColumnFilter.selection(cfs.metadata.partitionColumns()), false, false);
-
-                // check not deleted (CASSANDRA-6527)
-                assert iter.partitionLevelDeletion().equals(DeletionTime.LIVE);
-                assert iter.next().clustering().get(0).equals(key);
-            }
-
-            // TODO actually test some reads
-        }
-        catch (Throwable e)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            System.err.println("Failed to read " + version);
-            throw e;
+            logger.info("Preparing legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
         }
     }
 
-    @Test
-    public void testLegacyCqlTables() throws Exception
+    private static void verifyCache(String legacyVersion, long startCount) throws InterruptedException, java.util.concurrent.ExecutionException
     {
-        createKeyspace();
-
-        loadLegacyTables();
+        //For https://issues.apache.org/jira/browse/CASSANDRA-10778
+        //Validate whether the key cache successfully saves in the presence of old keys as
+        //well as loads the correct number of keys
+        long endCount = CacheService.instance.keyCache.size();
+        Assert.assertTrue(endCount > startCount);
+        CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
+        CacheService.instance.invalidateKeyCache();
+        Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
+        CacheService.instance.keyCache.loadSaved();
+        if (BigFormat.instance.getVersion(legacyVersion).storeRows())
+            Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
+        else
+            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
     }
 
-    private static void loadLegacyTables() throws Exception
+    private static void verifyReads(String legacyVersion)
     {
-        for (String legacyVersion : legacyVersions)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            logger.info("Preparing legacy version {}", legacyVersion);
-
-            createTables(legacyVersion);
-
-            loadLegacyTable("legacy_%s_simple", legacyVersion);
-            loadLegacyTable("legacy_%s_simple_counter", legacyVersion);
-            loadLegacyTable("legacy_%s_clust", legacyVersion);
-            loadLegacyTable("legacy_%s_clust_counter", legacyVersion);
-
-            CacheService.instance.invalidateKeyCache();
-            long startCount = CacheService.instance.keyCache.size();
             for (int ck = 0; ck < 50; ck++)
             {
                 String ckValue = Integer.toString(ck) + longString;
@@ -265,58 +219,94 @@ public class LegacySSTableTest
                     UntypedResultSet rs;
                     if (ck == 0)
                     {
-                        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple WHERE pk=?", legacyVersion), pkValue);
-                        Assert.assertNotNull(rs);
-                        Assert.assertEquals(1, rs.size());
-                        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
-                        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter WHERE pk=?", legacyVersion), pkValue);
-                        Assert.assertNotNull(rs);
-                        Assert.assertEquals(1, rs.size());
-                        Assert.assertEquals(1L, rs.one().getLong("val"));
+                        readSimpleTable(legacyVersion, getCompactNameSuffix(compact),  pkValue);
+                        readSimpleCounterTable(legacyVersion, getCompactNameSuffix(compact), pkValue);
                     }
 
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
-                    assertLegacyClustRows(1, rs);
-
-                    String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
-                    String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion), pkValue, ckValue, ckValue2, ckValue3);
-                    assertLegacyClustRows(3, rs);
-
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
-                    Assert.assertNotNull(rs);
-                    Assert.assertEquals(1, rs.size());
-                    Assert.assertEquals(1L, rs.one().getLong("val"));
+                    readClusteringTable(legacyVersion, getCompactNameSuffix(compact), ck, ckValue, pkValue);
+                    readClusteringCounterTable(legacyVersion, getCompactNameSuffix(compact), ckValue, pkValue);
                 }
             }
-
-            //For https://issues.apache.org/jira/browse/CASSANDRA-10778
-            //Validate whether the key cache successfully saves in the presence of old keys as
-            //well as loads the correct number of keys
-            long endCount = CacheService.instance.keyCache.size();
-            Assert.assertTrue(endCount > startCount);
-            CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
-            CacheService.instance.invalidateKeyCache();
-            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
-            CacheService.instance.keyCache.loadSaved();
-            if (BigFormat.instance.getVersion(legacyVersion).storeRows())
-                Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
-            else
-                Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
         }
     }
 
-    private void createKeyspace()
+    private static void readClusteringCounterTable(String legacyVersion, String compactSuffix, String ckValue, String pkValue)
+    {
+        logger.debug("Read legacy_{}_clust_counter{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals(1L, rs.one().getLong("val"));
+    }
+
+    private static void readClusteringTable(String legacyVersion, String compactSuffix, int ck, String ckValue, String pkValue)
+    {
+        logger.debug("Read legacy_{}_clust{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        assertLegacyClustRows(1, rs);
+
+        String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
+        String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion, compactSuffix), pkValue, ckValue, ckValue2, ckValue3);
+        assertLegacyClustRows(3, rs);
+    }
+
+    private static void readSimpleCounterTable(String legacyVersion, String compactSuffix, String pkValue)
+    {
+        logger.debug("Read legacy_{}_simple_counter{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals(1L, rs.one().getLong("val"));
+    }
+
+    private static void readSimpleTable(String legacyVersion, String compactSuffix, String pkValue)
+    {
+        logger.debug("Read simple: legacy_{}_simple{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
+    }
+
+    private static void createKeyspace()
     {
         QueryProcessor.executeInternal("CREATE KEYSPACE legacy_tables WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
     }
 
     private static void createTables(String legacyVersion)
     {
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple (pk text PRIMARY KEY, val text)", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter (pk text PRIMARY KEY, val counter)", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter (pk text, ck text, val counter, PRIMARY KEY (pk, ck))", legacyVersion));
+        for (int i=0; i<=1; i++)
+        {
+            String compactSuffix = getCompactNameSuffix(i);
+            String tableSuffix = i == 0? "" : " WITH COMPACT STORAGE";
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple%s (pk text PRIMARY KEY, val text)%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter%s (pk text PRIMARY KEY, val counter)%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust%s (pk text, ck text, val text, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter%s (pk text, ck text, val counter, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
+        }
+    }
+
+    private static String getCompactNameSuffix(int i)
+    {
+        return i == 0? "" : "_compact";
+    }
+
+    private static void truncateTables(String legacyVersion)
+    {
+        for (int compact = 0; compact <= 1; compact++)
+        {
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact)));
+        }
+        CacheService.instance.invalidateCounterCache();
+        CacheService.instance.invalidateKeyCache();
     }
 
     private static void assertLegacyClustRows(int count, UntypedResultSet rs)
@@ -332,9 +322,9 @@ public class LegacySSTableTest
         }
     }
 
-    private static void loadLegacyTable(String tablePattern, String legacyVersion) throws IOException
+    private static void loadLegacyTable(String tablePattern, String legacyVersion, String compactSuffix) throws IOException
     {
-        String table = String.format(tablePattern, legacyVersion);
+        String table = String.format(tablePattern, legacyVersion, compactSuffix);
 
         logger.info("Loading legacy table {}", table);
 
@@ -349,7 +339,7 @@ public class LegacySSTableTest
     }
 
     /**
-     * Generates sstables for 4 CQL tables (see {@link #createTables(String)}) in <i>current</i>
+     * Generates sstables for 8 CQL tables (see {@link #createTables(String)}) in <i>current</i>
      * sstable format (version) into {@code test/data/legacy-sstables/VERSION}, where
      * {@code VERSION} matches {@link Version#getVersion() BigFormat.latestVersion.getVersion()}.
      * <p>
@@ -357,13 +347,10 @@ public class LegacySSTableTest
      * during development. I.e. remove the {@code @Ignore} annotation temporarily.
      * </p>
      */
-    @Test
     @Ignore
+    @Test
     public void testGenerateSstables() throws Throwable
     {
-        createKeyspace();
-        createTables(BigFormat.latestVersion.getVersion());
-
         Random rand = new Random();
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < 128; i++)
@@ -372,25 +359,28 @@ public class LegacySSTableTest
         }
         String randomString = sb.toString();
 
-        for (int pk = 0; pk < 5; pk++)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            String valPk = Integer.toString(pk);
-            QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple (pk, val) VALUES ('%s', '%s')",
-                                                         BigFormat.latestVersion, valPk, "foo bar baz"));
+            for (int pk = 0; pk < 5; pk++)
+            {
+                String valPk = Integer.toString(pk);
+                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple%s (pk, val) VALUES ('%s', '%s')",
+                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, "foo bar baz"));
 
-            QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter SET val = val + 1 WHERE pk = '%s'",
-                                                         BigFormat.latestVersion, valPk));
+                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter%s SET val = val + 1 WHERE pk = '%s'",
+                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk));
 
-            for (int ck = 0; ck < 50; ck++)
-            {
-                String valCk = Integer.toString(ck);
+                for (int ck = 0; ck < 50; ck++)
+                {
+                    String valCk = Integer.toString(ck);
 
-                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust (pk, ck, val) VALUES ('%s', '%s', '%s')",
-                                                             BigFormat.latestVersion, valPk, valCk + longString, randomString));
+                    QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust%s (pk, ck, val) VALUES ('%s', '%s', '%s')",
+                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString, randomString));
 
-                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
-                                                             BigFormat.latestVersion, valPk, valCk + longString));
+                    QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter%s SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
+                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString));
 
+                }
             }
         }
 
@@ -398,10 +388,13 @@ public class LegacySSTableTest
 
         File ksDir = new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables", BigFormat.latestVersion));
         ksDir.mkdirs();
-        copySstablesFromTestData(String.format("legacy_%s_simple", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_simple_counter", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_clust", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_clust_counter", BigFormat.latestVersion), ksDir);
+        for (int compact = 0; compact <= 1; compact++)
+        {
+            copySstablesFromTestData(String.format("legacy_%s_simple%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_simple_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_clust%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_clust_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+        }
     }
 
     private void copySstablesFromTestData(String table, File ksDir) throws IOException
@@ -420,12 +413,17 @@ public class LegacySSTableTest
 
     private static void copySstablesToTestData(String legacyVersion, String table, File cfDir) throws IOException
     {
-        for (File file : new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table)).listFiles())
+        for (File file : getTableDir(legacyVersion, table).listFiles())
         {
             copyFile(cfDir, file);
         }
     }
 
+    private static File getTableDir(String legacyVersion, String table)
+    {
+        return new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table));
+    }
+
     private static void copyFile(File cfDir, File file) throws IOException
     {
         byte[] buf = new byte[65536];

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index c3a4539..360d262 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -406,7 +406,7 @@ public class BufferedRandomAccessFileTest
             assert tmpFile.getPath().equals(r.getPath());
 
             // Create a mark and move the rw there.
-            final FileMark mark = r.mark();
+            final DataPosition mark = r.mark();
             r.reset(mark);
 
             // Expect this call to succeed.
@@ -457,7 +457,7 @@ public class BufferedRandomAccessFileTest
         RandomAccessReader file = RandomAccessReader.open(channel);
 
         file.seek(10);
-        FileMark mark = file.mark();
+        DataPosition mark = file.mark();
 
         file.seek(file.length());
         assertTrue(file.isEOF());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
index f0d4383..e83c015 100644
--- a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
@@ -356,7 +356,7 @@ public class RandomAccessReaderTest
             assertFalse(reader.isEOF());
             assertEquals((numIterations - 1) * expected.length(), reader.bytesRemaining());
 
-            FileMark mark = reader.mark();
+            DataPosition mark = reader.mark();
             assertEquals(0, reader.bytesPastMark());
             assertEquals(0, reader.bytesPastMark(mark));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
new file mode 100644
index 0000000..175ab53
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
@@ -0,0 +1,539 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class RewindableDataInputStreamPlusTest
+{
+
+    private final int INITIAL_BUFFER_SIZE = 1;
+
+    private File file;
+
+    @Before
+    public void setup() throws Exception
+    {
+        this.file = new File(System.getProperty("java.io.tmpdir"), "subdir/test.buffer");
+    }
+
+    @Test
+    public void testMarkAndResetSimple() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+            // long
+            out.writeLong(1L);
+            // float
+            out.writeFloat(1.0f);
+            // double
+            out.writeDouble(1.0d);
+
+            // String
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+        }
+
+        for (int memCapacity = 0; memCapacity <= 16; memCapacity++)
+        {
+            int diskCapacity = 16 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                try {
+                    //should mark before resetting
+                    reader.reset(null);
+                    fail("Should have thrown IOException");
+                } catch (IOException e) {}
+
+                assertTrue(reader.readBoolean());
+
+                reader.mark();
+
+                try {
+                    //cannot mark already marked stream
+                    reader.mark();
+                    fail("Should have thrown IllegalStateException");
+                } catch (IllegalStateException e) {}
+
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(3, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                try {
+                    //cannot mark when reading from cache
+                    reader.mark();
+                    fail("Should have thrown IllegalStateException");
+                } catch (IllegalStateException e) {}
+
+                //read again previous sequence
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                //finish reading again previous sequence
+                assertEquals(1, reader.readShort());
+
+                reader.mark();
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(16, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                //read again previous sequence
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                assertEquals(1.0f, reader.readFloat(), 0);
+                //finish reading again previous sequence
+
+                //mark again
+                reader.mark();
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals(8, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                //read again previous sequence
+                assertEquals(1.0d, reader.readDouble(), 0);
+                //finish reading again previous sequence
+
+                //mark and reset
+                reader.mark();
+                reader.reset(null);
+
+                assertEquals("abc", reader.readUTF());
+
+                //check max file size
+                assertEquals(diskCapacity, file.length());
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+    @Test
+    public void testVeryLargeCapacity() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+        }
+
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, Integer.MAX_VALUE, file,
+                                                                                      Integer.MAX_VALUE))
+        {
+            reader.mark();
+            assertEquals("abc", reader.readUTF());
+            reader.reset();
+            assertEquals("abc", reader.readUTF());
+        }
+        assertFalse(file.exists());
+
+
+        baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            out.writeBoolean(true);
+            out.writeBoolean(true);
+            testData = baos.toByteArray();
+        }
+    }
+
+    @Test
+    public void testMarkAndResetBigBuffer() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+            // long
+            out.writeLong(1L);
+            // float
+            out.writeFloat(1.0f);
+            // double
+            out.writeDouble(1.0d);
+
+            // String
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+
+            // 1 (boolean) + 1 (byte) + 2 (char) + 2 (short) + 4 (int) + 8 (long)
+            // + 4 (float) + 8 (double) + 5 bytes (utf string)
+        }
+
+        for (int memCapacity = 0; memCapacity <= 18; memCapacity++)
+        {
+            int diskCapacity = 18 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                //read a big amount before resetting
+                reader.mark();
+                assertTrue(reader.readBoolean());
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(1, reader.readShort());
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                reader.reset();
+
+                //read from buffer
+                assertTrue(reader.readBoolean());
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(1, reader.readShort());
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+
+                assertEquals(17, reader.available());
+
+                //mark again
+                reader.mark();
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals("abc", reader.readUTF());
+                reader.reset();
+
+                assertEquals(17, reader.available());
+
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals("abc", reader.readUTF());
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+
+    @Test
+    public void testCircularSpillFile() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+
+            // String
+            out.writeUTF("ab");
+            testData = baos.toByteArray();
+
+            // 1 (boolean) + 1 (byte) + 2 (char) + 2 (short) + 4 (int) + 4 bytes (utf string)
+        }
+
+        //read at most 4 bytes multiple times (and then check file size)
+        int MEM_SIZE = 0;
+        int DISK_SIZE = 4;
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, MEM_SIZE, file,
+                                                                                      DISK_SIZE))
+        {
+            //read 2 bytes and reset
+            reader.mark();
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            assertEquals(2, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            //finish reading again previous sequence
+
+            //read 4 bytes and reset
+            reader.mark();
+            assertEquals('a', reader.readChar());
+            assertEquals(1, reader.readShort());
+            assertEquals(4, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertEquals('a', reader.readChar());
+            assertEquals(1, reader.readShort());
+            //finish reading again previous sequence
+
+            //read 4 bytes and reset
+            reader.mark();
+            assertEquals(1, reader.readInt());
+            assertEquals(4, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertEquals(1, reader.readInt());
+
+            //check max file size
+            assertEquals(DISK_SIZE, file.length());
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testExhaustCapacity() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            testData = baos.toByteArray();
+        }
+
+        //test capacity exhausted when reading more than 4 bytes
+        testCapacityExhausted(testData, 0, 2);
+        testCapacityExhausted(testData, 2, 0);
+        testCapacityExhausted(testData, 1, 1);
+    }
+
+    private void testCapacityExhausted(byte[] testData, int memSize, int diskSize) throws IOException
+    {
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, memSize, file,
+                                                                                      diskSize))
+        {
+            //read 2 bytes and reset
+            reader.mark();
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            assertEquals(2, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            //finish reading again previous sequence
+
+            reader.mark();
+            //read 3 bytes - OK
+            assertEquals('a', reader.readChar());
+            //read 1 more bytes - CAPACITY will exhaust when trying to reset :(
+            assertEquals(1, reader.readShort());
+
+            try
+            {
+                reader.reset();
+                fail("Should have thrown IOException");
+            }
+            catch (IOException e) {}
+
+            //check max file size
+            assertEquals(diskSize, file.length());
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testMarkAndResetUnsignedRead() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // byte
+            out.writeByte(0x1);
+            // short
+            out.writeShort(2);
+            testData = baos.toByteArray();
+        }
+
+        for (int memCapacity = 0; memCapacity <= 1; memCapacity++)
+        {
+            int diskCapacity = 1 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                assertEquals(1, reader.readUnsignedByte());
+                reader.reset();
+                assertEquals(1, reader.readUnsignedByte());
+
+                //will read first byte of short 2
+                reader.mark();
+                assertEquals(0, reader.readUnsignedByte());
+                reader.reset();
+
+                assertEquals(2, reader.readUnsignedShort());
+
+                reader.mark();
+                reader.reset();
+                assertEquals(0, reader.available());
+            }
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testMarkAndResetSkipBytes() throws Exception
+    {
+        String testStr = "1234567890";
+        byte[] testData = testStr.getBytes();
+
+        for (int memCapacity = 0; memCapacity <= 7; memCapacity++)
+        {
+            int diskCapacity = 7 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                // read first 5 bytes and rewind
+                byte[] out = new byte[5];
+                reader.readFully(out, 0, 5);
+                assertEquals("12345", new String(out));
+
+                // then skip 2 bytes (67)
+                reader.skipBytes(2);
+
+                assertEquals(7, reader.bytesPastMark(null));
+                reader.reset();
+
+                //now read part of the previously skipped bytes
+                out = new byte[5];
+                reader.readFully(out);
+                assertEquals("12345", new String(out));
+
+                //skip 3 bytes (2 from cache, 1 from stream)
+                reader.skip(3);
+
+                // mark and read 2 more bytes
+                reader.mark();
+                out = new byte[2];
+                reader.readFully(out);
+                assertEquals("90", new String(out));
+                assertEquals(0, reader.available());
+                reader.reset();
+
+                //reset and read only the next byte "9" in the third position
+                reader.readFully(out, 1, 1);
+                assertEquals("99", new String(out));
+
+                //now we read the remainder via readline
+                assertEquals(1, reader.available());
+                assertEquals("0", reader.readLine());
+
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+    @Test
+    public void testMarkAndResetReadFully() throws Exception
+    {
+        String testStr = "1234567890";
+        byte[] testData = testStr.getBytes();
+
+        for (int memCapacity = 0; memCapacity <= 5; memCapacity++)
+        {
+            int diskCapacity = 5 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                // read first 5 bytes and rewind
+                byte[] out = new byte[5];
+                reader.readFully(out, 0, 5);
+                assertEquals("12345", new String(out));
+                reader.reset();
+
+                // read half from cache, half from parent stream
+                out = new byte[7];
+                reader.readFully(out);
+                assertEquals("1234567", new String(out));
+
+                // mark and read 3 more bytes
+                reader.mark();
+                out = new byte[3];
+                reader.readFully(out);
+                assertEquals("890", new String(out));
+                assertEquals(0, reader.available());
+                reader.reset();
+
+                //reset and read only the next byte "8" in the third position
+                reader.readFully(out, 2, 1);
+                assertEquals("898", new String(out));
+
+                //now we read the remainder via readline
+                assertEquals(2, reader.available());
+                assertEquals("90", reader.readLine());
+            }
+            assertFalse(file.exists());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java b/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
index 221e55c..7693b45 100644
--- a/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
+++ b/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
@@ -18,6 +18,10 @@
  */
 package org.apache.cassandra.utils;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -25,8 +29,10 @@ import java.io.DataOutputStream;
 
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.apache.cassandra.io.util.BytesReadTracker;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
+import org.apache.cassandra.io.util.TrackedInputStream;
 
 public class BytesReadTrackerTest
 {
@@ -34,6 +40,33 @@ public class BytesReadTrackerTest
     @Test
     public void testBytesRead() throws Exception
     {
+        internalTestBytesRead(true);
+        internalTestBytesRead(false);
+    }
+
+    @Test
+    public void testUnsignedRead() throws Exception
+    {
+        internalTestUnsignedRead(true);
+        internalTestUnsignedRead(false);
+    }
+
+    @Test
+    public void testSkipBytesAndReadFully() throws Exception
+    {
+        internalTestSkipBytesAndReadFully(true);
+        internalTestSkipBytesAndReadFully(false);
+    }
+
+    @Test
+    public void testReadLine() throws Exception
+    {
+        internalTestReadLine(true);
+        internalTestReadLine(false);
+    }
+
+    public void internalTestBytesRead(boolean inputStream) throws Exception
+    {
         byte[] testData;
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -66,45 +99,46 @@ public class BytesReadTrackerTest
             out.close();
         }
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // boolean = 1byte
-            boolean bool = tracker.readBoolean();
+            boolean bool = reader.readBoolean();
             assertTrue(bool);
             assertEquals(1, tracker.getBytesRead());
             // byte = 1byte
-            byte b = tracker.readByte();
+            byte b = reader.readByte();
             assertEquals(b, 0x1);
             assertEquals(2, tracker.getBytesRead());
             // char = 2byte
-            char c = tracker.readChar();
+            char c = reader.readChar();
             assertEquals('a', c);
             assertEquals(4, tracker.getBytesRead());
             // short = 2bytes
-            short s = tracker.readShort();
+            short s = reader.readShort();
             assertEquals(1, s);
             assertEquals((short) 6, tracker.getBytesRead());
             // int = 4bytes
-            int i = tracker.readInt();
+            int i = reader.readInt();
             assertEquals(1, i);
             assertEquals(10, tracker.getBytesRead());
             // long = 8bytes
-            long l = tracker.readLong();
+            long l = reader.readLong();
             assertEquals(1L, l);
             assertEquals(18, tracker.getBytesRead());
             // float = 4bytes
-            float f = tracker.readFloat();
+            float f = reader.readFloat();
             assertEquals(1.0f, f, 0);
             assertEquals(22, tracker.getBytesRead());
             // double = 8bytes
-            double d = tracker.readDouble();
+            double d = reader.readDouble();
             assertEquals(1.0d, d, 0);
             assertEquals(30, tracker.getBytesRead());
             // String("abc") = 2(string size) + 3 = 5 bytes
-            String str = tracker.readUTF();
+            String str = reader.readUTF();
             assertEquals("abc", str);
             assertEquals(35, tracker.getBytesRead());
 
@@ -119,8 +153,7 @@ public class BytesReadTrackerTest
         assertEquals(0, tracker.getBytesRead());
     }
 
-    @Test
-    public void testUnsignedRead() throws Exception
+    public void internalTestUnsignedRead(boolean inputStream) throws Exception
     {
         byte[] testData;
 
@@ -139,17 +172,18 @@ public class BytesReadTrackerTest
             out.close();
         }
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // byte = 1byte
-            int b = tracker.readUnsignedByte();
+            int b = reader.readUnsignedByte();
             assertEquals(b, 1);
             assertEquals(1, tracker.getBytesRead());
             // short = 2bytes
-            int s = tracker.readUnsignedShort();
+            int s = reader.readUnsignedShort();
             assertEquals(1, s);
             assertEquals(3, tracker.getBytesRead());
 
@@ -161,30 +195,30 @@ public class BytesReadTrackerTest
         }
     }
 
-    @Test
-    public void testSkipBytesAndReadFully() throws Exception
+    public void internalTestSkipBytesAndReadFully(boolean inputStream) throws Exception
     {
         String testStr = "1234567890";
         byte[] testData = testStr.getBytes();
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // read first 5 bytes
             byte[] out = new byte[5];
-            tracker.readFully(out, 0, 5);
+            reader.readFully(out, 0, 5);
             assertEquals("12345", new String(out));
             assertEquals(5, tracker.getBytesRead());
 
             // then skip 2 bytes
-            tracker.skipBytes(2);
+            reader.skipBytes(2);
             assertEquals(7, tracker.getBytesRead());
 
             // and read the rest
             out = new byte[3];
-            tracker.readFully(out);
+            reader.readFully(out);
             assertEquals("890", new String(out));
             assertEquals(10, tracker.getBytesRead());
 
@@ -196,16 +230,24 @@ public class BytesReadTrackerTest
         }
     }
 
-    @Test(expected = UnsupportedOperationException.class)
-    public void testReadLine() throws Exception
+    public void internalTestReadLine(boolean inputStream) throws Exception
     {
         DataInputStream in = new DataInputStream(new ByteArrayInputStream("1".getBytes()));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
-            // throws UnsupportedOperationException
-            tracker.readLine();
+            String line = reader.readLine();
+            if (inputStream)
+                assertEquals(line, "1");
+            else
+                fail("Should have thrown UnsupportedOperationException");
+        }
+        catch (UnsupportedOperationException e)
+        {
+            if (inputStream)
+                fail("Should have not thrown UnsupportedOperationException");
         }
         finally
         {


[10/12] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.5

Posted by yu...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.5


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/587773fa
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/587773fa
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/587773fa

Branch: refs/heads/trunk
Commit: 587773fa478ff64aa46cf17760eb31d6f83fc46d
Parents: e3716ee e8651b6
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Mar 17 10:42:20 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 17 10:42:20 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   2 +
 .../org/apache/cassandra/db/Directories.java    |  30 +
 .../cassandra/db/SerializationHeader.java       |   5 +
 .../org/apache/cassandra/db/Serializers.java    | 114 ++--
 .../columniterator/AbstractSSTableIterator.java |   4 +-
 .../EncryptedFileSegmentInputStream.java        |   4 +-
 .../cassandra/hints/ChecksummedDataInput.java   |   8 +-
 .../org/apache/cassandra/hints/HintMessage.java |   4 +-
 .../io/compress/CompressedSequentialWriter.java |   8 +-
 .../io/sstable/SSTableSimpleIterator.java       |  11 +-
 .../sstable/format/RangeAwareSSTableWriter.java |   8 +-
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../io/sstable/format/SSTableWriter.java        |   2 +-
 .../io/sstable/format/big/BigTableWriter.java   |   4 +-
 .../cassandra/io/util/BytesReadTracker.java     |  30 +
 .../apache/cassandra/io/util/DataPosition.java  |  21 +
 .../apache/cassandra/io/util/FileDataInput.java |   8 +-
 .../org/apache/cassandra/io/util/FileMark.java  |  20 -
 .../io/util/FileSegmentInputStream.java         |  12 +-
 .../cassandra/io/util/RandomAccessReader.java   |   8 +-
 .../cassandra/io/util/RewindableDataInput.java  |  30 +
 .../io/util/RewindableDataInputStreamPlus.java  | 569 +++++++++++++++++++
 .../cassandra/io/util/SequentialWriter.java     |   6 +-
 .../cassandra/io/util/TrackedDataInputPlus.java | 150 +++++
 .../cassandra/io/util/TrackedInputStream.java   |  76 +++
 .../cassandra/service/StorageService.java       |   1 +
 .../cassandra/streaming/StreamReader.java       |  85 ++-
 .../compress/CompressedStreamReader.java        |  18 +-
 .../streaming/messages/FileMessageHeader.java   |   4 +-
 .../apache/cassandra/tools/nodetool/Repair.java |   2 +-
 .../cassandra/utils/BytesReadTracker.java       | 153 -----
 .../cassandra/utils/CloseableIterator.java      |   1 -
 ...acy_jb_clust_compact-jb-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-Data.db | Bin 0 -> 5270 bytes
 ...ables-legacy_jb_clust_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_jb_clust_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_jb_clust_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...bles-legacy_jb_clust_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-TOC.txt |   7 +
 ...lust_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_jb_clust_counter_compact-jb-1-Data.db | Bin 0 -> 4228 bytes
 ...gacy_jb_clust_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_jb_clust_counter_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ..._jb_clust_counter_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...acy_jb_clust_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...legacy_jb_clust_counter_compact-jb-1-TOC.txt |   7 +
 ...cy_jb_simple_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_jb_simple_compact-jb-1-Data.db | Bin 0 -> 108 bytes
 ...bles-legacy_jb_simple_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_jb_simple_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_jb_simple_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...les-legacy_jb_simple_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...tables-legacy_jb_simple_compact-jb-1-TOC.txt |   7 +
 ...mple_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_jb_simple_counter_compact-jb-1-Data.db | Bin 0 -> 118 bytes
 ...acy_jb_simple_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_jb_simple_counter_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...jb_simple_counter_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...cy_jb_simple_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...egacy_jb_simple_counter_compact-jb-1-TOC.txt |   7 +
 ...acy_ka_clust_compact-ka-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-Data.db | Bin 0 -> 5277 bytes
 ...les-legacy_ka_clust_compact-ka-1-Digest.sha1 |   1 +
 ...ables-legacy_ka_clust_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_ka_clust_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_ka_clust_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...bles-legacy_ka_clust_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-TOC.txt |   8 +
 ...lust_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_ka_clust_counter_compact-ka-1-Data.db | Bin 0 -> 4527 bytes
 ...cy_ka_clust_counter_compact-ka-1-Digest.sha1 |   1 +
 ...gacy_ka_clust_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_ka_clust_counter_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ..._ka_clust_counter_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...acy_ka_clust_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...legacy_ka_clust_counter_compact-ka-1-TOC.txt |   8 +
 ...cy_ka_simple_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_ka_simple_compact-ka-1-Data.db | Bin 0 -> 105 bytes
 ...es-legacy_ka_simple_compact-ka-1-Digest.sha1 |   1 +
 ...bles-legacy_ka_simple_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_ka_simple_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_ka_simple_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...les-legacy_ka_simple_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...tables-legacy_ka_simple_compact-ka-1-TOC.txt |   8 +
 ...mple_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_ka_simple_counter_compact-ka-1-Data.db | Bin 0 -> 124 bytes
 ...y_ka_simple_counter_compact-ka-1-Digest.sha1 |   1 +
 ...acy_ka_simple_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_ka_simple_counter_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...ka_simple_counter_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...cy_ka_simple_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...egacy_ka_simple_counter_compact-ka-1-TOC.txt |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_la_clust_compact/la-1-big-Data.db    | Bin 0 -> 5286 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_clust_compact/la-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_la_clust_compact/la-1-big-Index.db   | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../legacy_la_clust_compact/la-1-big-Summary.db | Bin 0 -> 75 bytes
 .../legacy_la_clust_compact/la-1-big-TOC.txt    |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 4527 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_la_simple_compact/la-1-big-Data.db   | Bin 0 -> 106 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_simple_compact/la-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_la_simple_compact/la-1-big-Index.db  | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../legacy_la_simple_compact/la-1-big-TOC.txt   |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 124 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_ma_clust_compact/ma-1-big-Data.db    | Bin 0 -> 5393 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_clust_compact/ma-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_ma_clust_compact/ma-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7046 bytes
 .../legacy_ma_clust_compact/ma-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_ma_clust_compact/ma-1-big-TOC.txt    |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 4606 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7055 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_ma_simple_compact/ma-1-big-Data.db   | Bin 0 -> 91 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_simple_compact/ma-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_ma_simple_compact/ma-1-big-Index.db  | Bin 0 -> 26 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4640 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_ma_simple_compact/ma-1-big-TOC.txt   |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4649 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../cassandra/AbstractSerializationsTester.java |   1 -
 .../apache/cassandra/db/DirectoriesTest.java    |  98 ++--
 .../cassandra/gms/SerializationsTest.java       |   1 -
 .../CompressedRandomAccessReaderTest.java       |   6 +-
 .../CompressedSequentialWriterTest.java         |   4 +-
 .../cassandra/io/sstable/LegacySSTableTest.java | 369 ++++++------
 .../io/util/BufferedRandomAccessFileTest.java   |   4 +-
 .../io/util/RandomAccessReaderTest.java         |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java | 539 ++++++++++++++++++
 .../cassandra/utils/BytesReadTrackerTest.java   | 104 +++-
 167 files changed, 2115 insertions(+), 550 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a01e511,51cfc16..53dd292
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,6 -1,6 +1,7 @@@
 -3.0.5
 +3.5
 +Merged from 3.0:
+  * Support streaming pre-3.0 sstables (CASSANDRA-10990)
 - * Add backpressure to compressed commit log (CASSANDRA-10971)
 + * Add backpressure to compressed or encrypted commit log (CASSANDRA-10971)
   * SSTableExport supports secondary index tables (CASSANDRA-11330)
   * Fix sstabledump to include missing info in debug output (CASSANDRA-11321)
   * Establish and implement canonical bulk reading workload(s) (CASSANDRA-10331)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/Serializers.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Serializers.java
index 17f1de0,348fda3..cef06a3
--- a/src/java/org/apache/cassandra/db/Serializers.java
+++ b/src/java/org/apache/cassandra/db/Serializers.java
@@@ -46,62 -46,77 +46,77 @@@ public class Serializer
      // unecessary (since IndexInfo.Serializer won't depend on the metadata either).
      public ISerializer<ClusteringPrefix> indexEntryClusteringPrefixSerializer(final Version version, final SerializationHeader header)
      {
-         if (!version.storeRows())
+         if (!version.storeRows() || header ==  null) //null header indicates streaming from pre-3.0 sstables
          {
-             return new ISerializer<ClusteringPrefix>()
+             return oldFormatSerializer(version);
+         }
+ 
+         return newFormatSerializer(version, header);
+     }
+ 
+     private ISerializer<ClusteringPrefix> oldFormatSerializer(final Version version)
+     {
+         return new ISerializer<ClusteringPrefix>()
+         {
+             SerializationHeader newHeader = SerializationHeader.makeWithoutStats(metadata);
+ 
+             public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
              {
-                 public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
-                 {
-                     // We should only use this for reading old sstable, never write new ones.
-                     throw new UnsupportedOperationException();
-                 }
+                 //we deserialize in the old format and serialize in the new format
+                 ClusteringPrefix.serializer.serialize(clustering, out,
+                                                       version.correspondingMessagingVersion(),
+                                                       newHeader.clusteringTypes());
+             }
+ 
+             public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+             {
+                 // We're reading the old cellname/composite
+                 ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
+                 assert bb.hasRemaining(); // empty cellnames were invalid
+ 
+                 int clusteringSize = metadata.clusteringColumns().size();
+                 // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
+                 if (clusteringSize == 0)
+                     return Clustering.EMPTY;
+ 
+                 if (!metadata.isCompound())
 -                    return new Clustering(bb);
++                    return Clustering.make(bb);
  
-                 public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+                 List<ByteBuffer> components = CompositeType.splitName(bb);
+                 byte eoc = CompositeType.lastEOC(bb);
+ 
+                 if (eoc == 0 || components.size() >= clusteringSize)
                  {
-                     // We're reading the old cellname/composite
-                     ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
-                     assert bb.hasRemaining(); // empty cellnames were invalid
- 
-                     int clusteringSize = metadata.clusteringColumns().size();
-                     // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
-                     if (clusteringSize == 0)
-                         return Clustering.EMPTY;
- 
-                     if (!metadata.isCompound())
-                         return Clustering.make(bb);
- 
-                     List<ByteBuffer> components = CompositeType.splitName(bb);
-                     byte eoc = CompositeType.lastEOC(bb);
- 
-                     if (eoc == 0 || components.size() >= clusteringSize)
-                     {
-                         // That's a clustering.
-                         if (components.size() > clusteringSize)
-                             components = components.subList(0, clusteringSize);
- 
-                         return Clustering.make(components.toArray(new ByteBuffer[clusteringSize]));
-                     }
-                     else
-                     {
-                         // It's a range tombstone bound. It is a start since that's the only part we've ever included
-                         // in the index entries.
-                         Slice.Bound.Kind boundKind = eoc > 0
-                                                    ? Slice.Bound.Kind.EXCL_START_BOUND
-                                                    : Slice.Bound.Kind.INCL_START_BOUND;
- 
-                         return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
-                     }
-                 }
+                     // That's a clustering.
+                     if (components.size() > clusteringSize)
+                         components = components.subList(0, clusteringSize);
  
-                 public long serializedSize(ClusteringPrefix clustering)
 -                    return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
++                    return Clustering.make(components.toArray(new ByteBuffer[clusteringSize]));
+                 }
+                 else
                  {
-                     // We should only use this for reading old sstable, never write new ones.
-                     throw new UnsupportedOperationException();
+                     // It's a range tombstone bound. It is a start since that's the only part we've ever included
+                     // in the index entries.
+                     Slice.Bound.Kind boundKind = eoc > 0
+                                                  ? Slice.Bound.Kind.EXCL_START_BOUND
+                                                  : Slice.Bound.Kind.INCL_START_BOUND;
+ 
+                     return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
                  }
-             };
-         }
+             }
  
-         return new ISerializer<ClusteringPrefix>()
+             public long serializedSize(ClusteringPrefix clustering)
+             {
+                 return ClusteringPrefix.serializer.serializedSize(clustering, version.correspondingMessagingVersion(),
+                                                                   newHeader.clusteringTypes());
+             }
+         };
+     }
+ 
+ 
+     private ISerializer<ClusteringPrefix> newFormatSerializer(final Version version, final SerializationHeader header)
+     {
+         return new ISerializer<ClusteringPrefix>() //Reading and writing from/to the new sstable format
          {
              public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
              {
@@@ -119,4 -134,5 +134,5 @@@
              }
          };
      }
- }
+ 
 -}
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index d55161b,0e2012e..7f2e3bb
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@@ -29,10 -30,10 +29,10 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.cassandra.io.sstable.CorruptSSTableException;
  import org.apache.cassandra.io.sstable.IndexHelper;
  import org.apache.cassandra.io.util.FileDataInput;
- import org.apache.cassandra.io.util.FileMark;
+ import org.apache.cassandra.io.util.DataPosition;
  import org.apache.cassandra.utils.ByteBufferUtil;
  
 -abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
 +abstract class AbstractSSTableIterator implements UnfilteredRowIterator
  {
      protected final SSTableReader sstable;
      protected final DecoratedKey key;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
index 6915196,0000000..56bb7d6
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
+++ b/src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
@@@ -1,73 -1,0 +1,73 @@@
 +package org.apache.cassandra.db.commitlog;
 +
 +import java.io.DataInput;
 +import java.nio.ByteBuffer;
 +
++import org.apache.cassandra.io.util.DataPosition;
 +import org.apache.cassandra.io.util.FileDataInput;
- import org.apache.cassandra.io.util.FileMark;
 +import org.apache.cassandra.io.util.FileSegmentInputStream;
 +
 +/**
 + * Each segment of an encrypted file may contain many encrypted chunks, and each chunk needs to be individually decrypted
 + * to reconstruct the full segment.
 + */
 +public class EncryptedFileSegmentInputStream extends FileSegmentInputStream implements FileDataInput, DataInput
 +{
 +    private final long segmentOffset;
 +    private final int expectedLength;
 +    private final ChunkProvider chunkProvider;
 +
 +    /**
 +     * offset the decrypted chunks already processed in this segment.
 +     */
 +    private int totalChunkOffset;
 +
 +    public EncryptedFileSegmentInputStream(String filePath, long segmentOffset, int position, int expectedLength, ChunkProvider chunkProvider)
 +    {
 +        super(chunkProvider.nextChunk(), filePath, position);
 +        this.segmentOffset = segmentOffset;
 +        this.expectedLength = expectedLength;
 +        this.chunkProvider = chunkProvider;
 +    }
 +
 +    public interface ChunkProvider
 +    {
 +        /**
 +         * Get the next chunk from the backing provider, if any chunks remain.
 +         * @return Next chunk, else null if no more chunks remain.
 +         */
 +        ByteBuffer nextChunk();
 +    }
 +
 +    public long getFilePointer()
 +    {
 +        return segmentOffset + totalChunkOffset + buffer.position();
 +    }
 +
 +    public boolean isEOF()
 +    {
 +        return totalChunkOffset + buffer.position() >= expectedLength;
 +    }
 +
 +    public long bytesRemaining()
 +    {
 +        return expectedLength - (totalChunkOffset + buffer.position());
 +    }
 +
 +    public void seek(long position)
 +    {
 +        // implement this when we actually need it
 +        throw new UnsupportedOperationException();
 +    }
 +
-     public long bytesPastMark(FileMark mark)
++    public long bytesPastMark(DataPosition mark)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    public void reBuffer()
 +    {
 +        totalChunkOffset += buffer.position();
 +        buffer = chunkProvider.nextChunk();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
index 674ed7f,0000000..9fcdfa4
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
@@@ -1,205 -1,0 +1,205 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.sstable.format;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.DecoratedKey;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.PartitionPosition;
 +import org.apache.cassandra.db.SerializationHeader;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 +import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 +import org.apache.cassandra.service.StorageService;
 +
 +public class RangeAwareSSTableWriter implements SSTableMultiWriter
 +{
 +    private final List<PartitionPosition> boundaries;
 +    private final Directories.DataDirectory[] directories;
 +    private final int sstableLevel;
 +    private final long estimatedKeys;
 +    private final long repairedAt;
 +    private final SSTableFormat.Type format;
-     private final SerializationHeader.Component header;
++    private final SerializationHeader header;
 +    private final LifecycleTransaction txn;
 +    private int currentIndex = -1;
 +    public final ColumnFamilyStore cfs;
 +    private final List<SSTableMultiWriter> finishedWriters = new ArrayList<>();
 +    private final List<SSTableReader> finishedReaders = new ArrayList<>();
 +    private SSTableMultiWriter currentWriter = null;
 +
-     public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, SSTableFormat.Type format, int sstableLevel, long totalSize, LifecycleTransaction txn, SerializationHeader.Component header) throws IOException
++    public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, SSTableFormat.Type format, int sstableLevel, long totalSize, LifecycleTransaction txn, SerializationHeader header) throws IOException
 +    {
 +        directories = cfs.getDirectories().getWriteableLocations();
 +        this.sstableLevel = sstableLevel;
 +        this.cfs = cfs;
 +        this.estimatedKeys = estimatedKeys / directories.length;
 +        this.repairedAt = repairedAt;
 +        this.format = format;
 +        this.txn = txn;
 +        this.header = header;
 +        boundaries = StorageService.getDiskBoundaries(cfs, directories);
 +        if (boundaries == null)
 +        {
 +            Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize);
 +            if (localDir == null)
 +                throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
 +            Descriptor desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
-             currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), txn);
++            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header, txn);
 +        }
 +    }
 +
 +    private void maybeSwitchWriter(DecoratedKey key)
 +    {
 +        if (boundaries == null)
 +            return;
 +
 +        boolean switched = false;
 +        while (currentIndex < 0 || key.compareTo(boundaries.get(currentIndex)) > 0)
 +        {
 +            switched = true;
 +            currentIndex++;
 +        }
 +
 +        if (switched)
 +        {
 +            if (currentWriter != null)
 +                finishedWriters.add(currentWriter);
 +
 +            Descriptor desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(directories[currentIndex])), format);
-             currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), txn);
++            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header, txn);
 +        }
 +    }
 +
 +    public boolean append(UnfilteredRowIterator partition)
 +    {
 +        maybeSwitchWriter(partition.partitionKey());
 +        return currentWriter.append(partition);
 +    }
 +
 +    @Override
 +    public Collection<SSTableReader> finish(long repairedAt, long maxDataAge, boolean openResult)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter writer : finishedWriters)
 +        {
 +            if (writer.getFilePointer() > 0)
 +                finishedReaders.addAll(writer.finish(repairedAt, maxDataAge, openResult));
 +            else
 +                SSTableMultiWriter.abortOrDie(writer);
 +        }
 +        return finishedReaders;
 +    }
 +
 +    @Override
 +    public Collection<SSTableReader> finish(boolean openResult)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter writer : finishedWriters)
 +        {
 +            if (writer.getFilePointer() > 0)
 +                finishedReaders.addAll(writer.finish(openResult));
 +            else
 +                SSTableMultiWriter.abortOrDie(writer);
 +        }
 +        return finishedReaders;
 +    }
 +
 +    @Override
 +    public Collection<SSTableReader> finished()
 +    {
 +        return finishedReaders;
 +    }
 +
 +    @Override
 +    public SSTableMultiWriter setOpenResult(boolean openResult)
 +    {
 +        finishedWriters.forEach((w) -> w.setOpenResult(openResult));
 +        currentWriter.setOpenResult(openResult);
 +        return this;
 +    }
 +
 +    public String getFilename()
 +    {
 +        return String.join("/", cfs.keyspace.getName(), cfs.getTableName());
 +    }
 +
 +    @Override
 +    public long getFilePointer()
 +    {
 +        return currentWriter.getFilePointer();
 +    }
 +
 +    @Override
 +    public UUID getCfId()
 +    {
 +        return currentWriter.getCfId();
 +    }
 +
 +    @Override
 +    public Throwable commit(Throwable accumulate)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter writer : finishedWriters)
 +            accumulate = writer.commit(accumulate);
 +        return accumulate;
 +    }
 +
 +    @Override
 +    public Throwable abort(Throwable accumulate)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter finishedWriter : finishedWriters)
 +            accumulate = finishedWriter.abort(accumulate);
 +
 +        return accumulate;
 +    }
 +
 +    @Override
 +    public void prepareToCommit()
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        finishedWriters.forEach(SSTableMultiWriter::prepareToCommit);
 +    }
 +
 +    @Override
 +    public void close()
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        finishedWriters.forEach(SSTableMultiWriter::close);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index ab38ba9,5f35029..6aaf776
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@@ -83,9 -80,8 +83,9 @@@ public abstract class SSTableWriter ext
          this.keyCount = keyCount;
          this.repairedAt = repairedAt;
          this.metadataCollector = metadataCollector;
-         this.header = header;
+         this.header = header != null ? header : SerializationHeader.makeWithoutStats(metadata); //null header indicates streaming from pre-3.0 sstable
          this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, descriptor.version, header);
 +        this.observers = observers == null ? Collections.emptySet() : observers;
      }
  
      public static SSTableWriter create(Descriptor descriptor,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReader.java
index 7348027,f8db26b..7d7cf8a
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@@ -35,15 -35,18 +35,18 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.rows.*;
 -import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.SSTableMultiWriter;
  import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
 +import org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter;
  import org.apache.cassandra.io.sstable.format.SSTableFormat;
  import org.apache.cassandra.io.sstable.format.Version;
+ import org.apache.cassandra.io.util.RewindableDataInputStreamPlus;
  import org.apache.cassandra.io.util.DataInputPlus;
+ import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.streaming.messages.FileMessageHeader;
  import org.apache.cassandra.utils.ByteBufferUtil;
- import org.apache.cassandra.utils.BytesReadTracker;
+ import org.apache.cassandra.io.util.TrackedInputStream;
+ import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.Pair;
  
  
@@@ -124,7 -129,7 +127,7 @@@ public class StreamReade
          {
              if (deserializer != null)
                  logger.warn("[Stream {}] Error while reading partition {} from stream on ks='{}' and table='{}'.",
--                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getColumnFamilyName());
++                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getTableName());
              if (writer != null)
              {
                  writer.abort(e);
@@@ -142,10 -157,9 +155,10 @@@
          Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize);
          if (localDir == null)
              throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
 -        desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
  
-         RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, format, sstableLevel, totalSize, session.getTransaction(cfId), header);
 -        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, getHeader(cfs.metadata), session.getTransaction(cfId));
++        RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, format, sstableLevel, totalSize, session.getTransaction(cfId), getHeader(cfs.metadata));
 +        StreamHook.instance.reportIncomingFile(cfs, writer, session, fileSeqNum);
 +        return writer;
      }
  
      protected void drain(InputStream dis, long bytesRead) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index 5a47787,9719587..318484f
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@@ -24,9 -24,7 +24,8 @@@ import java.nio.channels.ReadableByteCh
  
  import com.google.common.base.Throwables;
  
- import org.apache.cassandra.db.DecoratedKey;
  import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 +import org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;


[06/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
Support streaming of older version sstables in 3.0

patch by Paulo Motta; reviewed by Yuki Morishita for CASSANDRA-10990


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e8651b66
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e8651b66
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e8651b66

Branch: refs/heads/cassandra-3.5
Commit: e8651b6625c7f6260852f2a9c45fb189c63ab528
Parents: 7f1339c
Author: Paulo Motta <pa...@gmail.com>
Authored: Fri Feb 5 12:38:39 2016 -0300
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 17 10:04:44 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   2 +
 .../org/apache/cassandra/db/Directories.java    |  30 +
 .../cassandra/db/SerializationHeader.java       |   5 +
 .../org/apache/cassandra/db/Serializers.java    | 114 ++--
 .../columniterator/AbstractSSTableIterator.java |   4 +-
 .../cassandra/hints/ChecksummedDataInput.java   |   8 +-
 .../org/apache/cassandra/hints/HintMessage.java |   4 +-
 .../io/compress/CompressedSequentialWriter.java |   8 +-
 .../io/sstable/SSTableSimpleIterator.java       |  11 +-
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../io/sstable/format/SSTableWriter.java        |   2 +-
 .../io/sstable/format/big/BigTableWriter.java   |   4 +-
 .../cassandra/io/util/BytesReadTracker.java     |  30 +
 .../apache/cassandra/io/util/DataPosition.java  |  21 +
 .../apache/cassandra/io/util/FileDataInput.java |   8 +-
 .../org/apache/cassandra/io/util/FileMark.java  |  20 -
 .../io/util/FileSegmentInputStream.java         |   6 +-
 .../cassandra/io/util/RandomAccessReader.java   |   8 +-
 .../cassandra/io/util/RewindableDataInput.java  |  30 +
 .../io/util/RewindableDataInputStreamPlus.java  | 569 +++++++++++++++++++
 .../cassandra/io/util/SequentialWriter.java     |   6 +-
 .../cassandra/io/util/TrackedDataInputPlus.java | 150 +++++
 .../cassandra/io/util/TrackedInputStream.java   |  76 +++
 .../cassandra/service/StorageService.java       |   1 +
 .../cassandra/streaming/StreamReader.java       |  84 ++-
 .../compress/CompressedStreamReader.java        |  18 +-
 .../streaming/messages/FileMessageHeader.java   |   4 +-
 .../apache/cassandra/tools/nodetool/Repair.java |   2 +-
 .../cassandra/utils/BytesReadTracker.java       | 153 -----
 .../cassandra/utils/CloseableIterator.java      |   1 -
 ...acy_jb_clust_compact-jb-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-Data.db | Bin 0 -> 5270 bytes
 ...ables-legacy_jb_clust_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_jb_clust_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_jb_clust_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...bles-legacy_jb_clust_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-TOC.txt |   7 +
 ...lust_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_jb_clust_counter_compact-jb-1-Data.db | Bin 0 -> 4228 bytes
 ...gacy_jb_clust_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_jb_clust_counter_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ..._jb_clust_counter_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...acy_jb_clust_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...legacy_jb_clust_counter_compact-jb-1-TOC.txt |   7 +
 ...cy_jb_simple_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_jb_simple_compact-jb-1-Data.db | Bin 0 -> 108 bytes
 ...bles-legacy_jb_simple_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_jb_simple_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_jb_simple_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...les-legacy_jb_simple_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...tables-legacy_jb_simple_compact-jb-1-TOC.txt |   7 +
 ...mple_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_jb_simple_counter_compact-jb-1-Data.db | Bin 0 -> 118 bytes
 ...acy_jb_simple_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_jb_simple_counter_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...jb_simple_counter_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...cy_jb_simple_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...egacy_jb_simple_counter_compact-jb-1-TOC.txt |   7 +
 ...acy_ka_clust_compact-ka-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-Data.db | Bin 0 -> 5277 bytes
 ...les-legacy_ka_clust_compact-ka-1-Digest.sha1 |   1 +
 ...ables-legacy_ka_clust_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_ka_clust_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_ka_clust_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...bles-legacy_ka_clust_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-TOC.txt |   8 +
 ...lust_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_ka_clust_counter_compact-ka-1-Data.db | Bin 0 -> 4527 bytes
 ...cy_ka_clust_counter_compact-ka-1-Digest.sha1 |   1 +
 ...gacy_ka_clust_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_ka_clust_counter_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ..._ka_clust_counter_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...acy_ka_clust_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...legacy_ka_clust_counter_compact-ka-1-TOC.txt |   8 +
 ...cy_ka_simple_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_ka_simple_compact-ka-1-Data.db | Bin 0 -> 105 bytes
 ...es-legacy_ka_simple_compact-ka-1-Digest.sha1 |   1 +
 ...bles-legacy_ka_simple_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_ka_simple_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_ka_simple_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...les-legacy_ka_simple_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...tables-legacy_ka_simple_compact-ka-1-TOC.txt |   8 +
 ...mple_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_ka_simple_counter_compact-ka-1-Data.db | Bin 0 -> 124 bytes
 ...y_ka_simple_counter_compact-ka-1-Digest.sha1 |   1 +
 ...acy_ka_simple_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_ka_simple_counter_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...ka_simple_counter_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...cy_ka_simple_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...egacy_ka_simple_counter_compact-ka-1-TOC.txt |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_la_clust_compact/la-1-big-Data.db    | Bin 0 -> 5286 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_clust_compact/la-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_la_clust_compact/la-1-big-Index.db   | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../legacy_la_clust_compact/la-1-big-Summary.db | Bin 0 -> 75 bytes
 .../legacy_la_clust_compact/la-1-big-TOC.txt    |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 4527 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_la_simple_compact/la-1-big-Data.db   | Bin 0 -> 106 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_simple_compact/la-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_la_simple_compact/la-1-big-Index.db  | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../legacy_la_simple_compact/la-1-big-TOC.txt   |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 124 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_ma_clust_compact/ma-1-big-Data.db    | Bin 0 -> 5393 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_clust_compact/ma-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_ma_clust_compact/ma-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7046 bytes
 .../legacy_ma_clust_compact/ma-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_ma_clust_compact/ma-1-big-TOC.txt    |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 4606 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7055 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_ma_simple_compact/ma-1-big-Data.db   | Bin 0 -> 91 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_simple_compact/ma-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_ma_simple_compact/ma-1-big-Index.db  | Bin 0 -> 26 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4640 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_ma_simple_compact/ma-1-big-TOC.txt   |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4649 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../cassandra/AbstractSerializationsTester.java |   1 -
 .../apache/cassandra/db/DirectoriesTest.java    |  98 ++--
 .../cassandra/gms/SerializationsTest.java       |   1 -
 .../CompressedRandomAccessReaderTest.java       |   6 +-
 .../CompressedSequentialWriterTest.java         |   4 +-
 .../cassandra/io/sstable/LegacySSTableTest.java | 368 ++++++------
 .../io/util/BufferedRandomAccessFileTest.java   |   4 +-
 .../io/util/RandomAccessReaderTest.java         |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java | 539 ++++++++++++++++++
 .../cassandra/utils/BytesReadTrackerTest.java   | 104 +++-
 165 files changed, 2101 insertions(+), 544 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index defc25a..51cfc16 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.5
+ * Support streaming pre-3.0 sstables (CASSANDRA-10990)
  * Add backpressure to compressed commit log (CASSANDRA-10971)
  * SSTableExport supports secondary index tables (CASSANDRA-11330)
  * Fix sstabledump to include missing info in debug output (CASSANDRA-11321)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c564d8d..f28df1c 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -571,6 +571,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
          // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
         clearEphemeralSnapshots(directories);
 
+        directories.removeTemporaryDirectories();
+
         logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.cfName);
         LifecycleTransaction.removeUnfinishedLeftovers(metadata);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 8744d43..83321ac 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -94,9 +94,11 @@ public class Directories
 
     public static final String BACKUPS_SUBDIR = "backups";
     public static final String SNAPSHOT_SUBDIR = "snapshots";
+    public static final String TMP_SUBDIR = "tmp";
     public static final String SECONDARY_INDEX_NAME_SEPARATOR = ".";
 
     public static final DataDirectory[] dataDirectories;
+
     static
     {
         String[] locations = DatabaseDescriptor.getAllDataFileLocations();
@@ -322,6 +324,34 @@ public class Directories
     }
 
     /**
+     * Returns a temporary subdirectory on non-blacklisted data directory
+     * that _currently_ has {@code writeSize} bytes as usable space.
+     * This method does not create the temporary directory.
+     *
+     * @throws IOError if all directories are blacklisted.
+     */
+    public File getTemporaryWriteableDirectoryAsFile(long writeSize)
+    {
+        File location = getLocationForDisk(getWriteableLocation(writeSize));
+        if (location == null)
+            return null;
+        return new File(location, TMP_SUBDIR);
+    }
+
+    public void removeTemporaryDirectories()
+    {
+        for (File dataDir : dataPaths)
+        {
+            File tmpDir = new File(dataDir, TMP_SUBDIR);
+            if (tmpDir.exists())
+            {
+                logger.debug("Removing temporary directory {}", tmpDir);
+                FileUtils.deleteRecursive(tmpDir);
+            }
+        }
+    }
+
+    /**
      * Returns a non-blacklisted data directory that _currently_ has {@code writeSize} bytes as usable space.
      *
      * @throws IOError if all directories are blacklisted.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/SerializationHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index 6e03756..0fd1281 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -70,6 +70,11 @@ public class SerializationHeader
         this.typeMap = typeMap;
     }
 
+    public static SerializationHeader makeWithoutStats(CFMetaData metadata)
+    {
+        return new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
+    }
+
     public static SerializationHeader forKeyCache(CFMetaData metadata)
     {
         // We don't save type information in the key cache (we could change

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/Serializers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Serializers.java b/src/java/org/apache/cassandra/db/Serializers.java
index 9b29d89..348fda3 100644
--- a/src/java/org/apache/cassandra/db/Serializers.java
+++ b/src/java/org/apache/cassandra/db/Serializers.java
@@ -29,8 +29,6 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
-
 /**
  * Holds references on serializers that depend on the table definition.
  */
@@ -48,62 +46,77 @@ public class Serializers
     // unecessary (since IndexInfo.Serializer won't depend on the metadata either).
     public ISerializer<ClusteringPrefix> indexEntryClusteringPrefixSerializer(final Version version, final SerializationHeader header)
     {
-        if (!version.storeRows())
+        if (!version.storeRows() || header ==  null) //null header indicates streaming from pre-3.0 sstables
         {
-            return new ISerializer<ClusteringPrefix>()
+            return oldFormatSerializer(version);
+        }
+
+        return newFormatSerializer(version, header);
+    }
+
+    private ISerializer<ClusteringPrefix> oldFormatSerializer(final Version version)
+    {
+        return new ISerializer<ClusteringPrefix>()
+        {
+            SerializationHeader newHeader = SerializationHeader.makeWithoutStats(metadata);
+
+            public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
             {
-                public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
-                {
-                    // We should only use this for reading old sstable, never write new ones.
-                    throw new UnsupportedOperationException();
-                }
+                //we deserialize in the old format and serialize in the new format
+                ClusteringPrefix.serializer.serialize(clustering, out,
+                                                      version.correspondingMessagingVersion(),
+                                                      newHeader.clusteringTypes());
+            }
+
+            public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+            {
+                // We're reading the old cellname/composite
+                ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
+                assert bb.hasRemaining(); // empty cellnames were invalid
+
+                int clusteringSize = metadata.clusteringColumns().size();
+                // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
+                if (clusteringSize == 0)
+                    return Clustering.EMPTY;
 
-                public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+                if (!metadata.isCompound())
+                    return new Clustering(bb);
+
+                List<ByteBuffer> components = CompositeType.splitName(bb);
+                byte eoc = CompositeType.lastEOC(bb);
+
+                if (eoc == 0 || components.size() >= clusteringSize)
                 {
-                    // We're reading the old cellname/composite
-                    ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
-                    assert bb.hasRemaining(); // empty cellnames were invalid
-
-                    int clusteringSize = metadata.clusteringColumns().size();
-                    // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
-                    if (clusteringSize == 0)
-                        return Clustering.EMPTY;
-
-                    if (!metadata.isCompound())
-                        return new Clustering(bb);
-
-                    List<ByteBuffer> components = CompositeType.splitName(bb);
-                    byte eoc = CompositeType.lastEOC(bb);
-
-                    if (eoc == 0 || components.size() >= clusteringSize)
-                    {
-                        // That's a clustering.
-                        if (components.size() > clusteringSize)
-                            components = components.subList(0, clusteringSize);
-
-                        return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
-                    }
-                    else
-                    {
-                        // It's a range tombstone bound. It is a start since that's the only part we've ever included
-                        // in the index entries.
-                        Slice.Bound.Kind boundKind = eoc > 0
-                                                   ? Slice.Bound.Kind.EXCL_START_BOUND
-                                                   : Slice.Bound.Kind.INCL_START_BOUND;
-
-                        return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
-                    }
-                }
+                    // That's a clustering.
+                    if (components.size() > clusteringSize)
+                        components = components.subList(0, clusteringSize);
 
-                public long serializedSize(ClusteringPrefix clustering)
+                    return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
+                }
+                else
                 {
-                    // We should only use this for reading old sstable, never write new ones.
-                    throw new UnsupportedOperationException();
+                    // It's a range tombstone bound. It is a start since that's the only part we've ever included
+                    // in the index entries.
+                    Slice.Bound.Kind boundKind = eoc > 0
+                                                 ? Slice.Bound.Kind.EXCL_START_BOUND
+                                                 : Slice.Bound.Kind.INCL_START_BOUND;
+
+                    return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
                 }
-            };
-        }
+            }
 
-        return new ISerializer<ClusteringPrefix>()
+            public long serializedSize(ClusteringPrefix clustering)
+            {
+                return ClusteringPrefix.serializer.serializedSize(clustering, version.correspondingMessagingVersion(),
+                                                                  newHeader.clusteringTypes());
+            }
+        };
+    }
+
+
+    private ISerializer<ClusteringPrefix> newFormatSerializer(final Version version, final SerializationHeader header)
+    {
+        return new ISerializer<ClusteringPrefix>() //Reading and writing from/to the new sstable format
         {
             public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
             {
@@ -121,4 +134,5 @@ public class Serializers
             }
         };
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index 8ac3dcb..0e2012e 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
@@ -401,7 +401,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         private int currentIndexIdx;
 
         // Marks the beginning of the block corresponding to currentIndexIdx.
-        private FileMark mark;
+        private DataPosition mark;
 
         public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
index 1dc6d1e..095d7f4 100644
--- a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
+++ b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
@@ -22,13 +22,9 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.zip.CRC32;
 
-import org.apache.cassandra.io.FSError;
-import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.compress.ICompressor;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.schema.CompressionParams;
 
 /**
  * A {@link RandomAccessReader} wrapper that calctulates the CRC in place.
@@ -48,7 +44,7 @@ public class ChecksummedDataInput extends RandomAccessReader.RandomAccessReaderW
     private boolean crcUpdateDisabled;
 
     private long limit;
-    private FileMark limitMark;
+    private DataPosition limitMark;
 
     protected ChecksummedDataInput(Builder builder)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/hints/HintMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintMessage.java b/src/java/org/apache/cassandra/hints/HintMessage.java
index e78738d..723ab6d 100644
--- a/src/java/org/apache/cassandra/hints/HintMessage.java
+++ b/src/java/org/apache/cassandra/hints/HintMessage.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 /**
@@ -117,7 +117,7 @@ public final class HintMessage
             UUID hostId = UUIDSerializer.serializer.deserialize(in, version);
 
             long hintSize = in.readUnsignedVInt();
-            BytesReadTracker countingIn = new BytesReadTracker(in);
+            TrackedDataInputPlus countingIn = new TrackedDataInputPlus(in);
             try
             {
                 return new HintMessage(hostId, Hint.serializer.deserialize(countingIn, version));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 14f1ba7..9bd1145 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.SequentialWriter;
 import org.apache.cassandra.schema.CompressionParams;
@@ -153,7 +153,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     }
 
     @Override
-    public FileMark mark()
+    public DataPosition mark()
     {
         if (!buffer.hasRemaining())
             doFlush(0);
@@ -161,7 +161,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     }
 
     @Override
-    public synchronized void resetAndTruncate(FileMark mark)
+    public synchronized void resetAndTruncate(DataPosition mark)
     {
         assert mark instanceof CompressedFileWriterMark;
 
@@ -306,7 +306,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class CompressedFileWriterMark implements FileMark
+    protected static class CompressedFileWriterMark implements DataPosition
     {
         // chunk offset in the compressed file
         final long chunkOffset;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
index 365d469..f82db4e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.IOError;
 import java.util.Iterator;
 
+import org.apache.cassandra.io.util.RewindableDataInput;
 import org.apache.cassandra.utils.AbstractIterator;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -29,7 +30,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.net.MessagingService;
 
 /**
@@ -113,11 +114,9 @@ public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered>
                 // need to extract them. Which imply 2 passes (one to extract the static, then one for other value).
                 if (metadata.isStaticCompactTable())
                 {
-                    // Because we don't support streaming from old file version, the only case we should get there is for compaction,
-                    // where the DataInput should be a file based one.
-                    assert in instanceof FileDataInput;
-                    FileDataInput file = (FileDataInput)in;
-                    FileMark mark = file.mark();
+                    assert in instanceof RewindableDataInput;
+                    RewindableDataInput file = (RewindableDataInput)in;
+                    DataPosition mark = file.mark();
                     Row staticRow = LegacyLayout.extractStaticColumns(metadata, file, metadata.partitionColumns().statics);
                     file.reset(mark);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 8a778b7..b9561ec 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -427,7 +427,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                                              System.currentTimeMillis(),
                                              statsMetadata,
                                              OpenReason.NORMAL,
-                                             header.toHeader(metadata));
+                                             header == null? null : header.toHeader(metadata));
 
         // special implementation of load to use non-pooled SegmentedFile builders
         try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 4cbbd70..5f35029 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -80,7 +80,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
         this.keyCount = keyCount;
         this.repairedAt = repairedAt;
         this.metadataCollector = metadataCollector;
-        this.header = header;
+        this.header = header != null ? header : SerializationHeader.makeWithoutStats(metadata); //null header indicates streaming from pre-3.0 sstable
         this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, descriptor.version, header);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index 2335e47..d3630d7 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -55,7 +55,7 @@ public class BigTableWriter extends SSTableWriter
     private final SegmentedFile.Builder dbuilder;
     protected final SequentialWriter dataFile;
     private DecoratedKey lastWrittenKey;
-    private FileMark dataMark;
+    private DataPosition dataMark;
 
     public BigTableWriter(Descriptor descriptor, 
                           Long keyCount, 
@@ -368,7 +368,7 @@ public class BigTableWriter extends SSTableWriter
         public final SegmentedFile.Builder builder;
         public final IndexSummaryBuilder summary;
         public final IFilter bf;
-        private FileMark mark;
+        private DataPosition mark;
 
         IndexWriter(long keyCount, final SequentialWriter dataFile)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BytesReadTracker.java b/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
new file mode 100644
index 0000000..fc83856
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+public interface BytesReadTracker
+{
+    public long getBytesRead();
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count);
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/DataPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/DataPosition.java b/src/java/org/apache/cassandra/io/util/DataPosition.java
new file mode 100644
index 0000000..e106dae
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/DataPosition.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+public interface DataPosition
+{}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileDataInput.java b/src/java/org/apache/cassandra/io/util/FileDataInput.java
index f56193b..1059b01 100644
--- a/src/java/org/apache/cassandra/io/util/FileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/FileDataInput.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.io.util;
 import java.io.Closeable;
 import java.io.IOException;
 
-public interface FileDataInput extends DataInputPlus, Closeable
+public interface FileDataInput extends RewindableDataInput, Closeable
 {
     String getPath();
 
@@ -30,11 +30,5 @@ public interface FileDataInput extends DataInputPlus, Closeable
 
     void seek(long pos) throws IOException;
 
-    FileMark mark();
-
-    void reset(FileMark mark) throws IOException;
-
-    long bytesPastMark(FileMark mark);
-
     long getFilePointer();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileMark.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileMark.java b/src/java/org/apache/cassandra/io/util/FileMark.java
deleted file mode 100644
index 781bc1e..0000000
--- a/src/java/org/apache/cassandra/io/util/FileMark.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.io.util;
-
-public interface FileMark {}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java b/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
index 425c7d6..a585215 100644
--- a/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
+++ b/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
@@ -74,17 +74,17 @@ public class FileSegmentInputStream extends DataInputBuffer implements FileDataI
         return false;
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         throw new UnsupportedOperationException();
     }
 
-    public void reset(FileMark mark)
+    public void reset(DataPosition mark)
     {
         throw new UnsupportedOperationException();
     }
 
-    public long bytesPastMark(FileMark mark)
+    public long bytesPastMark(DataPosition mark)
     {
         return 0;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index b495bf0..1943773 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -200,19 +200,19 @@ public class RandomAccessReader extends RebufferingInputStream implements FileDa
         return bytes;
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         markedPointer = current();
         return new BufferedRandomAccessFileMark(markedPointer);
     }
 
-    public void reset(FileMark mark)
+    public void reset(DataPosition mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
         seek(((BufferedRandomAccessFileMark) mark).pointer);
     }
 
-    public long bytesPastMark(FileMark mark)
+    public long bytesPastMark(DataPosition mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
         long bytes = current() - ((BufferedRandomAccessFileMark) mark).pointer;
@@ -262,7 +262,7 @@ public class RandomAccessReader extends RebufferingInputStream implements FileDa
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class BufferedRandomAccessFileMark implements FileMark
+    protected static class BufferedRandomAccessFileMark implements DataPosition
     {
         final long pointer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
new file mode 100644
index 0000000..c202f60
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.IOException;
+
+public interface RewindableDataInput extends DataInputPlus
+{
+    DataPosition mark();
+
+    void reset(DataPosition mark) throws IOException;
+
+    long bytesPastMark(DataPosition mark);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
new file mode 100644
index 0000000..3a680f4
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
@@ -0,0 +1,569 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * Adds mark/reset functionality to another input stream by caching read bytes to a memory buffer and
+ * spilling to disk if necessary.
+ *
+ * When the stream is marked via {@link this#mark()} or {@link this#mark(int)}, up to
+ * <code>maxMemBufferSize</code> will be cached in memory (heap). If more than
+ * <code>maxMemBufferSize</code> bytes are read while the stream is marked, the
+ * following bytes are cached on the <code>spillFile</code> for up to <code>maxDiskBufferSize</code>.
+ *
+ * Please note that successive calls to {@link this#mark()} and {@link this#reset()} will write
+ * sequentially to the same <code>spillFile</code> until <code>maxDiskBufferSize</code> is reached.
+ * At this point, if less than <code>maxDiskBufferSize</code> bytes are currently cached on the
+ * <code>spillFile</code>, the remaining bytes are written to the beginning of the file,
+ * treating the <code>spillFile</code> as a circular buffer.
+ *
+ * If more than <code>maxMemBufferSize + maxDiskBufferSize</code> are cached while the stream is marked,
+ * the following {@link this#reset()} invocation will throw a {@link IllegalStateException}.
+ *
+ */
+public class RewindableDataInputStreamPlus extends FilterInputStream implements RewindableDataInput, Closeable
+{
+    private boolean marked = false;
+    private boolean exhausted = false;
+    private AtomicBoolean closed = new AtomicBoolean(false);
+
+    protected int memAvailable = 0;
+    protected int diskTailAvailable = 0;
+    protected int diskHeadAvailable = 0;
+
+    private final File spillFile;
+    private final int initialMemBufferSize;
+    private final int maxMemBufferSize;
+    private final int maxDiskBufferSize;
+
+    private volatile byte memBuffer[];
+    private int memBufferSize;
+    private RandomAccessFile spillBuffer;
+
+    private final DataInputPlus dataReader;
+
+    public RewindableDataInputStreamPlus(InputStream in, int initialMemBufferSize, int maxMemBufferSize,
+                                         File spillFile, int maxDiskBufferSize)
+    {
+        super(in);
+        dataReader = new DataInputStreamPlus(this);
+        this.initialMemBufferSize = initialMemBufferSize;
+        this.maxMemBufferSize = maxMemBufferSize;
+        this.spillFile = spillFile;
+        this.maxDiskBufferSize = maxDiskBufferSize;
+    }
+
+    /* RewindableDataInput methods */
+
+    /**
+     * Marks the current position of a stream to return to this position later via the {@link this#reset(DataPosition)} method.
+     * @return An empty @link{DataPosition} object
+     */
+    public DataPosition mark()
+    {
+        mark(0);
+        return new RewindableDataInputPlusMark();
+    }
+
+    /**
+     * Rewinds to the previously marked position via the {@link this#mark()} method.
+     * @param mark it's not possible to return to a custom position, so this parameter is ignored.
+     * @throws IOException if an error ocurs while resetting
+     */
+    public void reset(DataPosition mark) throws IOException
+    {
+        reset();
+    }
+
+    public long bytesPastMark(DataPosition mark)
+    {
+        return maxMemBufferSize - memAvailable + (diskTailAvailable == -1? 0 : maxDiskBufferSize - diskHeadAvailable - diskTailAvailable);
+    }
+
+
+    protected static class RewindableDataInputPlusMark implements DataPosition
+    {
+    }
+
+    /* InputStream methods */
+
+    public boolean markSupported()
+    {
+        return true;
+    }
+
+    /**
+     * Marks the current position of a stream to return to this position
+     * later via the {@link this#reset()} method.
+     * @param readlimit the maximum amount of bytes to cache
+     */
+    public synchronized void mark(int readlimit)
+    {
+        if (marked)
+            throw new IllegalStateException("Cannot mark already marked stream.");
+
+        if (memAvailable > 0 || diskHeadAvailable > 0 || diskTailAvailable > 0)
+            throw new IllegalStateException("Can only mark stream after reading previously marked data.");
+
+        marked = true;
+        memAvailable = maxMemBufferSize;
+        diskHeadAvailable = -1;
+        diskTailAvailable = -1;
+    }
+
+    public synchronized void reset() throws IOException
+    {
+        if (!marked)
+            throw new IOException("Must call mark() before calling reset().");
+
+        if (exhausted)
+            throw new IOException(String.format("Read more than capacity: %d bytes.", maxMemBufferSize + maxDiskBufferSize));
+
+        memAvailable = maxMemBufferSize - memAvailable;
+        memBufferSize = memAvailable;
+
+        if (diskTailAvailable == -1)
+        {
+            diskHeadAvailable = 0;
+            diskTailAvailable = 0;
+        }
+        else
+        {
+            int initialPos = diskTailAvailable > 0 ? 0 : (int)getIfNotClosed(spillBuffer).getFilePointer();
+            int diskMarkpos = initialPos + diskHeadAvailable;
+            getIfNotClosed(spillBuffer).seek(diskMarkpos);
+
+            diskHeadAvailable = diskMarkpos - diskHeadAvailable;
+            diskTailAvailable = (maxDiskBufferSize - diskTailAvailable) - diskMarkpos;
+        }
+
+        marked = false;
+    }
+
+    public int available() throws IOException
+    {
+
+        return super.available() + (marked? 0 : memAvailable + diskHeadAvailable + diskTailAvailable);
+    }
+
+    public int read() throws IOException
+    {
+        int read = readOne();
+        if (read == -1)
+            return read;
+
+        if (marked)
+        {
+            //mark exhausted
+            if (isExhausted(1))
+            {
+                exhausted = true;
+                return read;
+            }
+
+            writeOne(read);
+        }
+
+        return read;
+    }
+
+    public int read(byte[] b, int off, int len) throws IOException
+    {
+        int readBytes = readMulti(b, off, len);
+        if (readBytes == -1)
+            return readBytes;
+
+        if (marked)
+        {
+            //check we have space on buffer
+            if (isExhausted(readBytes))
+            {
+                exhausted = true;
+                return readBytes;
+            }
+
+            writeMulti(b, off, readBytes);
+        }
+
+        return readBytes;
+    }
+
+    private void maybeCreateDiskBuffer() throws IOException
+    {
+        if (spillBuffer == null)
+        {
+            if (!spillFile.getParentFile().exists())
+                spillFile.getParentFile().mkdirs();
+            spillFile.createNewFile();
+
+            this.spillBuffer = new RandomAccessFile(spillFile, "rw");
+        }
+    }
+
+
+    private int readOne() throws IOException
+    {
+        if (!marked)
+        {
+            if (memAvailable > 0)
+            {
+                int pos = memBufferSize - memAvailable;
+                memAvailable--;
+                return getIfNotClosed(memBuffer)[pos] & 0xff;
+            }
+
+            if (diskTailAvailable > 0 || diskHeadAvailable > 0)
+            {
+                int read = getIfNotClosed(spillBuffer).read();
+                if (diskTailAvailable > 0)
+                    diskTailAvailable--;
+                else if (diskHeadAvailable > 0)
+                    diskHeadAvailable++;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+                return read;
+            }
+        }
+
+        return getIfNotClosed(in).read();
+    }
+
+    private boolean isExhausted(int readBytes)
+    {
+        return exhausted || readBytes > memAvailable + (long)(diskTailAvailable == -1? maxDiskBufferSize : diskTailAvailable + diskHeadAvailable);
+    }
+
+    private int readMulti(byte[] b, int off, int len) throws IOException
+    {
+        int readBytes = 0;
+        if (!marked)
+        {
+            if (memAvailable > 0)
+            {
+                readBytes += memAvailable < len ? memAvailable : len;
+                int pos = memBufferSize - memAvailable;
+                System.arraycopy(memBuffer, pos, b, off, readBytes);
+                memAvailable -= readBytes;
+                off += readBytes;
+                len -= readBytes;
+            }
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int readFromTail = diskTailAvailable < len? diskTailAvailable : len;
+                getIfNotClosed(spillBuffer).read(b, off, readFromTail);
+                readBytes += readFromTail;
+                diskTailAvailable -= readFromTail;
+                off += readFromTail;
+                len -= readFromTail;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+            }
+            if (len > 0 && diskHeadAvailable > 0)
+            {
+                int readFromHead = diskHeadAvailable < len? diskHeadAvailable : len;
+                getIfNotClosed(spillBuffer).read(b, off, readFromHead);
+                readBytes += readFromHead;
+                diskHeadAvailable -= readFromHead;
+                off += readFromHead;
+                len -= readFromHead;
+            }
+        }
+
+        if (len > 0)
+            readBytes += getIfNotClosed(in).read(b, off, len);
+
+        return readBytes;
+    }
+
+    private void writeMulti(byte[] b, int off, int len) throws IOException
+    {
+        if (memAvailable > 0)
+        {
+            if (memBuffer == null)
+                memBuffer = new byte[initialMemBufferSize];
+            int pos = maxMemBufferSize - memAvailable;
+            int memWritten = memAvailable < len? memAvailable : len;
+            if (pos + memWritten >= getIfNotClosed(memBuffer).length)
+                growMemBuffer(pos, memWritten);
+            System.arraycopy(b, off, memBuffer, pos, memWritten);
+            off += memWritten;
+            len -= memWritten;
+            memAvailable -= memWritten;
+        }
+
+        if (len > 0)
+        {
+            if (diskTailAvailable == -1)
+            {
+                maybeCreateDiskBuffer();
+                diskHeadAvailable = (int)spillBuffer.getFilePointer();
+                diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
+            }
+
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int diskTailWritten = diskTailAvailable < len? diskTailAvailable : len;
+                getIfNotClosed(spillBuffer).write(b, off, diskTailWritten);
+                off += diskTailWritten;
+                len -= diskTailWritten;
+                diskTailAvailable -= diskTailWritten;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+            }
+
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int diskHeadWritten = diskHeadAvailable < len? diskHeadAvailable : len;
+                getIfNotClosed(spillBuffer).write(b, off, diskHeadWritten);
+            }
+        }
+    }
+
+    private void writeOne(int value) throws IOException
+    {
+        if (memAvailable > 0)
+        {
+            if (memBuffer == null)
+                memBuffer = new byte[initialMemBufferSize];
+            int pos = maxMemBufferSize - memAvailable;
+            if (pos == getIfNotClosed(memBuffer).length)
+                growMemBuffer(pos, 1);
+            getIfNotClosed(memBuffer)[pos] = (byte)value;
+            memAvailable--;
+            return;
+        }
+
+        if (diskTailAvailable == -1)
+        {
+            maybeCreateDiskBuffer();
+            diskHeadAvailable = (int)spillBuffer.getFilePointer();
+            diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
+        }
+
+        if (diskTailAvailable > 0 || diskHeadAvailable > 0)
+        {
+            getIfNotClosed(spillBuffer).write(value);
+            if (diskTailAvailable > 0)
+                diskTailAvailable--;
+            else if (diskHeadAvailable > 0)
+                diskHeadAvailable--;
+            if (diskTailAvailable == 0)
+                spillBuffer.seek(0);
+            return;
+        }
+    }
+
+    public int read(byte[] b) throws IOException
+    {
+        return read(b, 0, b.length);
+    }
+
+    private void growMemBuffer(int pos, int writeSize)
+    {
+        int newSize = Math.min(2 * (pos + writeSize), maxMemBufferSize);
+        byte newBuffer[] = new byte[newSize];
+        System.arraycopy(memBuffer, 0, newBuffer, 0, (int)pos);
+        memBuffer = newBuffer;
+    }
+
+    public long skip(long n) throws IOException
+    {
+        long skipped = 0;
+
+        if (marked)
+        {
+            //if marked, we need to cache skipped bytes
+            while (n-- > 0 && read() != -1)
+            {
+                skipped++;
+            }
+            return skipped;
+        }
+
+        if (memAvailable > 0)
+        {
+            skipped += memAvailable < n ? memAvailable : n;
+            memAvailable -= skipped;
+            n -= skipped;
+        }
+        if (n > 0 && diskTailAvailable > 0)
+        {
+            int skipFromTail = diskTailAvailable < n? diskTailAvailable : (int)n;
+            getIfNotClosed(spillBuffer).skipBytes(skipFromTail);
+            diskTailAvailable -= skipFromTail;
+            skipped += skipFromTail;
+            n -= skipFromTail;
+            if (diskTailAvailable == 0)
+                spillBuffer.seek(0);
+        }
+        if (n > 0 && diskHeadAvailable > 0)
+        {
+            int skipFromHead = diskHeadAvailable < n? diskHeadAvailable : (int)n;
+            getIfNotClosed(spillBuffer).skipBytes(skipFromHead);
+            diskHeadAvailable -= skipFromHead;
+            skipped += skipFromHead;
+            n -= skipFromHead;
+        }
+
+        if (n > 0)
+            skipped += getIfNotClosed(in).skip(n);
+
+        return skipped;
+    }
+
+    private <T> T getIfNotClosed(T in) throws IOException {
+        if (closed.get())
+            throw new IOException("Stream closed");
+        return in;
+    }
+
+    public void close() throws IOException
+    {
+        close(true);
+    }
+
+    public void close(boolean closeUnderlying) throws IOException
+    {
+        if (closed.compareAndSet(false, true))
+        {
+            Throwable fail = null;
+            if (closeUnderlying)
+            {
+                try
+                {
+                    super.close();
+                }
+                catch (IOException e)
+                {
+                    fail = merge(fail, e);
+                }
+            }
+            try
+            {
+                if (spillBuffer != null)
+                {
+                    this.spillBuffer.close();
+                    this.spillBuffer = null;
+                }
+            } catch (IOException e)
+            {
+                fail = merge(fail, e);
+            }
+            try {
+                if (spillFile.exists())
+                {
+                    spillFile.delete();
+                }
+            }
+            catch (Throwable e)
+            {
+                fail = merge(fail, e);
+            }
+            maybeFail(fail, IOException.class);
+        }
+    }
+
+    /* DataInputPlus methods */
+
+    public void readFully(byte[] b) throws IOException
+    {
+        dataReader.readFully(b);
+    }
+
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        dataReader.readFully(b, off, len);
+    }
+
+    public int skipBytes(int n) throws IOException
+    {
+        return dataReader.skipBytes(n);
+    }
+
+    public boolean readBoolean() throws IOException
+    {
+        return dataReader.readBoolean();
+    }
+
+    public byte readByte() throws IOException
+    {
+        return dataReader.readByte();
+    }
+
+    public int readUnsignedByte() throws IOException
+    {
+        return dataReader.readUnsignedByte();
+    }
+
+    public short readShort() throws IOException
+    {
+        return dataReader.readShort();
+    }
+
+    public int readUnsignedShort() throws IOException
+    {
+        return dataReader.readUnsignedShort();
+    }
+
+    public char readChar() throws IOException
+    {
+        return dataReader.readChar();
+    }
+
+    public int readInt() throws IOException
+    {
+        return dataReader.readInt();
+    }
+
+    public long readLong() throws IOException
+    {
+        return dataReader.readLong();
+    }
+
+    public float readFloat() throws IOException
+    {
+        return dataReader.readFloat();
+    }
+
+    public double readDouble() throws IOException
+    {
+        return dataReader.readDouble();
+    }
+
+    public String readLine() throws IOException
+    {
+        return dataReader.readLine();
+    }
+
+    public String readUTF() throws IOException
+    {
+        return dataReader.readUTF();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index dd49868..26316a2 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -297,7 +297,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
         return bufferOffset + (buffer == null ? 0 : buffer.position());
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         return new BufferedFileWriterMark(current());
     }
@@ -306,7 +306,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
      * Drops all buffered data that's past the limits of our new file mark + buffer capacity, or syncs and truncates
      * the underlying file to the marked position
      */
-    public void resetAndTruncate(FileMark mark)
+    public void resetAndTruncate(DataPosition mark)
     {
         assert mark instanceof BufferedFileWriterMark;
 
@@ -404,7 +404,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class BufferedFileWriterMark implements FileMark
+    protected static class BufferedFileWriterMark implements DataPosition
     {
         final long pointer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java b/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
new file mode 100644
index 0000000..dc5bbb6
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * This class is to track bytes read from given DataInput
+ */
+public class TrackedDataInputPlus implements DataInputPlus, BytesReadTracker
+{
+    private long bytesRead;
+    final DataInput source;
+
+    public TrackedDataInputPlus(DataInput source)
+    {
+        this.source = source;
+    }
+
+    public long getBytesRead()
+    {
+        return bytesRead;
+    }
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count)
+    {
+        bytesRead = count;
+    }
+
+    public boolean readBoolean() throws IOException
+    {
+        boolean bool = source.readBoolean();
+        bytesRead += 1;
+        return bool;
+    }
+
+    public byte readByte() throws IOException
+    {
+        byte b = source.readByte();
+        bytesRead += 1;
+        return b;
+    }
+
+    public char readChar() throws IOException
+    {
+        char c = source.readChar();
+        bytesRead += 2;
+        return c;
+    }
+
+    public double readDouble() throws IOException
+    {
+        double d = source.readDouble();
+        bytesRead += 8;
+        return d;
+    }
+
+    public float readFloat() throws IOException
+    {
+        float f = source.readFloat();
+        bytesRead += 4;
+        return f;
+    }
+
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        source.readFully(b, off, len);
+        bytesRead += len;
+    }
+
+    public void readFully(byte[] b) throws IOException
+    {
+        source.readFully(b);
+        bytesRead += b.length;
+    }
+
+    public int readInt() throws IOException
+    {
+        int i = source.readInt();
+        bytesRead += 4;
+        return i;
+    }
+
+    public String readLine() throws IOException
+    {
+        // since this method is deprecated and cannot track bytes read
+        // just throw exception
+        throw new UnsupportedOperationException();
+    }
+
+    public long readLong() throws IOException
+    {
+        long l = source.readLong();
+        bytesRead += 8;
+        return l;
+    }
+
+    public short readShort() throws IOException
+    {
+        short s = source.readShort();
+        bytesRead += 2;
+        return s;
+    }
+
+    public String readUTF() throws IOException
+    {
+        return DataInputStream.readUTF(this);
+    }
+
+    public int readUnsignedByte() throws IOException
+    {
+        int i = source.readUnsignedByte();
+        bytesRead += 1;
+        return i;
+    }
+
+    public int readUnsignedShort() throws IOException
+    {
+        int i = source.readUnsignedShort();
+        bytesRead += 2;
+        return i;
+    }
+
+    public int skipBytes(int n) throws IOException
+    {
+        int skipped = source.skipBytes(n);
+        bytesRead += skipped;
+        return skipped;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/TrackedInputStream.java b/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
new file mode 100644
index 0000000..f398d30
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * This class is to track bytes read from given DataInputStream
+ */
+public class TrackedInputStream extends FilterInputStream implements BytesReadTracker
+{
+    private long bytesRead;
+
+    public TrackedInputStream(InputStream source)
+    {
+        super(source);
+    }
+
+    public long getBytesRead()
+    {
+        return bytesRead;
+    }
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count)
+    {
+        bytesRead = count;
+    }
+
+    public int read() throws IOException
+    {
+        int read = super.read();
+        bytesRead += 1;
+        return read;
+    }
+
+    public int read(byte[] b, int off, int len) throws IOException
+    {
+        int read = super.read(b, off, len);
+        bytesRead += read;
+        return read;
+    }
+
+    public int read(byte[] b) throws IOException
+    {
+        int read = super.read(b);
+        bytesRead += read;
+        return read;
+    }
+
+    public long skip(long n) throws IOException
+    {
+        long skip = super.skip(n);
+        bytesRead += skip;
+        return skip;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f445e25..fd7f4b6 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4221,6 +4221,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return Collections.unmodifiableList(keyspaceNamesList);
     }
 
+
     public List<String> getNonSystemKeyspaces()
     {
         List<String> keyspaceNamesList = new ArrayList<>(Schema.instance.getNonSystemKeyspaces());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index 268f974..f8db26b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -40,10 +40,13 @@ import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.util.RewindableDataInputStreamPlus;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedInputStream;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 
@@ -105,9 +108,9 @@ public class StreamReader
                      session.planId(), fileSeqNum, session.peer, repairedAt, totalSize, cfs.keyspace.getName(),
                      cfs.getColumnFamilyName());
 
-        DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
-        BytesReadTracker in = new BytesReadTracker(dis);
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, header.toHeader(cfs.metadata));
+        TrackedInputStream in = new TrackedInputStream(new LZFInputStream(Channels.newInputStream(channel)));
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+                                                                 totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
         {
@@ -131,12 +134,22 @@ public class StreamReader
             {
                 writer.abort(e);
             }
-            drain(dis, in.getBytesRead());
+            drain(in, in.getBytesRead());
             if (e instanceof IOException)
                 throw (IOException) e;
             else
                 throw Throwables.propagate(e);
         }
+        finally
+        {
+            if (deserializer != null)
+                deserializer.cleanup();
+        }
+    }
+
+    protected SerializationHeader getHeader(CFMetaData metadata)
+    {
+        return header != null? header.toHeader(metadata) : null; //pre-3.0 sstable have no SerializationHeader
     }
 
     protected SSTableMultiWriter createWriter(ColumnFamilyStore cfs, long totalSize, long repairedAt, SSTableFormat.Type format) throws IOException
@@ -146,8 +159,7 @@ public class StreamReader
             throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
         desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
 
-
-        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), session.getTransaction(cfId));
+        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, getHeader(cfs.metadata), session.getTransaction(cfId));
     }
 
     protected void drain(InputStream dis, long bytesRead) throws IOException
@@ -185,6 +197,13 @@ public class StreamReader
 
     public static class StreamDeserializer extends UnmodifiableIterator<Unfiltered> implements UnfilteredRowIterator
     {
+        public static final int INITIAL_MEM_BUFFER_SIZE = Integer.getInteger("cassandra.streamdes.initial_mem_buffer_size", 32768);
+        public static final int MAX_MEM_BUFFER_SIZE = Integer.getInteger("cassandra.streamdes.max_mem_buffer_size", 1048576);
+        public static final int MAX_SPILL_FILE_SIZE = Integer.getInteger("cassandra.streamdes.max_spill_file_size", Integer.MAX_VALUE);
+
+        public static final String BUFFER_FILE_PREFIX = "buf";
+        public static final String BUFFER_FILE_SUFFIX = "dat";
+
         private final CFMetaData metadata;
         private final DataInputPlus in;
         private final SerializationHeader header;
@@ -196,11 +215,20 @@ public class StreamReader
         private Row staticRow;
         private IOException exception;
 
-        public StreamDeserializer(CFMetaData metadata, DataInputPlus in, Version version, SerializationHeader header)
+        public StreamDeserializer(CFMetaData metadata, InputStream in, Version version, SerializationHeader header,
+                                  long totalSize, UUID sessionId) throws IOException
         {
-            assert version.storeRows() : "We don't allow streaming from pre-3.0 nodes";
             this.metadata = metadata;
-            this.in = in;
+            // streaming pre-3.0 sstables require mark/reset support from source stream
+            if (version.correspondingMessagingVersion() < MessagingService.VERSION_30)
+            {
+                logger.trace("Initializing rewindable input stream for reading legacy sstable with {} bytes with following " +
+                             "parameters: initial_mem_buffer_size={}, max_mem_buffer_size={}, max_spill_file_size={}.",
+                             totalSize, INITIAL_MEM_BUFFER_SIZE, MAX_MEM_BUFFER_SIZE, MAX_SPILL_FILE_SIZE);
+                File bufferFile = getTempBufferFile(metadata, totalSize, sessionId);
+                this.in = new RewindableDataInputStreamPlus(in, INITIAL_MEM_BUFFER_SIZE, MAX_MEM_BUFFER_SIZE, bufferFile, MAX_SPILL_FILE_SIZE);
+            } else
+                this.in = new DataInputPlus.DataInputStreamPlus(in);
             this.helper = new SerializationHelper(metadata, version.correspondingMessagingVersion(), SerializationHelper.Flag.PRESERVE_SIZE);
             this.header = header;
         }
@@ -292,5 +320,41 @@ public class StreamReader
         public void close()
         {
         }
+
+        /* We have a separate cleanup method because sometimes close is called before exhausting the
+           StreamDeserializer (for instance, when enclosed in an try-with-resources wrapper, such as in
+           BigTableWriter.append()).
+         */
+        public void cleanup()
+        {
+            if (in instanceof RewindableDataInputStreamPlus)
+            {
+                try
+                {
+                    ((RewindableDataInputStreamPlus) in).close(false);
+                }
+                catch (IOException e)
+                {
+                    logger.warn("Error while closing RewindableDataInputStreamPlus.", e);
+                }
+            }
+        }
+
+        private static File getTempBufferFile(CFMetaData metadata, long totalSize, UUID sessionId) throws IOException
+        {
+            ColumnFamilyStore cfs = Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName);
+            if (cfs == null)
+            {
+                // schema was dropped during streaming
+                throw new RuntimeException(String.format("CF %s.%s was dropped during streaming", metadata.ksName, metadata.cfName));
+            }
+
+            long maxSize = Math.min(MAX_SPILL_FILE_SIZE, totalSize);
+            File tmpDir = cfs.getDirectories().getTemporaryWriteableDirectoryAsFile(maxSize);
+            if (tmpDir == null)
+                throw new IOException(String.format("No sufficient disk space to stream legacy sstable from {}.{}. " +
+                                                         "Required disk space: %s.", FBUtilities.prettyPrintMemory(maxSize)));
+            return new File(tmpDir, String.format("%s-%s.%s", BUFFER_FILE_PREFIX, sessionId, BUFFER_FILE_SUFFIX));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index 5210d5b..9719587 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -24,7 +24,6 @@ import java.nio.channels.ReadableByteChannel;
 
 import com.google.common.base.Throwables;
 
-import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 
 import org.slf4j.Logger;
@@ -38,7 +37,7 @@ import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamReader;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedInputStream;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -83,8 +82,10 @@ public class CompressedStreamReader extends StreamReader
 
         CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo,
                                                               inputVersion.compressedChecksumType(), cfs::getCrcCheckChance);
-        BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, header.toHeader(cfs.metadata));
+        TrackedInputStream in = new TrackedInputStream(cis);
+
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+                                                                 totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
         {
@@ -115,17 +116,22 @@ public class CompressedStreamReader extends StreamReader
         {
             if (deserializer != null)
                 logger.warn("[Stream {}] Error while reading partition {} from stream on ks='{}' and table='{}'.",
-                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getColumnFamilyName());
+                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getTableName());
             if (writer != null)
             {
                 writer.abort(e);
             }
-            drain(cis, in.getBytesRead());
+            drain(in, in.getBytesRead());
             if (e instanceof IOException)
                 throw (IOException) e;
             else
                 throw Throwables.propagate(e);
         }
+        finally
+        {
+            if (deserializer != null)
+                deserializer.cleanup();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
index e1e13b7..2b5047d 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -204,7 +204,7 @@ public class FileMessageHeader
             out.writeLong(header.repairedAt);
             out.writeInt(header.sstableLevel);
 
-            if (version >= StreamMessage.VERSION_30)
+            if (version >= StreamMessage.VERSION_30 && header.version.storeRows())
                 SerializationHeader.serializer.serialize(header.version, header.header, out);
             return compressionInfo;
         }
@@ -227,7 +227,7 @@ public class FileMessageHeader
             CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, MessagingService.current_version);
             long repairedAt = in.readLong();
             int sstableLevel = in.readInt();
-            SerializationHeader.Component header = version >= StreamMessage.VERSION_30
+            SerializationHeader.Component header = version >= StreamMessage.VERSION_30 && sstableVersion.storeRows()
                                                  ? SerializationHeader.serializer.deserialize(sstableVersion, in)
                                                  : null;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/tools/nodetool/Repair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
index 84b463c..bd1a916 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Repair.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
@@ -56,7 +56,7 @@ public class Repair extends NodeToolCmd
     private boolean localDC = false;
 
     @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
-    private List<String> specificDataCenters = new ArrayList<>();
+    private List<String> specificDataCenters = new ArrayList<>();;
 
     @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
     private List<String> specificHosts = new ArrayList<>();


[08/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/utils/BytesReadTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BytesReadTracker.java b/src/java/org/apache/cassandra/utils/BytesReadTracker.java
deleted file mode 100644
index 5e98b25..0000000
--- a/src/java/org/apache/cassandra/utils/BytesReadTracker.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.utils;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-
-/**
- * This class is to track bytes read from given DataInput
- */
-public class BytesReadTracker implements DataInputPlus
-{
-
-    private long bytesRead;
-    final DataInput source;
-
-    public BytesReadTracker(DataInput source)
-    {
-        this.source = source;
-    }
-
-    public long getBytesRead()
-    {
-        return bytesRead;
-    }
-
-    /**
-     * reset counter to @param count
-     */
-    public void reset(long count)
-    {
-        bytesRead = count;
-    }
-
-    public boolean readBoolean() throws IOException
-    {
-        boolean bool = source.readBoolean();
-        bytesRead += 1;
-        return bool;
-    }
-
-    public byte readByte() throws IOException
-    {
-        byte b = source.readByte();
-        bytesRead += 1;
-        return b;
-    }
-
-    public char readChar() throws IOException
-    {
-        char c = source.readChar();
-        bytesRead += 2;
-        return c;
-    }
-
-    public double readDouble() throws IOException
-    {
-        double d = source.readDouble();
-        bytesRead += 8;
-        return d;
-    }
-
-    public float readFloat() throws IOException
-    {
-        float f = source.readFloat();
-        bytesRead += 4;
-        return f;
-    }
-
-    public void readFully(byte[] b, int off, int len) throws IOException
-    {
-        source.readFully(b, off, len);
-        bytesRead += len;
-    }
-
-    public void readFully(byte[] b) throws IOException
-    {
-        source.readFully(b);
-        bytesRead += b.length;
-    }
-
-    public int readInt() throws IOException
-    {
-        int i = source.readInt();
-        bytesRead += 4;
-        return i;
-    }
-
-    public String readLine() throws IOException
-    {
-        // since this method is deprecated and cannot track bytes read
-        // just throw exception
-        throw new UnsupportedOperationException();
-    }
-
-    public long readLong() throws IOException
-    {
-        long l = source.readLong();
-        bytesRead += 8;
-        return l;
-    }
-
-    public short readShort() throws IOException
-    {
-        short s = source.readShort();
-        bytesRead += 2;
-        return s;
-    }
-
-    public String readUTF() throws IOException
-    {
-        return DataInputStream.readUTF(this);
-    }
-
-    public int readUnsignedByte() throws IOException
-    {
-        int i = source.readUnsignedByte();
-        bytesRead += 1;
-        return i;
-    }
-
-    public int readUnsignedShort() throws IOException
-    {
-        int i = source.readUnsignedShort();
-        bytesRead += 2;
-        return i;
-    }
-
-    public int skipBytes(int n) throws IOException
-    {
-        int skipped = source.skipBytes(n);
-        bytesRead += skipped;
-        return skipped;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/utils/CloseableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CloseableIterator.java b/src/java/org/apache/cassandra/utils/CloseableIterator.java
index a7c4300..57034ae 100644
--- a/src/java/org/apache/cassandra/utils/CloseableIterator.java
+++ b/src/java/org/apache/cassandra/utils/CloseableIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.Closeable;
 import java.util.Iterator;
 
 // so we can instantiate anonymous classes implementing both interfaces

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..5eddda7
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db
new file mode 100644
index 0000000..61ef270
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..ab83acc
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db
new file mode 100644
index 0000000..896a529
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..34d459d
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db
new file mode 100644
index 0000000..b511d30
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..aa3c757
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db
new file mode 100644
index 0000000..896a529
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db
new file mode 100644
index 0000000..c7e8586
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..792e733
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db
new file mode 100644
index 0000000..af5e781
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db
new file mode 100644
index 0000000..f545b04
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..8657050
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db
new file mode 100644
index 0000000..af5e781
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..654094e
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db
new file mode 100644
index 0000000..4c87e07
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..4690757
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1331331706
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..ab55258
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db
new file mode 100644
index 0000000..774cbd1
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..e3b71a4
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db
new file mode 100644
index 0000000..90d42a5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..52e6552
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2793875907
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..8360ed5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db
new file mode 100644
index 0000000..774cbd1
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db
new file mode 100644
index 0000000..6a38c52
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..be8e5fb
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+606280675
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..e3fd855
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db
new file mode 100644
index 0000000..af8ad8b
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db
new file mode 100644
index 0000000..5f4a7db
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..a71f766
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+616768162
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..4a6e940
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db
new file mode 100644
index 0000000..af8ad8b
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2a72f70
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db
new file mode 100644
index 0000000..6bc08d2
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..943dd1e
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+1372047449
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..b2fd408
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..d4dec70
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db
new file mode 100644
index 0000000..63ee721
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..577407e
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2583914481
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..2bfc59d
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db
new file mode 100644
index 0000000..2e912a1
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..c07a57f
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+278403976
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..a81e03e
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db
new file mode 100644
index 0000000..323ff37
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..92237e7
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2048991053
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..37324a7
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..11e69f9
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..9af7ba2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..46e277c
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+231958969
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..10fce6d
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..70a08ed
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2c92e35
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..f8dadef
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..b7dbadc
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3570926375
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..5697f6f
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..45dbcfe
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..adb7fc4
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..69aec90
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..da73bd8
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3668325305
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..56f29df
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..d4b841a
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..56c95a8
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..b30b1c4
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..18ba1ff
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3769133549
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..d094f73
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..b81ea33
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db differ


[02/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/utils/BytesReadTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/BytesReadTracker.java b/src/java/org/apache/cassandra/utils/BytesReadTracker.java
deleted file mode 100644
index 5e98b25..0000000
--- a/src/java/org/apache/cassandra/utils/BytesReadTracker.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.utils;
-
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.IOException;
-
-import org.apache.cassandra.io.util.DataInputPlus;
-
-/**
- * This class is to track bytes read from given DataInput
- */
-public class BytesReadTracker implements DataInputPlus
-{
-
-    private long bytesRead;
-    final DataInput source;
-
-    public BytesReadTracker(DataInput source)
-    {
-        this.source = source;
-    }
-
-    public long getBytesRead()
-    {
-        return bytesRead;
-    }
-
-    /**
-     * reset counter to @param count
-     */
-    public void reset(long count)
-    {
-        bytesRead = count;
-    }
-
-    public boolean readBoolean() throws IOException
-    {
-        boolean bool = source.readBoolean();
-        bytesRead += 1;
-        return bool;
-    }
-
-    public byte readByte() throws IOException
-    {
-        byte b = source.readByte();
-        bytesRead += 1;
-        return b;
-    }
-
-    public char readChar() throws IOException
-    {
-        char c = source.readChar();
-        bytesRead += 2;
-        return c;
-    }
-
-    public double readDouble() throws IOException
-    {
-        double d = source.readDouble();
-        bytesRead += 8;
-        return d;
-    }
-
-    public float readFloat() throws IOException
-    {
-        float f = source.readFloat();
-        bytesRead += 4;
-        return f;
-    }
-
-    public void readFully(byte[] b, int off, int len) throws IOException
-    {
-        source.readFully(b, off, len);
-        bytesRead += len;
-    }
-
-    public void readFully(byte[] b) throws IOException
-    {
-        source.readFully(b);
-        bytesRead += b.length;
-    }
-
-    public int readInt() throws IOException
-    {
-        int i = source.readInt();
-        bytesRead += 4;
-        return i;
-    }
-
-    public String readLine() throws IOException
-    {
-        // since this method is deprecated and cannot track bytes read
-        // just throw exception
-        throw new UnsupportedOperationException();
-    }
-
-    public long readLong() throws IOException
-    {
-        long l = source.readLong();
-        bytesRead += 8;
-        return l;
-    }
-
-    public short readShort() throws IOException
-    {
-        short s = source.readShort();
-        bytesRead += 2;
-        return s;
-    }
-
-    public String readUTF() throws IOException
-    {
-        return DataInputStream.readUTF(this);
-    }
-
-    public int readUnsignedByte() throws IOException
-    {
-        int i = source.readUnsignedByte();
-        bytesRead += 1;
-        return i;
-    }
-
-    public int readUnsignedShort() throws IOException
-    {
-        int i = source.readUnsignedShort();
-        bytesRead += 2;
-        return i;
-    }
-
-    public int skipBytes(int n) throws IOException
-    {
-        int skipped = source.skipBytes(n);
-        bytesRead += skipped;
-        return skipped;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/utils/CloseableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/CloseableIterator.java b/src/java/org/apache/cassandra/utils/CloseableIterator.java
index a7c4300..57034ae 100644
--- a/src/java/org/apache/cassandra/utils/CloseableIterator.java
+++ b/src/java/org/apache/cassandra/utils/CloseableIterator.java
@@ -17,7 +17,6 @@
  */
 package org.apache.cassandra.utils;
 
-import java.io.Closeable;
 import java.util.Iterator;
 
 // so we can instantiate anonymous classes implementing both interfaces

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..5eddda7
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db
new file mode 100644
index 0000000..61ef270
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..ab83acc
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db
new file mode 100644
index 0000000..896a529
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_compact/legacy_tables-legacy_jb_clust_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..34d459d
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db
new file mode 100644
index 0000000..b511d30
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..aa3c757
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db
new file mode 100644
index 0000000..896a529
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_clust_counter_compact/legacy_tables-legacy_jb_clust_counter_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db
new file mode 100644
index 0000000..c7e8586
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..792e733
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db
new file mode 100644
index 0000000..af5e781
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_compact/legacy_tables-legacy_jb_simple_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db
new file mode 100644
index 0000000..f545b04
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db
new file mode 100644
index 0000000..8657050
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db
new file mode 100644
index 0000000..af5e781
Binary files /dev/null and b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
new file mode 100644
index 0000000..b67360a
--- /dev/null
+++ b/test/data/legacy-sstables/jb/legacy_tables/legacy_jb_simple_counter_compact/legacy_tables-legacy_jb_simple_counter_compact-jb-1-TOC.txt
@@ -0,0 +1,7 @@
+Data.db
+CompressionInfo.db
+Index.db
+Summary.db
+TOC.txt
+Statistics.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..654094e
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db
new file mode 100644
index 0000000..4c87e07
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..4690757
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1331331706
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..ab55258
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db
new file mode 100644
index 0000000..774cbd1
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_compact/legacy_tables-legacy_ka_clust_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..e3b71a4
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db
new file mode 100644
index 0000000..90d42a5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..52e6552
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2793875907
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..8360ed5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db
new file mode 100644
index 0000000..774cbd1
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_clust_counter_compact/legacy_tables-legacy_ka_clust_counter_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db
new file mode 100644
index 0000000..6a38c52
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..be8e5fb
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+606280675
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..e3fd855
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db
new file mode 100644
index 0000000..af8ad8b
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_compact/legacy_tables-legacy_ka_simple_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db
new file mode 100644
index 0000000..5f4a7db
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
new file mode 100644
index 0000000..a71f766
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Digest.sha1
@@ -0,0 +1 @@
+616768162
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db
new file mode 100644
index 0000000..4a6e940
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db
new file mode 100644
index 0000000..af8ad8b
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
new file mode 100644
index 0000000..7f7fe79
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_simple_counter_compact/legacy_tables-legacy_ka_simple_counter_compact-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Filter.db
+TOC.txt
+Statistics.db
+Summary.db
+Index.db
+Data.db
+Digest.sha1
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2a72f70
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db
new file mode 100644
index 0000000..6bc08d2
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..943dd1e
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+1372047449
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db
new file mode 100644
index 0000000..9e18f8e
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..b2fd408
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..d4dec70
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db
new file mode 100644
index 0000000..63ee721
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..577407e
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2583914481
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db
new file mode 100644
index 0000000..10df1e8
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..2bfc59d
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_clust_counter_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..d530b73
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db
new file mode 100644
index 0000000..2e912a1
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..c07a57f
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+278403976
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db
new file mode 100644
index 0000000..d2ec218
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..a81e03e
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db
new file mode 100644
index 0000000..01c5478
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db
new file mode 100644
index 0000000..323ff37
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32 b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
new file mode 100644
index 0000000..92237e7
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Digest.adler32
@@ -0,0 +1 @@
+2048991053
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db
new file mode 100644
index 0000000..c3cb27c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db
new file mode 100644
index 0000000..48c153c
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db
new file mode 100644
index 0000000..37324a7
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db
new file mode 100644
index 0000000..6cd998f
Binary files /dev/null and b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
new file mode 100644
index 0000000..0aef810
--- /dev/null
+++ b/test/data/legacy-sstables/la/legacy_tables/legacy_la_simple_counter_compact/la-1-big-TOC.txt
@@ -0,0 +1,8 @@
+TOC.txt
+Statistics.db
+Digest.adler32
+CompressionInfo.db
+Summary.db
+Data.db
+Filter.db
+Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..11e69f9
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..9af7ba2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..46e277c
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+231958969
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..10fce6d
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..70a08ed
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..2c92e35
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..f8dadef
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..b7dbadc
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3570926375
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..5697f6f
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..45dbcfe
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_clust_counter_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..adb7fc4
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..69aec90
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..da73bd8
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3668325305
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..56f29df
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..d4b841a
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db
new file mode 100644
index 0000000..56c95a8
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db
new file mode 100644
index 0000000..b30b1c4
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32 b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
new file mode 100644
index 0000000..18ba1ff
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Digest.crc32
@@ -0,0 +1 @@
+3769133549
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db
new file mode 100644
index 0000000..2e1d5d2
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db
new file mode 100644
index 0000000..d094f73
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db
new file mode 100644
index 0000000..b81ea33
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db
new file mode 100644
index 0000000..9b24e04
Binary files /dev/null and b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-Summary.db differ


[09/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
Support streaming of older version sstables in 3.0

patch by Paulo Motta; reviewed by Yuki Morishita for CASSANDRA-10990


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/e8651b66
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/e8651b66
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/e8651b66

Branch: refs/heads/trunk
Commit: e8651b6625c7f6260852f2a9c45fb189c63ab528
Parents: 7f1339c
Author: Paulo Motta <pa...@gmail.com>
Authored: Fri Feb 5 12:38:39 2016 -0300
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 17 10:04:44 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   2 +
 .../org/apache/cassandra/db/Directories.java    |  30 +
 .../cassandra/db/SerializationHeader.java       |   5 +
 .../org/apache/cassandra/db/Serializers.java    | 114 ++--
 .../columniterator/AbstractSSTableIterator.java |   4 +-
 .../cassandra/hints/ChecksummedDataInput.java   |   8 +-
 .../org/apache/cassandra/hints/HintMessage.java |   4 +-
 .../io/compress/CompressedSequentialWriter.java |   8 +-
 .../io/sstable/SSTableSimpleIterator.java       |  11 +-
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../io/sstable/format/SSTableWriter.java        |   2 +-
 .../io/sstable/format/big/BigTableWriter.java   |   4 +-
 .../cassandra/io/util/BytesReadTracker.java     |  30 +
 .../apache/cassandra/io/util/DataPosition.java  |  21 +
 .../apache/cassandra/io/util/FileDataInput.java |   8 +-
 .../org/apache/cassandra/io/util/FileMark.java  |  20 -
 .../io/util/FileSegmentInputStream.java         |   6 +-
 .../cassandra/io/util/RandomAccessReader.java   |   8 +-
 .../cassandra/io/util/RewindableDataInput.java  |  30 +
 .../io/util/RewindableDataInputStreamPlus.java  | 569 +++++++++++++++++++
 .../cassandra/io/util/SequentialWriter.java     |   6 +-
 .../cassandra/io/util/TrackedDataInputPlus.java | 150 +++++
 .../cassandra/io/util/TrackedInputStream.java   |  76 +++
 .../cassandra/service/StorageService.java       |   1 +
 .../cassandra/streaming/StreamReader.java       |  84 ++-
 .../compress/CompressedStreamReader.java        |  18 +-
 .../streaming/messages/FileMessageHeader.java   |   4 +-
 .../apache/cassandra/tools/nodetool/Repair.java |   2 +-
 .../cassandra/utils/BytesReadTracker.java       | 153 -----
 .../cassandra/utils/CloseableIterator.java      |   1 -
 ...acy_jb_clust_compact-jb-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-Data.db | Bin 0 -> 5270 bytes
 ...ables-legacy_jb_clust_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_jb_clust_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_jb_clust_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...bles-legacy_jb_clust_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-TOC.txt |   7 +
 ...lust_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_jb_clust_counter_compact-jb-1-Data.db | Bin 0 -> 4228 bytes
 ...gacy_jb_clust_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_jb_clust_counter_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ..._jb_clust_counter_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...acy_jb_clust_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...legacy_jb_clust_counter_compact-jb-1-TOC.txt |   7 +
 ...cy_jb_simple_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_jb_simple_compact-jb-1-Data.db | Bin 0 -> 108 bytes
 ...bles-legacy_jb_simple_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_jb_simple_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_jb_simple_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...les-legacy_jb_simple_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...tables-legacy_jb_simple_compact-jb-1-TOC.txt |   7 +
 ...mple_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_jb_simple_counter_compact-jb-1-Data.db | Bin 0 -> 118 bytes
 ...acy_jb_simple_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_jb_simple_counter_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...jb_simple_counter_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...cy_jb_simple_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...egacy_jb_simple_counter_compact-jb-1-TOC.txt |   7 +
 ...acy_ka_clust_compact-ka-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-Data.db | Bin 0 -> 5277 bytes
 ...les-legacy_ka_clust_compact-ka-1-Digest.sha1 |   1 +
 ...ables-legacy_ka_clust_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_ka_clust_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_ka_clust_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...bles-legacy_ka_clust_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-TOC.txt |   8 +
 ...lust_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_ka_clust_counter_compact-ka-1-Data.db | Bin 0 -> 4527 bytes
 ...cy_ka_clust_counter_compact-ka-1-Digest.sha1 |   1 +
 ...gacy_ka_clust_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_ka_clust_counter_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ..._ka_clust_counter_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...acy_ka_clust_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...legacy_ka_clust_counter_compact-ka-1-TOC.txt |   8 +
 ...cy_ka_simple_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_ka_simple_compact-ka-1-Data.db | Bin 0 -> 105 bytes
 ...es-legacy_ka_simple_compact-ka-1-Digest.sha1 |   1 +
 ...bles-legacy_ka_simple_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_ka_simple_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_ka_simple_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...les-legacy_ka_simple_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...tables-legacy_ka_simple_compact-ka-1-TOC.txt |   8 +
 ...mple_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_ka_simple_counter_compact-ka-1-Data.db | Bin 0 -> 124 bytes
 ...y_ka_simple_counter_compact-ka-1-Digest.sha1 |   1 +
 ...acy_ka_simple_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_ka_simple_counter_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...ka_simple_counter_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...cy_ka_simple_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...egacy_ka_simple_counter_compact-ka-1-TOC.txt |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_la_clust_compact/la-1-big-Data.db    | Bin 0 -> 5286 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_clust_compact/la-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_la_clust_compact/la-1-big-Index.db   | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../legacy_la_clust_compact/la-1-big-Summary.db | Bin 0 -> 75 bytes
 .../legacy_la_clust_compact/la-1-big-TOC.txt    |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 4527 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_la_simple_compact/la-1-big-Data.db   | Bin 0 -> 106 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_simple_compact/la-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_la_simple_compact/la-1-big-Index.db  | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../legacy_la_simple_compact/la-1-big-TOC.txt   |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 124 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_ma_clust_compact/ma-1-big-Data.db    | Bin 0 -> 5393 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_clust_compact/ma-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_ma_clust_compact/ma-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7046 bytes
 .../legacy_ma_clust_compact/ma-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_ma_clust_compact/ma-1-big-TOC.txt    |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 4606 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7055 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_ma_simple_compact/ma-1-big-Data.db   | Bin 0 -> 91 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_simple_compact/ma-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_ma_simple_compact/ma-1-big-Index.db  | Bin 0 -> 26 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4640 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_ma_simple_compact/ma-1-big-TOC.txt   |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4649 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../cassandra/AbstractSerializationsTester.java |   1 -
 .../apache/cassandra/db/DirectoriesTest.java    |  98 ++--
 .../cassandra/gms/SerializationsTest.java       |   1 -
 .../CompressedRandomAccessReaderTest.java       |   6 +-
 .../CompressedSequentialWriterTest.java         |   4 +-
 .../cassandra/io/sstable/LegacySSTableTest.java | 368 ++++++------
 .../io/util/BufferedRandomAccessFileTest.java   |   4 +-
 .../io/util/RandomAccessReaderTest.java         |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java | 539 ++++++++++++++++++
 .../cassandra/utils/BytesReadTrackerTest.java   | 104 +++-
 165 files changed, 2101 insertions(+), 544 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index defc25a..51cfc16 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.5
+ * Support streaming pre-3.0 sstables (CASSANDRA-10990)
  * Add backpressure to compressed commit log (CASSANDRA-10971)
  * SSTableExport supports secondary index tables (CASSANDRA-11330)
  * Fix sstabledump to include missing info in debug output (CASSANDRA-11321)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c564d8d..f28df1c 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -571,6 +571,8 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
          // clear ephemeral snapshots that were not properly cleared last session (CASSANDRA-7357)
         clearEphemeralSnapshots(directories);
 
+        directories.removeTemporaryDirectories();
+
         logger.trace("Removing temporary or obsoleted files from unfinished operations for table {}", metadata.cfName);
         LifecycleTransaction.removeUnfinishedLeftovers(metadata);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Directories.java b/src/java/org/apache/cassandra/db/Directories.java
index 8744d43..83321ac 100644
--- a/src/java/org/apache/cassandra/db/Directories.java
+++ b/src/java/org/apache/cassandra/db/Directories.java
@@ -94,9 +94,11 @@ public class Directories
 
     public static final String BACKUPS_SUBDIR = "backups";
     public static final String SNAPSHOT_SUBDIR = "snapshots";
+    public static final String TMP_SUBDIR = "tmp";
     public static final String SECONDARY_INDEX_NAME_SEPARATOR = ".";
 
     public static final DataDirectory[] dataDirectories;
+
     static
     {
         String[] locations = DatabaseDescriptor.getAllDataFileLocations();
@@ -322,6 +324,34 @@ public class Directories
     }
 
     /**
+     * Returns a temporary subdirectory on non-blacklisted data directory
+     * that _currently_ has {@code writeSize} bytes as usable space.
+     * This method does not create the temporary directory.
+     *
+     * @throws IOError if all directories are blacklisted.
+     */
+    public File getTemporaryWriteableDirectoryAsFile(long writeSize)
+    {
+        File location = getLocationForDisk(getWriteableLocation(writeSize));
+        if (location == null)
+            return null;
+        return new File(location, TMP_SUBDIR);
+    }
+
+    public void removeTemporaryDirectories()
+    {
+        for (File dataDir : dataPaths)
+        {
+            File tmpDir = new File(dataDir, TMP_SUBDIR);
+            if (tmpDir.exists())
+            {
+                logger.debug("Removing temporary directory {}", tmpDir);
+                FileUtils.deleteRecursive(tmpDir);
+            }
+        }
+    }
+
+    /**
      * Returns a non-blacklisted data directory that _currently_ has {@code writeSize} bytes as usable space.
      *
      * @throws IOError if all directories are blacklisted.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/SerializationHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index 6e03756..0fd1281 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -70,6 +70,11 @@ public class SerializationHeader
         this.typeMap = typeMap;
     }
 
+    public static SerializationHeader makeWithoutStats(CFMetaData metadata)
+    {
+        return new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
+    }
+
     public static SerializationHeader forKeyCache(CFMetaData metadata)
     {
         // We don't save type information in the key cache (we could change

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/Serializers.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Serializers.java b/src/java/org/apache/cassandra/db/Serializers.java
index 9b29d89..348fda3 100644
--- a/src/java/org/apache/cassandra/db/Serializers.java
+++ b/src/java/org/apache/cassandra/db/Serializers.java
@@ -29,8 +29,6 @@ import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
-import static org.apache.cassandra.io.sstable.IndexHelper.IndexInfo;
-
 /**
  * Holds references on serializers that depend on the table definition.
  */
@@ -48,62 +46,77 @@ public class Serializers
     // unecessary (since IndexInfo.Serializer won't depend on the metadata either).
     public ISerializer<ClusteringPrefix> indexEntryClusteringPrefixSerializer(final Version version, final SerializationHeader header)
     {
-        if (!version.storeRows())
+        if (!version.storeRows() || header ==  null) //null header indicates streaming from pre-3.0 sstables
         {
-            return new ISerializer<ClusteringPrefix>()
+            return oldFormatSerializer(version);
+        }
+
+        return newFormatSerializer(version, header);
+    }
+
+    private ISerializer<ClusteringPrefix> oldFormatSerializer(final Version version)
+    {
+        return new ISerializer<ClusteringPrefix>()
+        {
+            SerializationHeader newHeader = SerializationHeader.makeWithoutStats(metadata);
+
+            public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
             {
-                public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
-                {
-                    // We should only use this for reading old sstable, never write new ones.
-                    throw new UnsupportedOperationException();
-                }
+                //we deserialize in the old format and serialize in the new format
+                ClusteringPrefix.serializer.serialize(clustering, out,
+                                                      version.correspondingMessagingVersion(),
+                                                      newHeader.clusteringTypes());
+            }
+
+            public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+            {
+                // We're reading the old cellname/composite
+                ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
+                assert bb.hasRemaining(); // empty cellnames were invalid
+
+                int clusteringSize = metadata.clusteringColumns().size();
+                // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
+                if (clusteringSize == 0)
+                    return Clustering.EMPTY;
 
-                public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+                if (!metadata.isCompound())
+                    return new Clustering(bb);
+
+                List<ByteBuffer> components = CompositeType.splitName(bb);
+                byte eoc = CompositeType.lastEOC(bb);
+
+                if (eoc == 0 || components.size() >= clusteringSize)
                 {
-                    // We're reading the old cellname/composite
-                    ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
-                    assert bb.hasRemaining(); // empty cellnames were invalid
-
-                    int clusteringSize = metadata.clusteringColumns().size();
-                    // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
-                    if (clusteringSize == 0)
-                        return Clustering.EMPTY;
-
-                    if (!metadata.isCompound())
-                        return new Clustering(bb);
-
-                    List<ByteBuffer> components = CompositeType.splitName(bb);
-                    byte eoc = CompositeType.lastEOC(bb);
-
-                    if (eoc == 0 || components.size() >= clusteringSize)
-                    {
-                        // That's a clustering.
-                        if (components.size() > clusteringSize)
-                            components = components.subList(0, clusteringSize);
-
-                        return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
-                    }
-                    else
-                    {
-                        // It's a range tombstone bound. It is a start since that's the only part we've ever included
-                        // in the index entries.
-                        Slice.Bound.Kind boundKind = eoc > 0
-                                                   ? Slice.Bound.Kind.EXCL_START_BOUND
-                                                   : Slice.Bound.Kind.INCL_START_BOUND;
-
-                        return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
-                    }
-                }
+                    // That's a clustering.
+                    if (components.size() > clusteringSize)
+                        components = components.subList(0, clusteringSize);
 
-                public long serializedSize(ClusteringPrefix clustering)
+                    return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
+                }
+                else
                 {
-                    // We should only use this for reading old sstable, never write new ones.
-                    throw new UnsupportedOperationException();
+                    // It's a range tombstone bound. It is a start since that's the only part we've ever included
+                    // in the index entries.
+                    Slice.Bound.Kind boundKind = eoc > 0
+                                                 ? Slice.Bound.Kind.EXCL_START_BOUND
+                                                 : Slice.Bound.Kind.INCL_START_BOUND;
+
+                    return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
                 }
-            };
-        }
+            }
 
-        return new ISerializer<ClusteringPrefix>()
+            public long serializedSize(ClusteringPrefix clustering)
+            {
+                return ClusteringPrefix.serializer.serializedSize(clustering, version.correspondingMessagingVersion(),
+                                                                  newHeader.clusteringTypes());
+            }
+        };
+    }
+
+
+    private ISerializer<ClusteringPrefix> newFormatSerializer(final Version version, final SerializationHeader header)
+    {
+        return new ISerializer<ClusteringPrefix>() //Reading and writing from/to the new sstable format
         {
             public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
             {
@@ -121,4 +134,5 @@ public class Serializers
             }
         };
     }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index 8ac3dcb..0e2012e 100644
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.IndexHelper;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
@@ -401,7 +401,7 @@ abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
         private int currentIndexIdx;
 
         // Marks the beginning of the block corresponding to currentIndexIdx.
-        private FileMark mark;
+        private DataPosition mark;
 
         public IndexState(Reader reader, ClusteringComparator comparator, RowIndexEntry indexEntry, boolean reversed)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
index 1dc6d1e..095d7f4 100644
--- a/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
+++ b/src/java/org/apache/cassandra/hints/ChecksummedDataInput.java
@@ -22,13 +22,9 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.zip.CRC32;
 
-import org.apache.cassandra.io.FSError;
-import org.apache.cassandra.io.FSReadError;
-import org.apache.cassandra.io.compress.ICompressor;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.RandomAccessReader;
-import org.apache.cassandra.schema.CompressionParams;
 
 /**
  * A {@link RandomAccessReader} wrapper that calctulates the CRC in place.
@@ -48,7 +44,7 @@ public class ChecksummedDataInput extends RandomAccessReader.RandomAccessReaderW
     private boolean crcUpdateDisabled;
 
     private long limit;
-    private FileMark limitMark;
+    private DataPosition limitMark;
 
     protected ChecksummedDataInput(Builder builder)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/hints/HintMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/hints/HintMessage.java b/src/java/org/apache/cassandra/hints/HintMessage.java
index e78738d..723ab6d 100644
--- a/src/java/org/apache/cassandra/hints/HintMessage.java
+++ b/src/java/org/apache/cassandra/hints/HintMessage.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.net.MessageOut;
 import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
 import org.apache.cassandra.utils.UUIDSerializer;
 
 /**
@@ -117,7 +117,7 @@ public final class HintMessage
             UUID hostId = UUIDSerializer.serializer.deserialize(in, version);
 
             long hintSize = in.readUnsignedVInt();
-            BytesReadTracker countingIn = new BytesReadTracker(in);
+            TrackedDataInputPlus countingIn = new TrackedDataInputPlus(in);
             try
             {
                 return new HintMessage(hostId, Hint.serializer.deserialize(countingIn, version));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
index 14f1ba7..9bd1145 100644
--- a/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/compress/CompressedSequentialWriter.java
@@ -32,7 +32,7 @@ import org.apache.cassandra.io.FSWriteError;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.DataIntegrityMetadata;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.SequentialWriter;
 import org.apache.cassandra.schema.CompressionParams;
@@ -153,7 +153,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     }
 
     @Override
-    public FileMark mark()
+    public DataPosition mark()
     {
         if (!buffer.hasRemaining())
             doFlush(0);
@@ -161,7 +161,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     }
 
     @Override
-    public synchronized void resetAndTruncate(FileMark mark)
+    public synchronized void resetAndTruncate(DataPosition mark)
     {
         assert mark instanceof CompressedFileWriterMark;
 
@@ -306,7 +306,7 @@ public class CompressedSequentialWriter extends SequentialWriter
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class CompressedFileWriterMark implements FileMark
+    protected static class CompressedFileWriterMark implements DataPosition
     {
         // chunk offset in the compressed file
         final long chunkOffset;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
index 365d469..f82db4e 100644
--- a/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
+++ b/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.io.IOError;
 import java.util.Iterator;
 
+import org.apache.cassandra.io.util.RewindableDataInput;
 import org.apache.cassandra.utils.AbstractIterator;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -29,7 +30,7 @@ import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.rows.*;
 import org.apache.cassandra.io.util.DataInputPlus;
 import org.apache.cassandra.io.util.FileDataInput;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.net.MessagingService;
 
 /**
@@ -113,11 +114,9 @@ public abstract class SSTableSimpleIterator extends AbstractIterator<Unfiltered>
                 // need to extract them. Which imply 2 passes (one to extract the static, then one for other value).
                 if (metadata.isStaticCompactTable())
                 {
-                    // Because we don't support streaming from old file version, the only case we should get there is for compaction,
-                    // where the DataInput should be a file based one.
-                    assert in instanceof FileDataInput;
-                    FileDataInput file = (FileDataInput)in;
-                    FileMark mark = file.mark();
+                    assert in instanceof RewindableDataInput;
+                    RewindableDataInput file = (RewindableDataInput)in;
+                    DataPosition mark = file.mark();
                     Row staticRow = LegacyLayout.extractStaticColumns(metadata, file, metadata.partitionColumns().statics);
                     file.reset(mark);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index 8a778b7..b9561ec 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -427,7 +427,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
                                              System.currentTimeMillis(),
                                              statsMetadata,
                                              OpenReason.NORMAL,
-                                             header.toHeader(metadata));
+                                             header == null? null : header.toHeader(metadata));
 
         // special implementation of load to use non-pooled SegmentedFile builders
         try(SegmentedFile.Builder ibuilder = new BufferedSegmentedFile.Builder();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index 4cbbd70..5f35029 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@ -80,7 +80,7 @@ public abstract class SSTableWriter extends SSTable implements Transactional
         this.keyCount = keyCount;
         this.repairedAt = repairedAt;
         this.metadataCollector = metadataCollector;
-        this.header = header;
+        this.header = header != null ? header : SerializationHeader.makeWithoutStats(metadata); //null header indicates streaming from pre-3.0 sstable
         this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, descriptor.version, header);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
index 2335e47..d3630d7 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
@@ -55,7 +55,7 @@ public class BigTableWriter extends SSTableWriter
     private final SegmentedFile.Builder dbuilder;
     protected final SequentialWriter dataFile;
     private DecoratedKey lastWrittenKey;
-    private FileMark dataMark;
+    private DataPosition dataMark;
 
     public BigTableWriter(Descriptor descriptor, 
                           Long keyCount, 
@@ -368,7 +368,7 @@ public class BigTableWriter extends SSTableWriter
         public final SegmentedFile.Builder builder;
         public final IndexSummaryBuilder summary;
         public final IFilter bf;
-        private FileMark mark;
+        private DataPosition mark;
 
         IndexWriter(long keyCount, final SequentialWriter dataFile)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/BytesReadTracker.java b/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
new file mode 100644
index 0000000..fc83856
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/BytesReadTracker.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+public interface BytesReadTracker
+{
+    public long getBytesRead();
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count);
+
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/DataPosition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/DataPosition.java b/src/java/org/apache/cassandra/io/util/DataPosition.java
new file mode 100644
index 0000000..e106dae
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/DataPosition.java
@@ -0,0 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+public interface DataPosition
+{}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileDataInput.java b/src/java/org/apache/cassandra/io/util/FileDataInput.java
index f56193b..1059b01 100644
--- a/src/java/org/apache/cassandra/io/util/FileDataInput.java
+++ b/src/java/org/apache/cassandra/io/util/FileDataInput.java
@@ -20,7 +20,7 @@ package org.apache.cassandra.io.util;
 import java.io.Closeable;
 import java.io.IOException;
 
-public interface FileDataInput extends DataInputPlus, Closeable
+public interface FileDataInput extends RewindableDataInput, Closeable
 {
     String getPath();
 
@@ -30,11 +30,5 @@ public interface FileDataInput extends DataInputPlus, Closeable
 
     void seek(long pos) throws IOException;
 
-    FileMark mark();
-
-    void reset(FileMark mark) throws IOException;
-
-    long bytesPastMark(FileMark mark);
-
     long getFilePointer();
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileMark.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileMark.java b/src/java/org/apache/cassandra/io/util/FileMark.java
deleted file mode 100644
index 781bc1e..0000000
--- a/src/java/org/apache/cassandra/io/util/FileMark.java
+++ /dev/null
@@ -1,20 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.cassandra.io.util;
-
-public interface FileMark {}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java b/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
index 425c7d6..a585215 100644
--- a/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
+++ b/src/java/org/apache/cassandra/io/util/FileSegmentInputStream.java
@@ -74,17 +74,17 @@ public class FileSegmentInputStream extends DataInputBuffer implements FileDataI
         return false;
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         throw new UnsupportedOperationException();
     }
 
-    public void reset(FileMark mark)
+    public void reset(DataPosition mark)
     {
         throw new UnsupportedOperationException();
     }
 
-    public long bytesPastMark(FileMark mark)
+    public long bytesPastMark(DataPosition mark)
     {
         return 0;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
index b495bf0..1943773 100644
--- a/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
+++ b/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
@@ -200,19 +200,19 @@ public class RandomAccessReader extends RebufferingInputStream implements FileDa
         return bytes;
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         markedPointer = current();
         return new BufferedRandomAccessFileMark(markedPointer);
     }
 
-    public void reset(FileMark mark)
+    public void reset(DataPosition mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
         seek(((BufferedRandomAccessFileMark) mark).pointer);
     }
 
-    public long bytesPastMark(FileMark mark)
+    public long bytesPastMark(DataPosition mark)
     {
         assert mark instanceof BufferedRandomAccessFileMark;
         long bytes = current() - ((BufferedRandomAccessFileMark) mark).pointer;
@@ -262,7 +262,7 @@ public class RandomAccessReader extends RebufferingInputStream implements FileDa
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class BufferedRandomAccessFileMark implements FileMark
+    protected static class BufferedRandomAccessFileMark implements DataPosition
     {
         final long pointer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInput.java b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
new file mode 100644
index 0000000..c202f60
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInput.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.IOException;
+
+public interface RewindableDataInput extends DataInputPlus
+{
+    DataPosition mark();
+
+    void reset(DataPosition mark) throws IOException;
+
+    long bytesPastMark(DataPosition mark);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
new file mode 100644
index 0000000..3a680f4
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/RewindableDataInputStreamPlus.java
@@ -0,0 +1,569 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.Closeable;
+import java.io.File;
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.cassandra.utils.Throwables.maybeFail;
+import static org.apache.cassandra.utils.Throwables.merge;
+
+/**
+ * Adds mark/reset functionality to another input stream by caching read bytes to a memory buffer and
+ * spilling to disk if necessary.
+ *
+ * When the stream is marked via {@link this#mark()} or {@link this#mark(int)}, up to
+ * <code>maxMemBufferSize</code> will be cached in memory (heap). If more than
+ * <code>maxMemBufferSize</code> bytes are read while the stream is marked, the
+ * following bytes are cached on the <code>spillFile</code> for up to <code>maxDiskBufferSize</code>.
+ *
+ * Please note that successive calls to {@link this#mark()} and {@link this#reset()} will write
+ * sequentially to the same <code>spillFile</code> until <code>maxDiskBufferSize</code> is reached.
+ * At this point, if less than <code>maxDiskBufferSize</code> bytes are currently cached on the
+ * <code>spillFile</code>, the remaining bytes are written to the beginning of the file,
+ * treating the <code>spillFile</code> as a circular buffer.
+ *
+ * If more than <code>maxMemBufferSize + maxDiskBufferSize</code> are cached while the stream is marked,
+ * the following {@link this#reset()} invocation will throw a {@link IllegalStateException}.
+ *
+ */
+public class RewindableDataInputStreamPlus extends FilterInputStream implements RewindableDataInput, Closeable
+{
+    private boolean marked = false;
+    private boolean exhausted = false;
+    private AtomicBoolean closed = new AtomicBoolean(false);
+
+    protected int memAvailable = 0;
+    protected int diskTailAvailable = 0;
+    protected int diskHeadAvailable = 0;
+
+    private final File spillFile;
+    private final int initialMemBufferSize;
+    private final int maxMemBufferSize;
+    private final int maxDiskBufferSize;
+
+    private volatile byte memBuffer[];
+    private int memBufferSize;
+    private RandomAccessFile spillBuffer;
+
+    private final DataInputPlus dataReader;
+
+    public RewindableDataInputStreamPlus(InputStream in, int initialMemBufferSize, int maxMemBufferSize,
+                                         File spillFile, int maxDiskBufferSize)
+    {
+        super(in);
+        dataReader = new DataInputStreamPlus(this);
+        this.initialMemBufferSize = initialMemBufferSize;
+        this.maxMemBufferSize = maxMemBufferSize;
+        this.spillFile = spillFile;
+        this.maxDiskBufferSize = maxDiskBufferSize;
+    }
+
+    /* RewindableDataInput methods */
+
+    /**
+     * Marks the current position of a stream to return to this position later via the {@link this#reset(DataPosition)} method.
+     * @return An empty @link{DataPosition} object
+     */
+    public DataPosition mark()
+    {
+        mark(0);
+        return new RewindableDataInputPlusMark();
+    }
+
+    /**
+     * Rewinds to the previously marked position via the {@link this#mark()} method.
+     * @param mark it's not possible to return to a custom position, so this parameter is ignored.
+     * @throws IOException if an error ocurs while resetting
+     */
+    public void reset(DataPosition mark) throws IOException
+    {
+        reset();
+    }
+
+    public long bytesPastMark(DataPosition mark)
+    {
+        return maxMemBufferSize - memAvailable + (diskTailAvailable == -1? 0 : maxDiskBufferSize - diskHeadAvailable - diskTailAvailable);
+    }
+
+
+    protected static class RewindableDataInputPlusMark implements DataPosition
+    {
+    }
+
+    /* InputStream methods */
+
+    public boolean markSupported()
+    {
+        return true;
+    }
+
+    /**
+     * Marks the current position of a stream to return to this position
+     * later via the {@link this#reset()} method.
+     * @param readlimit the maximum amount of bytes to cache
+     */
+    public synchronized void mark(int readlimit)
+    {
+        if (marked)
+            throw new IllegalStateException("Cannot mark already marked stream.");
+
+        if (memAvailable > 0 || diskHeadAvailable > 0 || diskTailAvailable > 0)
+            throw new IllegalStateException("Can only mark stream after reading previously marked data.");
+
+        marked = true;
+        memAvailable = maxMemBufferSize;
+        diskHeadAvailable = -1;
+        diskTailAvailable = -1;
+    }
+
+    public synchronized void reset() throws IOException
+    {
+        if (!marked)
+            throw new IOException("Must call mark() before calling reset().");
+
+        if (exhausted)
+            throw new IOException(String.format("Read more than capacity: %d bytes.", maxMemBufferSize + maxDiskBufferSize));
+
+        memAvailable = maxMemBufferSize - memAvailable;
+        memBufferSize = memAvailable;
+
+        if (diskTailAvailable == -1)
+        {
+            diskHeadAvailable = 0;
+            diskTailAvailable = 0;
+        }
+        else
+        {
+            int initialPos = diskTailAvailable > 0 ? 0 : (int)getIfNotClosed(spillBuffer).getFilePointer();
+            int diskMarkpos = initialPos + diskHeadAvailable;
+            getIfNotClosed(spillBuffer).seek(diskMarkpos);
+
+            diskHeadAvailable = diskMarkpos - diskHeadAvailable;
+            diskTailAvailable = (maxDiskBufferSize - diskTailAvailable) - diskMarkpos;
+        }
+
+        marked = false;
+    }
+
+    public int available() throws IOException
+    {
+
+        return super.available() + (marked? 0 : memAvailable + diskHeadAvailable + diskTailAvailable);
+    }
+
+    public int read() throws IOException
+    {
+        int read = readOne();
+        if (read == -1)
+            return read;
+
+        if (marked)
+        {
+            //mark exhausted
+            if (isExhausted(1))
+            {
+                exhausted = true;
+                return read;
+            }
+
+            writeOne(read);
+        }
+
+        return read;
+    }
+
+    public int read(byte[] b, int off, int len) throws IOException
+    {
+        int readBytes = readMulti(b, off, len);
+        if (readBytes == -1)
+            return readBytes;
+
+        if (marked)
+        {
+            //check we have space on buffer
+            if (isExhausted(readBytes))
+            {
+                exhausted = true;
+                return readBytes;
+            }
+
+            writeMulti(b, off, readBytes);
+        }
+
+        return readBytes;
+    }
+
+    private void maybeCreateDiskBuffer() throws IOException
+    {
+        if (spillBuffer == null)
+        {
+            if (!spillFile.getParentFile().exists())
+                spillFile.getParentFile().mkdirs();
+            spillFile.createNewFile();
+
+            this.spillBuffer = new RandomAccessFile(spillFile, "rw");
+        }
+    }
+
+
+    private int readOne() throws IOException
+    {
+        if (!marked)
+        {
+            if (memAvailable > 0)
+            {
+                int pos = memBufferSize - memAvailable;
+                memAvailable--;
+                return getIfNotClosed(memBuffer)[pos] & 0xff;
+            }
+
+            if (diskTailAvailable > 0 || diskHeadAvailable > 0)
+            {
+                int read = getIfNotClosed(spillBuffer).read();
+                if (diskTailAvailable > 0)
+                    diskTailAvailable--;
+                else if (diskHeadAvailable > 0)
+                    diskHeadAvailable++;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+                return read;
+            }
+        }
+
+        return getIfNotClosed(in).read();
+    }
+
+    private boolean isExhausted(int readBytes)
+    {
+        return exhausted || readBytes > memAvailable + (long)(diskTailAvailable == -1? maxDiskBufferSize : diskTailAvailable + diskHeadAvailable);
+    }
+
+    private int readMulti(byte[] b, int off, int len) throws IOException
+    {
+        int readBytes = 0;
+        if (!marked)
+        {
+            if (memAvailable > 0)
+            {
+                readBytes += memAvailable < len ? memAvailable : len;
+                int pos = memBufferSize - memAvailable;
+                System.arraycopy(memBuffer, pos, b, off, readBytes);
+                memAvailable -= readBytes;
+                off += readBytes;
+                len -= readBytes;
+            }
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int readFromTail = diskTailAvailable < len? diskTailAvailable : len;
+                getIfNotClosed(spillBuffer).read(b, off, readFromTail);
+                readBytes += readFromTail;
+                diskTailAvailable -= readFromTail;
+                off += readFromTail;
+                len -= readFromTail;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+            }
+            if (len > 0 && diskHeadAvailable > 0)
+            {
+                int readFromHead = diskHeadAvailable < len? diskHeadAvailable : len;
+                getIfNotClosed(spillBuffer).read(b, off, readFromHead);
+                readBytes += readFromHead;
+                diskHeadAvailable -= readFromHead;
+                off += readFromHead;
+                len -= readFromHead;
+            }
+        }
+
+        if (len > 0)
+            readBytes += getIfNotClosed(in).read(b, off, len);
+
+        return readBytes;
+    }
+
+    private void writeMulti(byte[] b, int off, int len) throws IOException
+    {
+        if (memAvailable > 0)
+        {
+            if (memBuffer == null)
+                memBuffer = new byte[initialMemBufferSize];
+            int pos = maxMemBufferSize - memAvailable;
+            int memWritten = memAvailable < len? memAvailable : len;
+            if (pos + memWritten >= getIfNotClosed(memBuffer).length)
+                growMemBuffer(pos, memWritten);
+            System.arraycopy(b, off, memBuffer, pos, memWritten);
+            off += memWritten;
+            len -= memWritten;
+            memAvailable -= memWritten;
+        }
+
+        if (len > 0)
+        {
+            if (diskTailAvailable == -1)
+            {
+                maybeCreateDiskBuffer();
+                diskHeadAvailable = (int)spillBuffer.getFilePointer();
+                diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
+            }
+
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int diskTailWritten = diskTailAvailable < len? diskTailAvailable : len;
+                getIfNotClosed(spillBuffer).write(b, off, diskTailWritten);
+                off += diskTailWritten;
+                len -= diskTailWritten;
+                diskTailAvailable -= diskTailWritten;
+                if (diskTailAvailable == 0)
+                    spillBuffer.seek(0);
+            }
+
+            if (len > 0 && diskTailAvailable > 0)
+            {
+                int diskHeadWritten = diskHeadAvailable < len? diskHeadAvailable : len;
+                getIfNotClosed(spillBuffer).write(b, off, diskHeadWritten);
+            }
+        }
+    }
+
+    private void writeOne(int value) throws IOException
+    {
+        if (memAvailable > 0)
+        {
+            if (memBuffer == null)
+                memBuffer = new byte[initialMemBufferSize];
+            int pos = maxMemBufferSize - memAvailable;
+            if (pos == getIfNotClosed(memBuffer).length)
+                growMemBuffer(pos, 1);
+            getIfNotClosed(memBuffer)[pos] = (byte)value;
+            memAvailable--;
+            return;
+        }
+
+        if (diskTailAvailable == -1)
+        {
+            maybeCreateDiskBuffer();
+            diskHeadAvailable = (int)spillBuffer.getFilePointer();
+            diskTailAvailable = maxDiskBufferSize - diskHeadAvailable;
+        }
+
+        if (diskTailAvailable > 0 || diskHeadAvailable > 0)
+        {
+            getIfNotClosed(spillBuffer).write(value);
+            if (diskTailAvailable > 0)
+                diskTailAvailable--;
+            else if (diskHeadAvailable > 0)
+                diskHeadAvailable--;
+            if (diskTailAvailable == 0)
+                spillBuffer.seek(0);
+            return;
+        }
+    }
+
+    public int read(byte[] b) throws IOException
+    {
+        return read(b, 0, b.length);
+    }
+
+    private void growMemBuffer(int pos, int writeSize)
+    {
+        int newSize = Math.min(2 * (pos + writeSize), maxMemBufferSize);
+        byte newBuffer[] = new byte[newSize];
+        System.arraycopy(memBuffer, 0, newBuffer, 0, (int)pos);
+        memBuffer = newBuffer;
+    }
+
+    public long skip(long n) throws IOException
+    {
+        long skipped = 0;
+
+        if (marked)
+        {
+            //if marked, we need to cache skipped bytes
+            while (n-- > 0 && read() != -1)
+            {
+                skipped++;
+            }
+            return skipped;
+        }
+
+        if (memAvailable > 0)
+        {
+            skipped += memAvailable < n ? memAvailable : n;
+            memAvailable -= skipped;
+            n -= skipped;
+        }
+        if (n > 0 && diskTailAvailable > 0)
+        {
+            int skipFromTail = diskTailAvailable < n? diskTailAvailable : (int)n;
+            getIfNotClosed(spillBuffer).skipBytes(skipFromTail);
+            diskTailAvailable -= skipFromTail;
+            skipped += skipFromTail;
+            n -= skipFromTail;
+            if (diskTailAvailable == 0)
+                spillBuffer.seek(0);
+        }
+        if (n > 0 && diskHeadAvailable > 0)
+        {
+            int skipFromHead = diskHeadAvailable < n? diskHeadAvailable : (int)n;
+            getIfNotClosed(spillBuffer).skipBytes(skipFromHead);
+            diskHeadAvailable -= skipFromHead;
+            skipped += skipFromHead;
+            n -= skipFromHead;
+        }
+
+        if (n > 0)
+            skipped += getIfNotClosed(in).skip(n);
+
+        return skipped;
+    }
+
+    private <T> T getIfNotClosed(T in) throws IOException {
+        if (closed.get())
+            throw new IOException("Stream closed");
+        return in;
+    }
+
+    public void close() throws IOException
+    {
+        close(true);
+    }
+
+    public void close(boolean closeUnderlying) throws IOException
+    {
+        if (closed.compareAndSet(false, true))
+        {
+            Throwable fail = null;
+            if (closeUnderlying)
+            {
+                try
+                {
+                    super.close();
+                }
+                catch (IOException e)
+                {
+                    fail = merge(fail, e);
+                }
+            }
+            try
+            {
+                if (spillBuffer != null)
+                {
+                    this.spillBuffer.close();
+                    this.spillBuffer = null;
+                }
+            } catch (IOException e)
+            {
+                fail = merge(fail, e);
+            }
+            try {
+                if (spillFile.exists())
+                {
+                    spillFile.delete();
+                }
+            }
+            catch (Throwable e)
+            {
+                fail = merge(fail, e);
+            }
+            maybeFail(fail, IOException.class);
+        }
+    }
+
+    /* DataInputPlus methods */
+
+    public void readFully(byte[] b) throws IOException
+    {
+        dataReader.readFully(b);
+    }
+
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        dataReader.readFully(b, off, len);
+    }
+
+    public int skipBytes(int n) throws IOException
+    {
+        return dataReader.skipBytes(n);
+    }
+
+    public boolean readBoolean() throws IOException
+    {
+        return dataReader.readBoolean();
+    }
+
+    public byte readByte() throws IOException
+    {
+        return dataReader.readByte();
+    }
+
+    public int readUnsignedByte() throws IOException
+    {
+        return dataReader.readUnsignedByte();
+    }
+
+    public short readShort() throws IOException
+    {
+        return dataReader.readShort();
+    }
+
+    public int readUnsignedShort() throws IOException
+    {
+        return dataReader.readUnsignedShort();
+    }
+
+    public char readChar() throws IOException
+    {
+        return dataReader.readChar();
+    }
+
+    public int readInt() throws IOException
+    {
+        return dataReader.readInt();
+    }
+
+    public long readLong() throws IOException
+    {
+        return dataReader.readLong();
+    }
+
+    public float readFloat() throws IOException
+    {
+        return dataReader.readFloat();
+    }
+
+    public double readDouble() throws IOException
+    {
+        return dataReader.readDouble();
+    }
+
+    public String readLine() throws IOException
+    {
+        return dataReader.readLine();
+    }
+
+    public String readUTF() throws IOException
+    {
+        return dataReader.readUTF();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/SequentialWriter.java b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
index dd49868..26316a2 100644
--- a/src/java/org/apache/cassandra/io/util/SequentialWriter.java
+++ b/src/java/org/apache/cassandra/io/util/SequentialWriter.java
@@ -297,7 +297,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
         return bufferOffset + (buffer == null ? 0 : buffer.position());
     }
 
-    public FileMark mark()
+    public DataPosition mark()
     {
         return new BufferedFileWriterMark(current());
     }
@@ -306,7 +306,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
      * Drops all buffered data that's past the limits of our new file mark + buffer capacity, or syncs and truncates
      * the underlying file to the marked position
      */
-    public void resetAndTruncate(FileMark mark)
+    public void resetAndTruncate(DataPosition mark)
     {
         assert mark instanceof BufferedFileWriterMark;
 
@@ -404,7 +404,7 @@ public class SequentialWriter extends BufferedDataOutputStreamPlus implements Tr
     /**
      * Class to hold a mark to the position of the file
      */
-    protected static class BufferedFileWriterMark implements FileMark
+    protected static class BufferedFileWriterMark implements DataPosition
     {
         final long pointer;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java b/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
new file mode 100644
index 0000000..dc5bbb6
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/TrackedDataInputPlus.java
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.IOException;
+
+/**
+ * This class is to track bytes read from given DataInput
+ */
+public class TrackedDataInputPlus implements DataInputPlus, BytesReadTracker
+{
+    private long bytesRead;
+    final DataInput source;
+
+    public TrackedDataInputPlus(DataInput source)
+    {
+        this.source = source;
+    }
+
+    public long getBytesRead()
+    {
+        return bytesRead;
+    }
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count)
+    {
+        bytesRead = count;
+    }
+
+    public boolean readBoolean() throws IOException
+    {
+        boolean bool = source.readBoolean();
+        bytesRead += 1;
+        return bool;
+    }
+
+    public byte readByte() throws IOException
+    {
+        byte b = source.readByte();
+        bytesRead += 1;
+        return b;
+    }
+
+    public char readChar() throws IOException
+    {
+        char c = source.readChar();
+        bytesRead += 2;
+        return c;
+    }
+
+    public double readDouble() throws IOException
+    {
+        double d = source.readDouble();
+        bytesRead += 8;
+        return d;
+    }
+
+    public float readFloat() throws IOException
+    {
+        float f = source.readFloat();
+        bytesRead += 4;
+        return f;
+    }
+
+    public void readFully(byte[] b, int off, int len) throws IOException
+    {
+        source.readFully(b, off, len);
+        bytesRead += len;
+    }
+
+    public void readFully(byte[] b) throws IOException
+    {
+        source.readFully(b);
+        bytesRead += b.length;
+    }
+
+    public int readInt() throws IOException
+    {
+        int i = source.readInt();
+        bytesRead += 4;
+        return i;
+    }
+
+    public String readLine() throws IOException
+    {
+        // since this method is deprecated and cannot track bytes read
+        // just throw exception
+        throw new UnsupportedOperationException();
+    }
+
+    public long readLong() throws IOException
+    {
+        long l = source.readLong();
+        bytesRead += 8;
+        return l;
+    }
+
+    public short readShort() throws IOException
+    {
+        short s = source.readShort();
+        bytesRead += 2;
+        return s;
+    }
+
+    public String readUTF() throws IOException
+    {
+        return DataInputStream.readUTF(this);
+    }
+
+    public int readUnsignedByte() throws IOException
+    {
+        int i = source.readUnsignedByte();
+        bytesRead += 1;
+        return i;
+    }
+
+    public int readUnsignedShort() throws IOException
+    {
+        int i = source.readUnsignedShort();
+        bytesRead += 2;
+        return i;
+    }
+
+    public int skipBytes(int n) throws IOException
+    {
+        int skipped = source.skipBytes(n);
+        bytesRead += skipped;
+        return skipped;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/util/TrackedInputStream.java b/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
new file mode 100644
index 0000000..f398d30
--- /dev/null
+++ b/src/java/org/apache/cassandra/io/util/TrackedInputStream.java
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.cassandra.io.util;
+
+import java.io.FilterInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * This class is to track bytes read from given DataInputStream
+ */
+public class TrackedInputStream extends FilterInputStream implements BytesReadTracker
+{
+    private long bytesRead;
+
+    public TrackedInputStream(InputStream source)
+    {
+        super(source);
+    }
+
+    public long getBytesRead()
+    {
+        return bytesRead;
+    }
+
+    /**
+     * reset counter to @param count
+     */
+    public void reset(long count)
+    {
+        bytesRead = count;
+    }
+
+    public int read() throws IOException
+    {
+        int read = super.read();
+        bytesRead += 1;
+        return read;
+    }
+
+    public int read(byte[] b, int off, int len) throws IOException
+    {
+        int read = super.read(b, off, len);
+        bytesRead += read;
+        return read;
+    }
+
+    public int read(byte[] b) throws IOException
+    {
+        int read = super.read(b);
+        bytesRead += read;
+        return read;
+    }
+
+    public long skip(long n) throws IOException
+    {
+        long skip = super.skip(n);
+        bytesRead += skip;
+        return skip;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f445e25..fd7f4b6 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -4221,6 +4221,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         return Collections.unmodifiableList(keyspaceNamesList);
     }
 
+
     public List<String> getNonSystemKeyspaces()
     {
         List<String> keyspaceNamesList = new ArrayList<>(Schema.instance.getNonSystemKeyspaces());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReader.java b/src/java/org/apache/cassandra/streaming/StreamReader.java
index 268f974..f8db26b 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@ -40,10 +40,13 @@ import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
 import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.Version;
+import org.apache.cassandra.io.util.RewindableDataInputStreamPlus;
 import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedInputStream;
+import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 
 
@@ -105,9 +108,9 @@ public class StreamReader
                      session.planId(), fileSeqNum, session.peer, repairedAt, totalSize, cfs.keyspace.getName(),
                      cfs.getColumnFamilyName());
 
-        DataInputStream dis = new DataInputStream(new LZFInputStream(Channels.newInputStream(channel)));
-        BytesReadTracker in = new BytesReadTracker(dis);
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, header.toHeader(cfs.metadata));
+        TrackedInputStream in = new TrackedInputStream(new LZFInputStream(Channels.newInputStream(channel)));
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+                                                                 totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
         {
@@ -131,12 +134,22 @@ public class StreamReader
             {
                 writer.abort(e);
             }
-            drain(dis, in.getBytesRead());
+            drain(in, in.getBytesRead());
             if (e instanceof IOException)
                 throw (IOException) e;
             else
                 throw Throwables.propagate(e);
         }
+        finally
+        {
+            if (deserializer != null)
+                deserializer.cleanup();
+        }
+    }
+
+    protected SerializationHeader getHeader(CFMetaData metadata)
+    {
+        return header != null? header.toHeader(metadata) : null; //pre-3.0 sstable have no SerializationHeader
     }
 
     protected SSTableMultiWriter createWriter(ColumnFamilyStore cfs, long totalSize, long repairedAt, SSTableFormat.Type format) throws IOException
@@ -146,8 +159,7 @@ public class StreamReader
             throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
         desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
 
-
-        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), session.getTransaction(cfId));
+        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, getHeader(cfs.metadata), session.getTransaction(cfId));
     }
 
     protected void drain(InputStream dis, long bytesRead) throws IOException
@@ -185,6 +197,13 @@ public class StreamReader
 
     public static class StreamDeserializer extends UnmodifiableIterator<Unfiltered> implements UnfilteredRowIterator
     {
+        public static final int INITIAL_MEM_BUFFER_SIZE = Integer.getInteger("cassandra.streamdes.initial_mem_buffer_size", 32768);
+        public static final int MAX_MEM_BUFFER_SIZE = Integer.getInteger("cassandra.streamdes.max_mem_buffer_size", 1048576);
+        public static final int MAX_SPILL_FILE_SIZE = Integer.getInteger("cassandra.streamdes.max_spill_file_size", Integer.MAX_VALUE);
+
+        public static final String BUFFER_FILE_PREFIX = "buf";
+        public static final String BUFFER_FILE_SUFFIX = "dat";
+
         private final CFMetaData metadata;
         private final DataInputPlus in;
         private final SerializationHeader header;
@@ -196,11 +215,20 @@ public class StreamReader
         private Row staticRow;
         private IOException exception;
 
-        public StreamDeserializer(CFMetaData metadata, DataInputPlus in, Version version, SerializationHeader header)
+        public StreamDeserializer(CFMetaData metadata, InputStream in, Version version, SerializationHeader header,
+                                  long totalSize, UUID sessionId) throws IOException
         {
-            assert version.storeRows() : "We don't allow streaming from pre-3.0 nodes";
             this.metadata = metadata;
-            this.in = in;
+            // streaming pre-3.0 sstables require mark/reset support from source stream
+            if (version.correspondingMessagingVersion() < MessagingService.VERSION_30)
+            {
+                logger.trace("Initializing rewindable input stream for reading legacy sstable with {} bytes with following " +
+                             "parameters: initial_mem_buffer_size={}, max_mem_buffer_size={}, max_spill_file_size={}.",
+                             totalSize, INITIAL_MEM_BUFFER_SIZE, MAX_MEM_BUFFER_SIZE, MAX_SPILL_FILE_SIZE);
+                File bufferFile = getTempBufferFile(metadata, totalSize, sessionId);
+                this.in = new RewindableDataInputStreamPlus(in, INITIAL_MEM_BUFFER_SIZE, MAX_MEM_BUFFER_SIZE, bufferFile, MAX_SPILL_FILE_SIZE);
+            } else
+                this.in = new DataInputPlus.DataInputStreamPlus(in);
             this.helper = new SerializationHelper(metadata, version.correspondingMessagingVersion(), SerializationHelper.Flag.PRESERVE_SIZE);
             this.header = header;
         }
@@ -292,5 +320,41 @@ public class StreamReader
         public void close()
         {
         }
+
+        /* We have a separate cleanup method because sometimes close is called before exhausting the
+           StreamDeserializer (for instance, when enclosed in an try-with-resources wrapper, such as in
+           BigTableWriter.append()).
+         */
+        public void cleanup()
+        {
+            if (in instanceof RewindableDataInputStreamPlus)
+            {
+                try
+                {
+                    ((RewindableDataInputStreamPlus) in).close(false);
+                }
+                catch (IOException e)
+                {
+                    logger.warn("Error while closing RewindableDataInputStreamPlus.", e);
+                }
+            }
+        }
+
+        private static File getTempBufferFile(CFMetaData metadata, long totalSize, UUID sessionId) throws IOException
+        {
+            ColumnFamilyStore cfs = Keyspace.open(metadata.ksName).getColumnFamilyStore(metadata.cfName);
+            if (cfs == null)
+            {
+                // schema was dropped during streaming
+                throw new RuntimeException(String.format("CF %s.%s was dropped during streaming", metadata.ksName, metadata.cfName));
+            }
+
+            long maxSize = Math.min(MAX_SPILL_FILE_SIZE, totalSize);
+            File tmpDir = cfs.getDirectories().getTemporaryWriteableDirectoryAsFile(maxSize);
+            if (tmpDir == null)
+                throw new IOException(String.format("No sufficient disk space to stream legacy sstable from {}.{}. " +
+                                                         "Required disk space: %s.", FBUtilities.prettyPrintMemory(maxSize)));
+            return new File(tmpDir, String.format("%s-%s.%s", BUFFER_FILE_PREFIX, sessionId, BUFFER_FILE_SUFFIX));
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index 5210d5b..9719587 100644
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@ -24,7 +24,6 @@ import java.nio.channels.ReadableByteChannel;
 
 import com.google.common.base.Throwables;
 
-import org.apache.cassandra.db.DecoratedKey;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 
 import org.slf4j.Logger;
@@ -38,7 +37,7 @@ import org.apache.cassandra.streaming.ProgressInfo;
 import org.apache.cassandra.streaming.StreamReader;
 import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.streaming.messages.FileMessageHeader;
-import org.apache.cassandra.utils.BytesReadTracker;
+import org.apache.cassandra.io.util.TrackedInputStream;
 import org.apache.cassandra.utils.Pair;
 
 /**
@@ -83,8 +82,10 @@ public class CompressedStreamReader extends StreamReader
 
         CompressedInputStream cis = new CompressedInputStream(Channels.newInputStream(channel), compressionInfo,
                                                               inputVersion.compressedChecksumType(), cfs::getCrcCheckChance);
-        BytesReadTracker in = new BytesReadTracker(new DataInputStream(cis));
-        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, header.toHeader(cfs.metadata));
+        TrackedInputStream in = new TrackedInputStream(cis);
+
+        StreamDeserializer deserializer = new StreamDeserializer(cfs.metadata, in, inputVersion, getHeader(cfs.metadata),
+                                                                 totalSize, session.planId());
         SSTableMultiWriter writer = null;
         try
         {
@@ -115,17 +116,22 @@ public class CompressedStreamReader extends StreamReader
         {
             if (deserializer != null)
                 logger.warn("[Stream {}] Error while reading partition {} from stream on ks='{}' and table='{}'.",
-                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getColumnFamilyName());
+                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getTableName());
             if (writer != null)
             {
                 writer.abort(e);
             }
-            drain(cis, in.getBytesRead());
+            drain(in, in.getBytesRead());
             if (e instanceof IOException)
                 throw (IOException) e;
             else
                 throw Throwables.propagate(e);
         }
+        finally
+        {
+            if (deserializer != null)
+                deserializer.cleanup();
+        }
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
index e1e13b7..2b5047d 100644
--- a/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
+++ b/src/java/org/apache/cassandra/streaming/messages/FileMessageHeader.java
@@ -204,7 +204,7 @@ public class FileMessageHeader
             out.writeLong(header.repairedAt);
             out.writeInt(header.sstableLevel);
 
-            if (version >= StreamMessage.VERSION_30)
+            if (version >= StreamMessage.VERSION_30 && header.version.storeRows())
                 SerializationHeader.serializer.serialize(header.version, header.header, out);
             return compressionInfo;
         }
@@ -227,7 +227,7 @@ public class FileMessageHeader
             CompressionInfo compressionInfo = CompressionInfo.serializer.deserialize(in, MessagingService.current_version);
             long repairedAt = in.readLong();
             int sstableLevel = in.readInt();
-            SerializationHeader.Component header = version >= StreamMessage.VERSION_30
+            SerializationHeader.Component header = version >= StreamMessage.VERSION_30 && sstableVersion.storeRows()
                                                  ? SerializationHeader.serializer.deserialize(sstableVersion, in)
                                                  : null;
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/src/java/org/apache/cassandra/tools/nodetool/Repair.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/nodetool/Repair.java b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
index 84b463c..bd1a916 100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Repair.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Repair.java
@@ -56,7 +56,7 @@ public class Repair extends NodeToolCmd
     private boolean localDC = false;
 
     @Option(title = "specific_dc", name = {"-dc", "--in-dc"}, description = "Use -dc to repair specific datacenters")
-    private List<String> specificDataCenters = new ArrayList<>();
+    private List<String> specificDataCenters = new ArrayList<>();;
 
     @Option(title = "specific_host", name = {"-hosts", "--in-hosts"}, description = "Use -hosts to repair specific hosts")
     private List<String> specificHosts = new ArrayList<>();


[04/12] cassandra git commit: Support streaming of older version sstables in 3.0

Posted by yu...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
new file mode 100644
index 0000000..a29a600
--- /dev/null
+++ b/test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple_counter_compact/ma-1-big-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Statistics.db
+Digest.crc32
+Summary.db
+Data.db
+TOC.txt
+Filter.db
+CompressionInfo.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
index 501f4ae..3a1f348 100644
--- a/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
+++ b/test/unit/org/apache/cassandra/AbstractSerializationsTester.java
@@ -25,7 +25,6 @@ import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.io.util.DataOutputStreamPlus;
 import org.apache.cassandra.io.util.BufferedDataOutputStreamPlus;
-import org.apache.cassandra.io.util.NIODataInputStream;
 import org.apache.cassandra.net.MessagingService;
 
 import java.io.File;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/db/DirectoriesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/DirectoriesTest.java b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
index 8732881..f864bbc 100644
--- a/test/unit/org/apache/cassandra/db/DirectoriesTest.java
+++ b/test/unit/org/apache/cassandra/db/DirectoriesTest.java
@@ -243,46 +243,76 @@ public class DirectoriesTest
         for (CFMetaData cfm : CFM)
         {
             Directories directories = new Directories(cfm);
-            Directories.SSTableLister lister;
-            Set<File> listed;
+            checkFiles(cfm, directories);
+        }
+    }
 
-            // List all but no snapshot, backup
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+    private void checkFiles(CFMetaData cfm, Directories directories)
+    {
+        Directories.SSTableLister lister;
+        Set<File> listed;// List all but no snapshot, backup
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
+        }
 
-            // List all but including backup (but no snapshot)
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+        // List all but including backup (but no snapshot)
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW).includeBackups(true);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
+        }
 
-            // Skip temporary and compacted
-            lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
-            listed = new HashSet<>(lister.listFiles());
-            for (File f : files.get(cfm.cfName))
-            {
-                if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else if (f.getName().contains("tmp-"))
-                    assert !listed.contains(f) : f + " should not be listed";
-                else
-                    assert listed.contains(f) : f + " is missing";
-            }
+        // Skip temporary and compacted
+        lister = directories.sstableLister(Directories.OnTxnErr.THROW).skipTemporary(true);
+        listed = new HashSet<>(lister.listFiles());
+        for (File f : files.get(cfm.cfName))
+        {
+            if (f.getPath().contains(Directories.SNAPSHOT_SUBDIR) || f.getPath().contains(Directories.BACKUPS_SUBDIR))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else if (f.getName().contains("tmp-"))
+                assertFalse(f + " should not be listed", listed.contains(f));
+            else
+                assertTrue(f + " is missing", listed.contains(f));
         }
     }
 
+    @Test
+    public void testTemporaryFile() throws IOException
+    {
+        for (CFMetaData cfm : CFM)
+        {
+            Directories directories = new Directories(cfm);
+
+            File tempDir = directories.getTemporaryWriteableDirectoryAsFile(10);
+            tempDir.mkdir();
+            File tempFile = new File(tempDir, "tempFile");
+            tempFile.createNewFile();
+
+            assertTrue(tempDir.exists());
+            assertTrue(tempFile.exists());
+
+            //make sure temp dir/file will not affect existing sstable listing
+            checkFiles(cfm, directories);
+
+            directories.removeTemporaryDirectories();
+
+            //make sure temp dir/file deletion will not affect existing sstable listing
+            checkFiles(cfm, directories);
+
+            assertFalse(tempDir.exists());
+            assertFalse(tempFile.exists());
+        }
+    }
 
     @Test
     public void testDiskFailurePolicy_best_effort()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/gms/SerializationsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/gms/SerializationsTest.java b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
index 8b7ad1f..e50b461 100644
--- a/test/unit/org/apache/cassandra/gms/SerializationsTest.java
+++ b/test/unit/org/apache/cassandra/gms/SerializationsTest.java
@@ -19,7 +19,6 @@
 package org.apache.cassandra.gms;
 
 import org.apache.cassandra.AbstractSerializationsTester;
-import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
 import org.apache.cassandra.io.util.DataInputPlus.DataInputStreamPlus;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
index 9154d79..74b5c74 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -30,7 +30,7 @@ import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.io.sstable.CorruptSSTableException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.MmappedRegions;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriter;
@@ -84,7 +84,7 @@ public class CompressedRandomAccessReaderTest
                 for (int i = 0; i < 20; i++)
                     writer.write("x".getBytes());
 
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
                 // write enough garbage to create new chunks:
                 for (int i = 0; i < 40; ++i)
                     writer.write("y".getBytes());
@@ -126,7 +126,7 @@ public class CompressedRandomAccessReaderTest
                 : SequentialWriter.open(f))
             {
                 writer.write("The quick ".getBytes());
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
                 writer.write("blue fox jumps over the lazy dog".getBytes());
 
                 // write enough to be sure to change chunk

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
index 56c83da..9b09f0b 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -38,7 +38,7 @@ import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.db.marshal.UTF8Type;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.io.util.ChannelProxy;
-import org.apache.cassandra.io.util.FileMark;
+import org.apache.cassandra.io.util.DataPosition;
 import org.apache.cassandra.io.util.RandomAccessReader;
 import org.apache.cassandra.io.util.SequentialWriterTest;
 import org.apache.cassandra.schema.CompressionParams;
@@ -104,7 +104,7 @@ public class CompressedSequentialWriterTest extends SequentialWriterTest
                 dataPost.flip();
 
                 writer.write(dataPre);
-                FileMark mark = writer.mark();
+                DataPosition mark = writer.mark();
 
                 // Write enough garbage to transition chunk
                 for (int i = 0; i < CompressionParams.DEFAULT_CHUNK_LENGTH; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 00727b8..4b9a769 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -21,14 +21,13 @@ import java.io.File;
 import java.io.FileInputStream;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import java.util.ArrayList;
-import java.util.HashSet;
 import java.util.List;
 import java.util.Random;
-import java.util.Set;
 
+import org.junit.After;
 import org.junit.Assert;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -37,16 +36,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.cql3.QueryProcessor;
 import org.apache.cassandra.cql3.UntypedResultSet;
 import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DeletionTime;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.rows.SliceableUnfilteredRowIterator;
-import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.marshal.BytesType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -55,7 +48,6 @@ import org.apache.cassandra.io.sstable.format.SSTableFormat;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
 import org.apache.cassandra.io.sstable.format.Version;
 import org.apache.cassandra.io.sstable.format.big.BigFormat;
-import org.apache.cassandra.schema.KeyspaceParams;
 import org.apache.cassandra.service.CacheService;
 import org.apache.cassandra.service.StorageService;
 import org.apache.cassandra.streaming.StreamPlan;
@@ -63,8 +55,6 @@ import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
 /**
  * Tests backwards compatibility for SSTables
  */
@@ -73,10 +63,7 @@ public class LegacySSTableTest
     private static final Logger logger = LoggerFactory.getLogger(LegacySSTableTest.class);
 
     public static final String LEGACY_SSTABLE_PROP = "legacy-sstable-root";
-    public static final String KSNAME = "Keyspace1";
-    public static final String CFNAME = "Standard1";
 
-    public static Set<String> TEST_DATA;
     public static File LEGACY_SSTABLE_ROOT;
 
     /**
@@ -104,58 +91,78 @@ public class LegacySSTableTest
     public static void defineSchema() throws ConfigurationException
     {
         SchemaLoader.prepareServer();
-
-        CFMetaData metadata = CFMetaData.Builder.createDense(KSNAME, CFNAME, false, false)
-                                                .addPartitionKey("key", BytesType.instance)
-                                                .addClusteringColumn("column", BytesType.instance)
-                                                .addRegularColumn("value", BytesType.instance)
-                                                .build();
-
-        SchemaLoader.createKeyspace(KSNAME,
-                                    KeyspaceParams.simple(1),
-                                    metadata);
-        beforeClass();
-    }
-
-    public static void beforeClass()
-    {
+        StorageService.instance.initServer();
         Keyspace.setInitialized();
+        createKeyspace();
+        for (String legacyVersion : legacyVersions)
+        {
+            createTables(legacyVersion);
+        }
         String scp = System.getProperty(LEGACY_SSTABLE_PROP);
         assert scp != null;
         LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
         assert LEGACY_SSTABLE_ROOT.isDirectory();
+    }
 
-        TEST_DATA = new HashSet<String>();
-        for (int i = 100; i < 1000; ++i)
-            TEST_DATA.add(Integer.toString(i));
+    @After
+    public void tearDown()
+    {
+        for (String legacyVersion : legacyVersions)
+        {
+            truncateTables(legacyVersion);
+        }
     }
 
     /**
      * Get a descriptor for the legacy sstable at the given version.
      */
-    protected Descriptor getDescriptor(String ver)
+    protected Descriptor getDescriptor(String legacyVersion, String table)
     {
-        File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
-        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, SSTableFormat.Type.LEGACY);
+        return new Descriptor(legacyVersion, getTableDir(legacyVersion, table), "legacy_tables", table, 1,
+                              BigFormat.instance.getVersion(legacyVersion).hasNewFileName()?
+                              SSTableFormat.Type.BIG :SSTableFormat.Type.LEGACY);
     }
 
     @Test
-    public void testStreaming() throws Throwable
+    public void testLoadLegacyCqlTables() throws Exception
     {
-        StorageService.instance.initServer();
+        for (String legacyVersion : legacyVersions)
+        {
+            logger.info("Loading legacy version: {}", legacyVersion);
+            loadLegacyTables(legacyVersion);
+            CacheService.instance.invalidateKeyCache();
+            long startCount = CacheService.instance.keyCache.size();
+            verifyReads(legacyVersion);
+            verifyCache(legacyVersion, startCount);
+        }
+    }
+
+    @Test
+    public void testStreamLegacyCqlTables() throws Exception
+    {
+        for (String legacyVersion : legacyVersions)
+        {
+            streamLegacyTables(legacyVersion);
+            verifyReads(legacyVersion);
+        }
+    }
 
-        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
+    private void streamLegacyTables(String legacyVersion) throws Exception
+    {
+        for (int compact = 0; compact <= 1; compact++)
         {
-            if (!new File(LEGACY_SSTABLE_ROOT + File.separator + version.getName() + File.separator + KSNAME).isDirectory())
-                continue;
-            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatibleForStreaming())
-                testStreaming(version.getName());
+            logger.info("Streaming legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
+            streamLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
         }
     }
 
-    private void testStreaming(String version) throws Exception
+    private void streamLegacyTable(String tablePattern, String legacyVersion, String compactNameSuffix) throws Exception
     {
-        SSTableReader sstable = SSTableReader.open(getDescriptor(version));
+        String table = String.format(tablePattern, legacyVersion, compactNameSuffix);
+        SSTableReader sstable = SSTableReader.open(getDescriptor(legacyVersion, table));
         IPartitioner p = sstable.getPartitioner();
         List<Range<Token>> ranges = new ArrayList<>();
         ranges.add(new Range<>(p.getMinimumToken(), p.getToken(ByteBufferUtil.bytes("100"))));
@@ -166,94 +173,41 @@ public class LegacySSTableTest
                                                                sstable.estimatedKeysForRanges(ranges), sstable.getSSTableMetadata().repairedAt));
         new StreamPlan("LegacyStreamingTest").transferFiles(FBUtilities.getBroadcastAddress(), details)
                                              .execute().get();
-
-        ColumnFamilyStore cfs = Keyspace.open(KSNAME).getColumnFamilyStore(CFNAME);
-        assert cfs.getLiveSSTables().size() == 1;
-        sstable = cfs.getLiveSSTables().iterator().next();
-        for (String keystring : TEST_DATA)
-        {
-            ByteBuffer key = bytes(keystring);
-
-            SliceableUnfilteredRowIterator iter = sstable.iterator(Util.dk(key), ColumnFilter.selectionBuilder().add(cfs.metadata.getColumnDefinition(bytes("name"))).build(), false, false);
-
-            // check not deleted (CASSANDRA-6527)
-            assert iter.partitionLevelDeletion().equals(DeletionTime.LIVE);
-            assert iter.next().clustering().get(0).equals(key);
-        }
-        sstable.selfRef().release();
-    }
-
-    @Test
-    public void testVersions() throws Throwable
-    {
-        boolean notSkipped = false;
-
-        for (File version : LEGACY_SSTABLE_ROOT.listFiles())
-        {
-            if (!new File(LEGACY_SSTABLE_ROOT + File.separator + version.getName() + File.separator + KSNAME).isDirectory())
-                continue;
-            if (Version.validate(version.getName()) && SSTableFormat.Type.LEGACY.info.getVersion(version.getName()).isCompatible())
-            {
-                notSkipped = true;
-                testVersion(version.getName());
-            }
-        }
-
-        assert notSkipped;
     }
 
-    public void testVersion(String version) throws Throwable
+    private static void loadLegacyTables(String legacyVersion) throws Exception
     {
-        try
-        {
-            ColumnFamilyStore cfs = Keyspace.open(KSNAME).getColumnFamilyStore(CFNAME);
-
-
-            SSTableReader reader = SSTableReader.open(getDescriptor(version));
-            for (String keystring : TEST_DATA)
-            {
-
-                ByteBuffer key = bytes(keystring);
-
-                SliceableUnfilteredRowIterator iter = reader.iterator(Util.dk(key), ColumnFilter.selection(cfs.metadata.partitionColumns()), false, false);
-
-                // check not deleted (CASSANDRA-6527)
-                assert iter.partitionLevelDeletion().equals(DeletionTime.LIVE);
-                assert iter.next().clustering().get(0).equals(key);
-            }
-
-            // TODO actually test some reads
-        }
-        catch (Throwable e)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            System.err.println("Failed to read " + version);
-            throw e;
+            logger.info("Preparing legacy version {}{}", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact));
+            loadLegacyTable("legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact));
         }
     }
 
-    @Test
-    public void testLegacyCqlTables() throws Exception
+    private static void verifyCache(String legacyVersion, long startCount) throws InterruptedException, java.util.concurrent.ExecutionException
     {
-        createKeyspace();
-
-        loadLegacyTables();
+        //For https://issues.apache.org/jira/browse/CASSANDRA-10778
+        //Validate whether the key cache successfully saves in the presence of old keys as
+        //well as loads the correct number of keys
+        long endCount = CacheService.instance.keyCache.size();
+        Assert.assertTrue(endCount > startCount);
+        CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
+        CacheService.instance.invalidateKeyCache();
+        Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
+        CacheService.instance.keyCache.loadSaved();
+        if (BigFormat.instance.getVersion(legacyVersion).storeRows())
+            Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
+        else
+            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
     }
 
-    private static void loadLegacyTables() throws Exception
+    private static void verifyReads(String legacyVersion)
     {
-        for (String legacyVersion : legacyVersions)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            logger.info("Preparing legacy version {}", legacyVersion);
-
-            createTables(legacyVersion);
-
-            loadLegacyTable("legacy_%s_simple", legacyVersion);
-            loadLegacyTable("legacy_%s_simple_counter", legacyVersion);
-            loadLegacyTable("legacy_%s_clust", legacyVersion);
-            loadLegacyTable("legacy_%s_clust_counter", legacyVersion);
-
-            CacheService.instance.invalidateKeyCache();
-            long startCount = CacheService.instance.keyCache.size();
             for (int ck = 0; ck < 50; ck++)
             {
                 String ckValue = Integer.toString(ck) + longString;
@@ -265,58 +219,94 @@ public class LegacySSTableTest
                     UntypedResultSet rs;
                     if (ck == 0)
                     {
-                        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple WHERE pk=?", legacyVersion), pkValue);
-                        Assert.assertNotNull(rs);
-                        Assert.assertEquals(1, rs.size());
-                        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
-                        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter WHERE pk=?", legacyVersion), pkValue);
-                        Assert.assertNotNull(rs);
-                        Assert.assertEquals(1, rs.size());
-                        Assert.assertEquals(1L, rs.one().getLong("val"));
+                        readSimpleTable(legacyVersion, getCompactNameSuffix(compact),  pkValue);
+                        readSimpleCounterTable(legacyVersion, getCompactNameSuffix(compact), pkValue);
                     }
 
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
-                    assertLegacyClustRows(1, rs);
-
-                    String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
-                    String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion), pkValue, ckValue, ckValue2, ckValue3);
-                    assertLegacyClustRows(3, rs);
-
-                    rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter WHERE pk=? AND ck=?", legacyVersion), pkValue, ckValue);
-                    Assert.assertNotNull(rs);
-                    Assert.assertEquals(1, rs.size());
-                    Assert.assertEquals(1L, rs.one().getLong("val"));
+                    readClusteringTable(legacyVersion, getCompactNameSuffix(compact), ck, ckValue, pkValue);
+                    readClusteringCounterTable(legacyVersion, getCompactNameSuffix(compact), ckValue, pkValue);
                 }
             }
-
-            //For https://issues.apache.org/jira/browse/CASSANDRA-10778
-            //Validate whether the key cache successfully saves in the presence of old keys as
-            //well as loads the correct number of keys
-            long endCount = CacheService.instance.keyCache.size();
-            Assert.assertTrue(endCount > startCount);
-            CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
-            CacheService.instance.invalidateKeyCache();
-            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
-            CacheService.instance.keyCache.loadSaved();
-            if (BigFormat.instance.getVersion(legacyVersion).storeRows())
-                Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
-            else
-                Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
         }
     }
 
-    private void createKeyspace()
+    private static void readClusteringCounterTable(String legacyVersion, String compactSuffix, String ckValue, String pkValue)
+    {
+        logger.debug("Read legacy_{}_clust_counter{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals(1L, rs.one().getLong("val"));
+    }
+
+    private static void readClusteringTable(String legacyVersion, String compactSuffix, int ck, String ckValue, String pkValue)
+    {
+        logger.debug("Read legacy_{}_clust{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
+        assertLegacyClustRows(1, rs);
+
+        String ckValue2 = Integer.toString(ck < 10 ? 40 : ck - 1) + longString;
+        String ckValue3 = Integer.toString(ck > 39 ? 10 : ck + 1) + longString;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust%s WHERE pk=? AND ck IN (?, ?, ?)", legacyVersion, compactSuffix), pkValue, ckValue, ckValue2, ckValue3);
+        assertLegacyClustRows(3, rs);
+    }
+
+    private static void readSimpleCounterTable(String legacyVersion, String compactSuffix, String pkValue)
+    {
+        logger.debug("Read legacy_{}_simple_counter{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals(1L, rs.one().getLong("val"));
+    }
+
+    private static void readSimpleTable(String legacyVersion, String compactSuffix, String pkValue)
+    {
+        logger.debug("Read simple: legacy_{}_simple{}", legacyVersion, compactSuffix);
+        UntypedResultSet rs;
+        rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
+        Assert.assertNotNull(rs);
+        Assert.assertEquals(1, rs.size());
+        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
+    }
+
+    private static void createKeyspace()
     {
         QueryProcessor.executeInternal("CREATE KEYSPACE legacy_tables WITH replication = {'class': 'SimpleStrategy', 'replication_factor': '1'}");
     }
 
     private static void createTables(String legacyVersion)
     {
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple (pk text PRIMARY KEY, val text)", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter (pk text PRIMARY KEY, val counter)", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust (pk text, ck text, val text, PRIMARY KEY (pk, ck))", legacyVersion));
-        QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter (pk text, ck text, val counter, PRIMARY KEY (pk, ck))", legacyVersion));
+        for (int i=0; i<=1; i++)
+        {
+            String compactSuffix = getCompactNameSuffix(i);
+            String tableSuffix = i == 0? "" : " WITH COMPACT STORAGE";
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple%s (pk text PRIMARY KEY, val text)%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_simple_counter%s (pk text PRIMARY KEY, val counter)%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust%s (pk text, ck text, val text, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
+            QueryProcessor.executeInternal(String.format("CREATE TABLE legacy_tables.legacy_%s_clust_counter%s (pk text, ck text, val counter, PRIMARY KEY (pk, ck))%s", legacyVersion, compactSuffix, tableSuffix));
+        }
+    }
+
+    private static String getCompactNameSuffix(int i)
+    {
+        return i == 0? "" : "_compact";
+    }
+
+    private static void truncateTables(String legacyVersion)
+    {
+        for (int compact = 0; compact <= 1; compact++)
+        {
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_simple_counter%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust%s", legacyVersion, getCompactNameSuffix(compact)));
+            QueryProcessor.executeInternal(String.format("TRUNCATE legacy_tables.legacy_%s_clust_counter%s", legacyVersion, getCompactNameSuffix(compact)));
+        }
+        CacheService.instance.invalidateCounterCache();
+        CacheService.instance.invalidateKeyCache();
     }
 
     private static void assertLegacyClustRows(int count, UntypedResultSet rs)
@@ -332,9 +322,9 @@ public class LegacySSTableTest
         }
     }
 
-    private static void loadLegacyTable(String tablePattern, String legacyVersion) throws IOException
+    private static void loadLegacyTable(String tablePattern, String legacyVersion, String compactSuffix) throws IOException
     {
-        String table = String.format(tablePattern, legacyVersion);
+        String table = String.format(tablePattern, legacyVersion, compactSuffix);
 
         logger.info("Loading legacy table {}", table);
 
@@ -349,7 +339,7 @@ public class LegacySSTableTest
     }
 
     /**
-     * Generates sstables for 4 CQL tables (see {@link #createTables(String)}) in <i>current</i>
+     * Generates sstables for 8 CQL tables (see {@link #createTables(String)}) in <i>current</i>
      * sstable format (version) into {@code test/data/legacy-sstables/VERSION}, where
      * {@code VERSION} matches {@link Version#getVersion() BigFormat.latestVersion.getVersion()}.
      * <p>
@@ -357,13 +347,10 @@ public class LegacySSTableTest
      * during development. I.e. remove the {@code @Ignore} annotation temporarily.
      * </p>
      */
-    @Test
     @Ignore
+    @Test
     public void testGenerateSstables() throws Throwable
     {
-        createKeyspace();
-        createTables(BigFormat.latestVersion.getVersion());
-
         Random rand = new Random();
         StringBuilder sb = new StringBuilder();
         for (int i = 0; i < 128; i++)
@@ -372,25 +359,28 @@ public class LegacySSTableTest
         }
         String randomString = sb.toString();
 
-        for (int pk = 0; pk < 5; pk++)
+        for (int compact = 0; compact <= 1; compact++)
         {
-            String valPk = Integer.toString(pk);
-            QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple (pk, val) VALUES ('%s', '%s')",
-                                                         BigFormat.latestVersion, valPk, "foo bar baz"));
+            for (int pk = 0; pk < 5; pk++)
+            {
+                String valPk = Integer.toString(pk);
+                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_simple%s (pk, val) VALUES ('%s', '%s')",
+                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, "foo bar baz"));
 
-            QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter SET val = val + 1 WHERE pk = '%s'",
-                                                         BigFormat.latestVersion, valPk));
+                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_simple_counter%s SET val = val + 1 WHERE pk = '%s'",
+                                                             BigFormat.latestVersion, getCompactNameSuffix(compact), valPk));
 
-            for (int ck = 0; ck < 50; ck++)
-            {
-                String valCk = Integer.toString(ck);
+                for (int ck = 0; ck < 50; ck++)
+                {
+                    String valCk = Integer.toString(ck);
 
-                QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust (pk, ck, val) VALUES ('%s', '%s', '%s')",
-                                                             BigFormat.latestVersion, valPk, valCk + longString, randomString));
+                    QueryProcessor.executeInternal(String.format("INSERT INTO legacy_tables.legacy_%s_clust%s (pk, ck, val) VALUES ('%s', '%s', '%s')",
+                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString, randomString));
 
-                QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
-                                                             BigFormat.latestVersion, valPk, valCk + longString));
+                    QueryProcessor.executeInternal(String.format("UPDATE legacy_tables.legacy_%s_clust_counter%s SET val = val + 1 WHERE pk = '%s' AND ck='%s'",
+                                                                 BigFormat.latestVersion, getCompactNameSuffix(compact), valPk, valCk + longString));
 
+                }
             }
         }
 
@@ -398,10 +388,13 @@ public class LegacySSTableTest
 
         File ksDir = new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables", BigFormat.latestVersion));
         ksDir.mkdirs();
-        copySstablesFromTestData(String.format("legacy_%s_simple", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_simple_counter", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_clust", BigFormat.latestVersion), ksDir);
-        copySstablesFromTestData(String.format("legacy_%s_clust_counter", BigFormat.latestVersion), ksDir);
+        for (int compact = 0; compact <= 1; compact++)
+        {
+            copySstablesFromTestData(String.format("legacy_%s_simple%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_simple_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_clust%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+            copySstablesFromTestData(String.format("legacy_%s_clust_counter%s", BigFormat.latestVersion, getCompactNameSuffix(compact)), ksDir);
+        }
     }
 
     private void copySstablesFromTestData(String table, File ksDir) throws IOException
@@ -420,12 +413,17 @@ public class LegacySSTableTest
 
     private static void copySstablesToTestData(String legacyVersion, String table, File cfDir) throws IOException
     {
-        for (File file : new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table)).listFiles())
+        for (File file : getTableDir(legacyVersion, table).listFiles())
         {
             copyFile(cfDir, file);
         }
     }
 
+    private static File getTableDir(String legacyVersion, String table)
+    {
+        return new File(LEGACY_SSTABLE_ROOT, String.format("%s/legacy_tables/%s", legacyVersion, table));
+    }
+
     private static void copyFile(File cfDir, File file) throws IOException
     {
         byte[] buf = new byte[65536];

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
index c3a4539..360d262 100644
--- a/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
+++ b/test/unit/org/apache/cassandra/io/util/BufferedRandomAccessFileTest.java
@@ -406,7 +406,7 @@ public class BufferedRandomAccessFileTest
             assert tmpFile.getPath().equals(r.getPath());
 
             // Create a mark and move the rw there.
-            final FileMark mark = r.mark();
+            final DataPosition mark = r.mark();
             r.reset(mark);
 
             // Expect this call to succeed.
@@ -457,7 +457,7 @@ public class BufferedRandomAccessFileTest
         RandomAccessReader file = RandomAccessReader.open(channel);
 
         file.seek(10);
-        FileMark mark = file.mark();
+        DataPosition mark = file.mark();
 
         file.seek(file.length());
         assertTrue(file.isEOF());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
index f0d4383..e83c015 100644
--- a/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/util/RandomAccessReaderTest.java
@@ -356,7 +356,7 @@ public class RandomAccessReaderTest
             assertFalse(reader.isEOF());
             assertEquals((numIterations - 1) * expected.length(), reader.bytesRemaining());
 
-            FileMark mark = reader.mark();
+            DataPosition mark = reader.mark();
             assertEquals(0, reader.bytesPastMark());
             assertEquals(0, reader.bytesPastMark(mark));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
new file mode 100644
index 0000000..175ab53
--- /dev/null
+++ b/test/unit/org/apache/cassandra/io/util/RewindableDataInputStreamPlusTest.java
@@ -0,0 +1,539 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.io.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class RewindableDataInputStreamPlusTest
+{
+
+    private final int INITIAL_BUFFER_SIZE = 1;
+
+    private File file;
+
+    @Before
+    public void setup() throws Exception
+    {
+        this.file = new File(System.getProperty("java.io.tmpdir"), "subdir/test.buffer");
+    }
+
+    @Test
+    public void testMarkAndResetSimple() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+            // long
+            out.writeLong(1L);
+            // float
+            out.writeFloat(1.0f);
+            // double
+            out.writeDouble(1.0d);
+
+            // String
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+        }
+
+        for (int memCapacity = 0; memCapacity <= 16; memCapacity++)
+        {
+            int diskCapacity = 16 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                try {
+                    //should mark before resetting
+                    reader.reset(null);
+                    fail("Should have thrown IOException");
+                } catch (IOException e) {}
+
+                assertTrue(reader.readBoolean());
+
+                reader.mark();
+
+                try {
+                    //cannot mark already marked stream
+                    reader.mark();
+                    fail("Should have thrown IllegalStateException");
+                } catch (IllegalStateException e) {}
+
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(3, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                try {
+                    //cannot mark when reading from cache
+                    reader.mark();
+                    fail("Should have thrown IllegalStateException");
+                } catch (IllegalStateException e) {}
+
+                //read again previous sequence
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                //finish reading again previous sequence
+                assertEquals(1, reader.readShort());
+
+                reader.mark();
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(16, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                //read again previous sequence
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                assertEquals(1.0f, reader.readFloat(), 0);
+                //finish reading again previous sequence
+
+                //mark again
+                reader.mark();
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals(8, reader.bytesPastMark(null));
+                reader.reset(null);
+
+                //read again previous sequence
+                assertEquals(1.0d, reader.readDouble(), 0);
+                //finish reading again previous sequence
+
+                //mark and reset
+                reader.mark();
+                reader.reset(null);
+
+                assertEquals("abc", reader.readUTF());
+
+                //check max file size
+                assertEquals(diskCapacity, file.length());
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+    @Test
+    public void testVeryLargeCapacity() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+        }
+
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, Integer.MAX_VALUE, file,
+                                                                                      Integer.MAX_VALUE))
+        {
+            reader.mark();
+            assertEquals("abc", reader.readUTF());
+            reader.reset();
+            assertEquals("abc", reader.readUTF());
+        }
+        assertFalse(file.exists());
+
+
+        baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            out.writeBoolean(true);
+            out.writeBoolean(true);
+            testData = baos.toByteArray();
+        }
+    }
+
+    @Test
+    public void testMarkAndResetBigBuffer() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+            // long
+            out.writeLong(1L);
+            // float
+            out.writeFloat(1.0f);
+            // double
+            out.writeDouble(1.0d);
+
+            // String
+            out.writeUTF("abc");
+            testData = baos.toByteArray();
+
+            // 1 (boolean) + 1 (byte) + 2 (char) + 2 (short) + 4 (int) + 8 (long)
+            // + 4 (float) + 8 (double) + 5 bytes (utf string)
+        }
+
+        for (int memCapacity = 0; memCapacity <= 18; memCapacity++)
+        {
+            int diskCapacity = 18 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                //read a big amount before resetting
+                reader.mark();
+                assertTrue(reader.readBoolean());
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(1, reader.readShort());
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+                reader.reset();
+
+                //read from buffer
+                assertTrue(reader.readBoolean());
+                assertEquals(0x1, reader.readByte());
+                assertEquals('a', reader.readChar());
+                assertEquals(1, reader.readShort());
+                assertEquals(1, reader.readInt());
+                assertEquals(1L, reader.readLong());
+
+                assertEquals(17, reader.available());
+
+                //mark again
+                reader.mark();
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals("abc", reader.readUTF());
+                reader.reset();
+
+                assertEquals(17, reader.available());
+
+                assertEquals(1.0f, reader.readFloat(), 0);
+                assertEquals(1.0d, reader.readDouble(), 0);
+                assertEquals("abc", reader.readUTF());
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+
+    @Test
+    public void testCircularSpillFile() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            // int
+            out.writeInt(1);
+
+            // String
+            out.writeUTF("ab");
+            testData = baos.toByteArray();
+
+            // 1 (boolean) + 1 (byte) + 2 (char) + 2 (short) + 4 (int) + 4 bytes (utf string)
+        }
+
+        //read at most 4 bytes multiple times (and then check file size)
+        int MEM_SIZE = 0;
+        int DISK_SIZE = 4;
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, MEM_SIZE, file,
+                                                                                      DISK_SIZE))
+        {
+            //read 2 bytes and reset
+            reader.mark();
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            assertEquals(2, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            //finish reading again previous sequence
+
+            //read 4 bytes and reset
+            reader.mark();
+            assertEquals('a', reader.readChar());
+            assertEquals(1, reader.readShort());
+            assertEquals(4, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertEquals('a', reader.readChar());
+            assertEquals(1, reader.readShort());
+            //finish reading again previous sequence
+
+            //read 4 bytes and reset
+            reader.mark();
+            assertEquals(1, reader.readInt());
+            assertEquals(4, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertEquals(1, reader.readInt());
+
+            //check max file size
+            assertEquals(DISK_SIZE, file.length());
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testExhaustCapacity() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // boolean
+            out.writeBoolean(true);
+            // byte
+            out.writeByte(0x1);
+            // char
+            out.writeChar('a');
+            // short
+            out.writeShort(1);
+            testData = baos.toByteArray();
+        }
+
+        //test capacity exhausted when reading more than 4 bytes
+        testCapacityExhausted(testData, 0, 2);
+        testCapacityExhausted(testData, 2, 0);
+        testCapacityExhausted(testData, 1, 1);
+    }
+
+    private void testCapacityExhausted(byte[] testData, int memSize, int diskSize) throws IOException
+    {
+        try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                      INITIAL_BUFFER_SIZE, memSize, file,
+                                                                                      diskSize))
+        {
+            //read 2 bytes and reset
+            reader.mark();
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            assertEquals(2, reader.bytesPastMark(null));
+            reader.reset();
+
+            //read again previous sequence
+            assertTrue(reader.readBoolean());
+            assertEquals(0x1, reader.readByte());
+            //finish reading again previous sequence
+
+            reader.mark();
+            //read 3 bytes - OK
+            assertEquals('a', reader.readChar());
+            //read 1 more bytes - CAPACITY will exhaust when trying to reset :(
+            assertEquals(1, reader.readShort());
+
+            try
+            {
+                reader.reset();
+                fail("Should have thrown IOException");
+            }
+            catch (IOException e) {}
+
+            //check max file size
+            assertEquals(diskSize, file.length());
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testMarkAndResetUnsignedRead() throws Exception
+    {
+        byte[] testData;
+
+        ByteArrayOutputStream baos = new ByteArrayOutputStream();
+        try (DataOutputStream out = new DataOutputStream(baos))
+        {
+            // byte
+            out.writeByte(0x1);
+            // short
+            out.writeShort(2);
+            testData = baos.toByteArray();
+        }
+
+        for (int memCapacity = 0; memCapacity <= 1; memCapacity++)
+        {
+            int diskCapacity = 1 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                assertEquals(1, reader.readUnsignedByte());
+                reader.reset();
+                assertEquals(1, reader.readUnsignedByte());
+
+                //will read first byte of short 2
+                reader.mark();
+                assertEquals(0, reader.readUnsignedByte());
+                reader.reset();
+
+                assertEquals(2, reader.readUnsignedShort());
+
+                reader.mark();
+                reader.reset();
+                assertEquals(0, reader.available());
+            }
+        }
+        assertFalse(file.exists());
+    }
+
+    @Test
+    public void testMarkAndResetSkipBytes() throws Exception
+    {
+        String testStr = "1234567890";
+        byte[] testData = testStr.getBytes();
+
+        for (int memCapacity = 0; memCapacity <= 7; memCapacity++)
+        {
+            int diskCapacity = 7 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                // read first 5 bytes and rewind
+                byte[] out = new byte[5];
+                reader.readFully(out, 0, 5);
+                assertEquals("12345", new String(out));
+
+                // then skip 2 bytes (67)
+                reader.skipBytes(2);
+
+                assertEquals(7, reader.bytesPastMark(null));
+                reader.reset();
+
+                //now read part of the previously skipped bytes
+                out = new byte[5];
+                reader.readFully(out);
+                assertEquals("12345", new String(out));
+
+                //skip 3 bytes (2 from cache, 1 from stream)
+                reader.skip(3);
+
+                // mark and read 2 more bytes
+                reader.mark();
+                out = new byte[2];
+                reader.readFully(out);
+                assertEquals("90", new String(out));
+                assertEquals(0, reader.available());
+                reader.reset();
+
+                //reset and read only the next byte "9" in the third position
+                reader.readFully(out, 1, 1);
+                assertEquals("99", new String(out));
+
+                //now we read the remainder via readline
+                assertEquals(1, reader.available());
+                assertEquals("0", reader.readLine());
+
+            }
+            assertFalse(file.exists());
+        }
+    }
+
+    @Test
+    public void testMarkAndResetReadFully() throws Exception
+    {
+        String testStr = "1234567890";
+        byte[] testData = testStr.getBytes();
+
+        for (int memCapacity = 0; memCapacity <= 5; memCapacity++)
+        {
+            int diskCapacity = 5 - memCapacity;
+            try (RewindableDataInputStreamPlus reader = new RewindableDataInputStreamPlus(new ByteArrayInputStream(testData),
+                                                                                          INITIAL_BUFFER_SIZE, memCapacity, file,
+                                                                                          diskCapacity))
+            {
+                reader.mark();
+                // read first 5 bytes and rewind
+                byte[] out = new byte[5];
+                reader.readFully(out, 0, 5);
+                assertEquals("12345", new String(out));
+                reader.reset();
+
+                // read half from cache, half from parent stream
+                out = new byte[7];
+                reader.readFully(out);
+                assertEquals("1234567", new String(out));
+
+                // mark and read 3 more bytes
+                reader.mark();
+                out = new byte[3];
+                reader.readFully(out);
+                assertEquals("890", new String(out));
+                assertEquals(0, reader.available());
+                reader.reset();
+
+                //reset and read only the next byte "8" in the third position
+                reader.readFully(out, 2, 1);
+                assertEquals("898", new String(out));
+
+                //now we read the remainder via readline
+                assertEquals(2, reader.available());
+                assertEquals("90", reader.readLine());
+            }
+            assertFalse(file.exists());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e8651b66/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java b/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
index 221e55c..7693b45 100644
--- a/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
+++ b/test/unit/org/apache/cassandra/utils/BytesReadTrackerTest.java
@@ -18,6 +18,10 @@
  */
 package org.apache.cassandra.utils;
 
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
@@ -25,8 +29,10 @@ import java.io.DataOutputStream;
 
 import org.junit.Test;
 
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
+import org.apache.cassandra.io.util.BytesReadTracker;
+import org.apache.cassandra.io.util.DataInputPlus;
+import org.apache.cassandra.io.util.TrackedDataInputPlus;
+import org.apache.cassandra.io.util.TrackedInputStream;
 
 public class BytesReadTrackerTest
 {
@@ -34,6 +40,33 @@ public class BytesReadTrackerTest
     @Test
     public void testBytesRead() throws Exception
     {
+        internalTestBytesRead(true);
+        internalTestBytesRead(false);
+    }
+
+    @Test
+    public void testUnsignedRead() throws Exception
+    {
+        internalTestUnsignedRead(true);
+        internalTestUnsignedRead(false);
+    }
+
+    @Test
+    public void testSkipBytesAndReadFully() throws Exception
+    {
+        internalTestSkipBytesAndReadFully(true);
+        internalTestSkipBytesAndReadFully(false);
+    }
+
+    @Test
+    public void testReadLine() throws Exception
+    {
+        internalTestReadLine(true);
+        internalTestReadLine(false);
+    }
+
+    public void internalTestBytesRead(boolean inputStream) throws Exception
+    {
         byte[] testData;
 
         ByteArrayOutputStream baos = new ByteArrayOutputStream();
@@ -66,45 +99,46 @@ public class BytesReadTrackerTest
             out.close();
         }
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // boolean = 1byte
-            boolean bool = tracker.readBoolean();
+            boolean bool = reader.readBoolean();
             assertTrue(bool);
             assertEquals(1, tracker.getBytesRead());
             // byte = 1byte
-            byte b = tracker.readByte();
+            byte b = reader.readByte();
             assertEquals(b, 0x1);
             assertEquals(2, tracker.getBytesRead());
             // char = 2byte
-            char c = tracker.readChar();
+            char c = reader.readChar();
             assertEquals('a', c);
             assertEquals(4, tracker.getBytesRead());
             // short = 2bytes
-            short s = tracker.readShort();
+            short s = reader.readShort();
             assertEquals(1, s);
             assertEquals((short) 6, tracker.getBytesRead());
             // int = 4bytes
-            int i = tracker.readInt();
+            int i = reader.readInt();
             assertEquals(1, i);
             assertEquals(10, tracker.getBytesRead());
             // long = 8bytes
-            long l = tracker.readLong();
+            long l = reader.readLong();
             assertEquals(1L, l);
             assertEquals(18, tracker.getBytesRead());
             // float = 4bytes
-            float f = tracker.readFloat();
+            float f = reader.readFloat();
             assertEquals(1.0f, f, 0);
             assertEquals(22, tracker.getBytesRead());
             // double = 8bytes
-            double d = tracker.readDouble();
+            double d = reader.readDouble();
             assertEquals(1.0d, d, 0);
             assertEquals(30, tracker.getBytesRead());
             // String("abc") = 2(string size) + 3 = 5 bytes
-            String str = tracker.readUTF();
+            String str = reader.readUTF();
             assertEquals("abc", str);
             assertEquals(35, tracker.getBytesRead());
 
@@ -119,8 +153,7 @@ public class BytesReadTrackerTest
         assertEquals(0, tracker.getBytesRead());
     }
 
-    @Test
-    public void testUnsignedRead() throws Exception
+    public void internalTestUnsignedRead(boolean inputStream) throws Exception
     {
         byte[] testData;
 
@@ -139,17 +172,18 @@ public class BytesReadTrackerTest
             out.close();
         }
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // byte = 1byte
-            int b = tracker.readUnsignedByte();
+            int b = reader.readUnsignedByte();
             assertEquals(b, 1);
             assertEquals(1, tracker.getBytesRead());
             // short = 2bytes
-            int s = tracker.readUnsignedShort();
+            int s = reader.readUnsignedShort();
             assertEquals(1, s);
             assertEquals(3, tracker.getBytesRead());
 
@@ -161,30 +195,30 @@ public class BytesReadTrackerTest
         }
     }
 
-    @Test
-    public void testSkipBytesAndReadFully() throws Exception
+    public void internalTestSkipBytesAndReadFully(boolean inputStream) throws Exception
     {
         String testStr = "1234567890";
         byte[] testData = testStr.getBytes();
 
-        DataInputStream in = new DataInputStream(new ByteArrayInputStream(testData));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        DataInputPlus.DataInputStreamPlus in = new DataInputPlus.DataInputStreamPlus(new ByteArrayInputStream(testData));
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
             // read first 5 bytes
             byte[] out = new byte[5];
-            tracker.readFully(out, 0, 5);
+            reader.readFully(out, 0, 5);
             assertEquals("12345", new String(out));
             assertEquals(5, tracker.getBytesRead());
 
             // then skip 2 bytes
-            tracker.skipBytes(2);
+            reader.skipBytes(2);
             assertEquals(7, tracker.getBytesRead());
 
             // and read the rest
             out = new byte[3];
-            tracker.readFully(out);
+            reader.readFully(out);
             assertEquals("890", new String(out));
             assertEquals(10, tracker.getBytesRead());
 
@@ -196,16 +230,24 @@ public class BytesReadTrackerTest
         }
     }
 
-    @Test(expected = UnsupportedOperationException.class)
-    public void testReadLine() throws Exception
+    public void internalTestReadLine(boolean inputStream) throws Exception
     {
         DataInputStream in = new DataInputStream(new ByteArrayInputStream("1".getBytes()));
-        BytesReadTracker tracker = new BytesReadTracker(in);
+        BytesReadTracker tracker = inputStream? new TrackedInputStream(in) : new TrackedDataInputPlus(in);
+        DataInputPlus reader = inputStream? new DataInputPlus.DataInputStreamPlus((TrackedInputStream)tracker) : (DataInputPlus) tracker;
 
         try
         {
-            // throws UnsupportedOperationException
-            tracker.readLine();
+            String line = reader.readLine();
+            if (inputStream)
+                assertEquals(line, "1");
+            else
+                fail("Should have thrown UnsupportedOperationException");
+        }
+        catch (UnsupportedOperationException e)
+        {
+            if (inputStream)
+                fail("Should have not thrown UnsupportedOperationException");
         }
         finally
         {


[11/12] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.5

Posted by yu...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.5


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/587773fa
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/587773fa
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/587773fa

Branch: refs/heads/cassandra-3.5
Commit: 587773fa478ff64aa46cf17760eb31d6f83fc46d
Parents: e3716ee e8651b6
Author: Yuki Morishita <yu...@apache.org>
Authored: Thu Mar 17 10:42:20 2016 -0500
Committer: Yuki Morishita <yu...@apache.org>
Committed: Thu Mar 17 10:42:20 2016 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   2 +
 .../org/apache/cassandra/db/Directories.java    |  30 +
 .../cassandra/db/SerializationHeader.java       |   5 +
 .../org/apache/cassandra/db/Serializers.java    | 114 ++--
 .../columniterator/AbstractSSTableIterator.java |   4 +-
 .../EncryptedFileSegmentInputStream.java        |   4 +-
 .../cassandra/hints/ChecksummedDataInput.java   |   8 +-
 .../org/apache/cassandra/hints/HintMessage.java |   4 +-
 .../io/compress/CompressedSequentialWriter.java |   8 +-
 .../io/sstable/SSTableSimpleIterator.java       |  11 +-
 .../sstable/format/RangeAwareSSTableWriter.java |   8 +-
 .../io/sstable/format/SSTableReader.java        |   2 +-
 .../io/sstable/format/SSTableWriter.java        |   2 +-
 .../io/sstable/format/big/BigTableWriter.java   |   4 +-
 .../cassandra/io/util/BytesReadTracker.java     |  30 +
 .../apache/cassandra/io/util/DataPosition.java  |  21 +
 .../apache/cassandra/io/util/FileDataInput.java |   8 +-
 .../org/apache/cassandra/io/util/FileMark.java  |  20 -
 .../io/util/FileSegmentInputStream.java         |  12 +-
 .../cassandra/io/util/RandomAccessReader.java   |   8 +-
 .../cassandra/io/util/RewindableDataInput.java  |  30 +
 .../io/util/RewindableDataInputStreamPlus.java  | 569 +++++++++++++++++++
 .../cassandra/io/util/SequentialWriter.java     |   6 +-
 .../cassandra/io/util/TrackedDataInputPlus.java | 150 +++++
 .../cassandra/io/util/TrackedInputStream.java   |  76 +++
 .../cassandra/service/StorageService.java       |   1 +
 .../cassandra/streaming/StreamReader.java       |  85 ++-
 .../compress/CompressedStreamReader.java        |  18 +-
 .../streaming/messages/FileMessageHeader.java   |   4 +-
 .../apache/cassandra/tools/nodetool/Repair.java |   2 +-
 .../cassandra/utils/BytesReadTracker.java       | 153 -----
 .../cassandra/utils/CloseableIterator.java      |   1 -
 ...acy_jb_clust_compact-jb-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-Data.db | Bin 0 -> 5270 bytes
 ...ables-legacy_jb_clust_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_jb_clust_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_jb_clust_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...bles-legacy_jb_clust_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ..._tables-legacy_jb_clust_compact-jb-1-TOC.txt |   7 +
 ...lust_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_jb_clust_counter_compact-jb-1-Data.db | Bin 0 -> 4228 bytes
 ...gacy_jb_clust_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_jb_clust_counter_compact-jb-1-Index.db | Bin 0 -> 157685 bytes
 ..._jb_clust_counter_compact-jb-1-Statistics.db | Bin 0 -> 6791 bytes
 ...acy_jb_clust_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...legacy_jb_clust_counter_compact-jb-1-TOC.txt |   7 +
 ...cy_jb_simple_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_jb_simple_compact-jb-1-Data.db | Bin 0 -> 108 bytes
 ...bles-legacy_jb_simple_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_jb_simple_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_jb_simple_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...les-legacy_jb_simple_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...tables-legacy_jb_simple_compact-jb-1-TOC.txt |   7 +
 ...mple_counter_compact-jb-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_jb_simple_counter_compact-jb-1-Data.db | Bin 0 -> 118 bytes
 ...acy_jb_simple_counter_compact-jb-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_jb_simple_counter_compact-jb-1-Index.db | Bin 0 -> 75 bytes
 ...jb_simple_counter_compact-jb-1-Statistics.db | Bin 0 -> 4395 bytes
 ...cy_jb_simple_counter_compact-jb-1-Summary.db | Bin 0 -> 71 bytes
 ...egacy_jb_simple_counter_compact-jb-1-TOC.txt |   7 +
 ...acy_ka_clust_compact-ka-1-CompressionInfo.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-Data.db | Bin 0 -> 5277 bytes
 ...les-legacy_ka_clust_compact-ka-1-Digest.sha1 |   1 +
 ...ables-legacy_ka_clust_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...tables-legacy_ka_clust_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ...s-legacy_ka_clust_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...bles-legacy_ka_clust_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ..._tables-legacy_ka_clust_compact-ka-1-TOC.txt |   8 +
 ...lust_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 75 bytes
 ...legacy_ka_clust_counter_compact-ka-1-Data.db | Bin 0 -> 4527 bytes
 ...cy_ka_clust_counter_compact-ka-1-Digest.sha1 |   1 +
 ...gacy_ka_clust_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...egacy_ka_clust_counter_compact-ka-1-Index.db | Bin 0 -> 157685 bytes
 ..._ka_clust_counter_compact-ka-1-Statistics.db | Bin 0 -> 6859 bytes
 ...acy_ka_clust_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...legacy_ka_clust_counter_compact-ka-1-TOC.txt |   8 +
 ...cy_ka_simple_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...tables-legacy_ka_simple_compact-ka-1-Data.db | Bin 0 -> 105 bytes
 ...es-legacy_ka_simple_compact-ka-1-Digest.sha1 |   1 +
 ...bles-legacy_ka_simple_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...ables-legacy_ka_simple_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...-legacy_ka_simple_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...les-legacy_ka_simple_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...tables-legacy_ka_simple_compact-ka-1-TOC.txt |   8 +
 ...mple_counter_compact-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 ...egacy_ka_simple_counter_compact-ka-1-Data.db | Bin 0 -> 124 bytes
 ...y_ka_simple_counter_compact-ka-1-Digest.sha1 |   1 +
 ...acy_ka_simple_counter_compact-ka-1-Filter.db | Bin 0 -> 24 bytes
 ...gacy_ka_simple_counter_compact-ka-1-Index.db | Bin 0 -> 75 bytes
 ...ka_simple_counter_compact-ka-1-Statistics.db | Bin 0 -> 4453 bytes
 ...cy_ka_simple_counter_compact-ka-1-Summary.db | Bin 0 -> 83 bytes
 ...egacy_ka_simple_counter_compact-ka-1-TOC.txt |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_la_clust_compact/la-1-big-Data.db    | Bin 0 -> 5286 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_clust_compact/la-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_la_clust_compact/la-1-big-Index.db   | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../legacy_la_clust_compact/la-1-big-Summary.db | Bin 0 -> 75 bytes
 .../legacy_la_clust_compact/la-1-big-TOC.txt    |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 4527 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 157685 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 6859 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_la_simple_compact/la-1-big-Data.db   | Bin 0 -> 106 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../legacy_la_simple_compact/la-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_la_simple_compact/la-1-big-Index.db  | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../legacy_la_simple_compact/la-1-big-TOC.txt   |   8 +
 .../la-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../la-1-big-Data.db                            | Bin 0 -> 124 bytes
 .../la-1-big-Digest.adler32                     |   1 +
 .../la-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../la-1-big-Index.db                           | Bin 0 -> 75 bytes
 .../la-1-big-Statistics.db                      | Bin 0 -> 4453 bytes
 .../la-1-big-Summary.db                         | Bin 0 -> 75 bytes
 .../la-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 83 bytes
 .../legacy_ma_clust_compact/ma-1-big-Data.db    | Bin 0 -> 5393 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_clust_compact/ma-1-big-Filter.db  | Bin 0 -> 24 bytes
 .../legacy_ma_clust_compact/ma-1-big-Index.db   | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7046 bytes
 .../legacy_ma_clust_compact/ma-1-big-Summary.db | Bin 0 -> 47 bytes
 .../legacy_ma_clust_compact/ma-1-big-TOC.txt    |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 75 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 4606 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 157553 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 7055 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../legacy_ma_simple_compact/ma-1-big-Data.db   | Bin 0 -> 91 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../legacy_ma_simple_compact/ma-1-big-Filter.db | Bin 0 -> 24 bytes
 .../legacy_ma_simple_compact/ma-1-big-Index.db  | Bin 0 -> 26 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4640 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../legacy_ma_simple_compact/ma-1-big-TOC.txt   |   8 +
 .../ma-1-big-CompressionInfo.db                 | Bin 0 -> 43 bytes
 .../ma-1-big-Data.db                            | Bin 0 -> 114 bytes
 .../ma-1-big-Digest.crc32                       |   1 +
 .../ma-1-big-Filter.db                          | Bin 0 -> 24 bytes
 .../ma-1-big-Index.db                           | Bin 0 -> 27 bytes
 .../ma-1-big-Statistics.db                      | Bin 0 -> 4649 bytes
 .../ma-1-big-Summary.db                         | Bin 0 -> 47 bytes
 .../ma-1-big-TOC.txt                            |   8 +
 .../cassandra/AbstractSerializationsTester.java |   1 -
 .../apache/cassandra/db/DirectoriesTest.java    |  98 ++--
 .../cassandra/gms/SerializationsTest.java       |   1 -
 .../CompressedRandomAccessReaderTest.java       |   6 +-
 .../CompressedSequentialWriterTest.java         |   4 +-
 .../cassandra/io/sstable/LegacySSTableTest.java | 369 ++++++------
 .../io/util/BufferedRandomAccessFileTest.java   |   4 +-
 .../io/util/RandomAccessReaderTest.java         |   2 +-
 .../util/RewindableDataInputStreamPlusTest.java | 539 ++++++++++++++++++
 .../cassandra/utils/BytesReadTrackerTest.java   | 104 +++-
 167 files changed, 2115 insertions(+), 550 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a01e511,51cfc16..53dd292
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,6 -1,6 +1,7 @@@
 -3.0.5
 +3.5
 +Merged from 3.0:
+  * Support streaming pre-3.0 sstables (CASSANDRA-10990)
 - * Add backpressure to compressed commit log (CASSANDRA-10971)
 + * Add backpressure to compressed or encrypted commit log (CASSANDRA-10971)
   * SSTableExport supports secondary index tables (CASSANDRA-11330)
   * Fix sstabledump to include missing info in debug output (CASSANDRA-11321)
   * Establish and implement canonical bulk reading workload(s) (CASSANDRA-10331)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/Directories.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/Serializers.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/Serializers.java
index 17f1de0,348fda3..cef06a3
--- a/src/java/org/apache/cassandra/db/Serializers.java
+++ b/src/java/org/apache/cassandra/db/Serializers.java
@@@ -46,62 -46,77 +46,77 @@@ public class Serializer
      // unecessary (since IndexInfo.Serializer won't depend on the metadata either).
      public ISerializer<ClusteringPrefix> indexEntryClusteringPrefixSerializer(final Version version, final SerializationHeader header)
      {
-         if (!version.storeRows())
+         if (!version.storeRows() || header ==  null) //null header indicates streaming from pre-3.0 sstables
          {
-             return new ISerializer<ClusteringPrefix>()
+             return oldFormatSerializer(version);
+         }
+ 
+         return newFormatSerializer(version, header);
+     }
+ 
+     private ISerializer<ClusteringPrefix> oldFormatSerializer(final Version version)
+     {
+         return new ISerializer<ClusteringPrefix>()
+         {
+             SerializationHeader newHeader = SerializationHeader.makeWithoutStats(metadata);
+ 
+             public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
              {
-                 public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
-                 {
-                     // We should only use this for reading old sstable, never write new ones.
-                     throw new UnsupportedOperationException();
-                 }
+                 //we deserialize in the old format and serialize in the new format
+                 ClusteringPrefix.serializer.serialize(clustering, out,
+                                                       version.correspondingMessagingVersion(),
+                                                       newHeader.clusteringTypes());
+             }
+ 
+             public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+             {
+                 // We're reading the old cellname/composite
+                 ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
+                 assert bb.hasRemaining(); // empty cellnames were invalid
+ 
+                 int clusteringSize = metadata.clusteringColumns().size();
+                 // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
+                 if (clusteringSize == 0)
+                     return Clustering.EMPTY;
+ 
+                 if (!metadata.isCompound())
 -                    return new Clustering(bb);
++                    return Clustering.make(bb);
  
-                 public ClusteringPrefix deserialize(DataInputPlus in) throws IOException
+                 List<ByteBuffer> components = CompositeType.splitName(bb);
+                 byte eoc = CompositeType.lastEOC(bb);
+ 
+                 if (eoc == 0 || components.size() >= clusteringSize)
                  {
-                     // We're reading the old cellname/composite
-                     ByteBuffer bb = ByteBufferUtil.readWithShortLength(in);
-                     assert bb.hasRemaining(); // empty cellnames were invalid
- 
-                     int clusteringSize = metadata.clusteringColumns().size();
-                     // If the table has no clustering column, then the cellname will just be the "column" name, which we ignore here.
-                     if (clusteringSize == 0)
-                         return Clustering.EMPTY;
- 
-                     if (!metadata.isCompound())
-                         return Clustering.make(bb);
- 
-                     List<ByteBuffer> components = CompositeType.splitName(bb);
-                     byte eoc = CompositeType.lastEOC(bb);
- 
-                     if (eoc == 0 || components.size() >= clusteringSize)
-                     {
-                         // That's a clustering.
-                         if (components.size() > clusteringSize)
-                             components = components.subList(0, clusteringSize);
- 
-                         return Clustering.make(components.toArray(new ByteBuffer[clusteringSize]));
-                     }
-                     else
-                     {
-                         // It's a range tombstone bound. It is a start since that's the only part we've ever included
-                         // in the index entries.
-                         Slice.Bound.Kind boundKind = eoc > 0
-                                                    ? Slice.Bound.Kind.EXCL_START_BOUND
-                                                    : Slice.Bound.Kind.INCL_START_BOUND;
- 
-                         return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
-                     }
-                 }
+                     // That's a clustering.
+                     if (components.size() > clusteringSize)
+                         components = components.subList(0, clusteringSize);
  
-                 public long serializedSize(ClusteringPrefix clustering)
 -                    return new Clustering(components.toArray(new ByteBuffer[clusteringSize]));
++                    return Clustering.make(components.toArray(new ByteBuffer[clusteringSize]));
+                 }
+                 else
                  {
-                     // We should only use this for reading old sstable, never write new ones.
-                     throw new UnsupportedOperationException();
+                     // It's a range tombstone bound. It is a start since that's the only part we've ever included
+                     // in the index entries.
+                     Slice.Bound.Kind boundKind = eoc > 0
+                                                  ? Slice.Bound.Kind.EXCL_START_BOUND
+                                                  : Slice.Bound.Kind.INCL_START_BOUND;
+ 
+                     return Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
                  }
-             };
-         }
+             }
  
-         return new ISerializer<ClusteringPrefix>()
+             public long serializedSize(ClusteringPrefix clustering)
+             {
+                 return ClusteringPrefix.serializer.serializedSize(clustering, version.correspondingMessagingVersion(),
+                                                                   newHeader.clusteringTypes());
+             }
+         };
+     }
+ 
+ 
+     private ISerializer<ClusteringPrefix> newFormatSerializer(final Version version, final SerializationHeader header)
+     {
+         return new ISerializer<ClusteringPrefix>() //Reading and writing from/to the new sstable format
          {
              public void serialize(ClusteringPrefix clustering, DataOutputPlus out) throws IOException
              {
@@@ -119,4 -134,5 +134,5 @@@
              }
          };
      }
- }
+ 
 -}
++}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
index d55161b,0e2012e..7f2e3bb
--- a/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
+++ b/src/java/org/apache/cassandra/db/columniterator/AbstractSSTableIterator.java
@@@ -29,10 -30,10 +29,10 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.cassandra.io.sstable.CorruptSSTableException;
  import org.apache.cassandra.io.sstable.IndexHelper;
  import org.apache.cassandra.io.util.FileDataInput;
- import org.apache.cassandra.io.util.FileMark;
+ import org.apache.cassandra.io.util.DataPosition;
  import org.apache.cassandra.utils.ByteBufferUtil;
  
 -abstract class AbstractSSTableIterator implements SliceableUnfilteredRowIterator
 +abstract class AbstractSSTableIterator implements UnfilteredRowIterator
  {
      protected final SSTableReader sstable;
      protected final DecoratedKey key;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
index 6915196,0000000..56bb7d6
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
+++ b/src/java/org/apache/cassandra/db/commitlog/EncryptedFileSegmentInputStream.java
@@@ -1,73 -1,0 +1,73 @@@
 +package org.apache.cassandra.db.commitlog;
 +
 +import java.io.DataInput;
 +import java.nio.ByteBuffer;
 +
++import org.apache.cassandra.io.util.DataPosition;
 +import org.apache.cassandra.io.util.FileDataInput;
- import org.apache.cassandra.io.util.FileMark;
 +import org.apache.cassandra.io.util.FileSegmentInputStream;
 +
 +/**
 + * Each segment of an encrypted file may contain many encrypted chunks, and each chunk needs to be individually decrypted
 + * to reconstruct the full segment.
 + */
 +public class EncryptedFileSegmentInputStream extends FileSegmentInputStream implements FileDataInput, DataInput
 +{
 +    private final long segmentOffset;
 +    private final int expectedLength;
 +    private final ChunkProvider chunkProvider;
 +
 +    /**
 +     * offset the decrypted chunks already processed in this segment.
 +     */
 +    private int totalChunkOffset;
 +
 +    public EncryptedFileSegmentInputStream(String filePath, long segmentOffset, int position, int expectedLength, ChunkProvider chunkProvider)
 +    {
 +        super(chunkProvider.nextChunk(), filePath, position);
 +        this.segmentOffset = segmentOffset;
 +        this.expectedLength = expectedLength;
 +        this.chunkProvider = chunkProvider;
 +    }
 +
 +    public interface ChunkProvider
 +    {
 +        /**
 +         * Get the next chunk from the backing provider, if any chunks remain.
 +         * @return Next chunk, else null if no more chunks remain.
 +         */
 +        ByteBuffer nextChunk();
 +    }
 +
 +    public long getFilePointer()
 +    {
 +        return segmentOffset + totalChunkOffset + buffer.position();
 +    }
 +
 +    public boolean isEOF()
 +    {
 +        return totalChunkOffset + buffer.position() >= expectedLength;
 +    }
 +
 +    public long bytesRemaining()
 +    {
 +        return expectedLength - (totalChunkOffset + buffer.position());
 +    }
 +
 +    public void seek(long position)
 +    {
 +        // implement this when we actually need it
 +        throw new UnsupportedOperationException();
 +    }
 +
-     public long bytesPastMark(FileMark mark)
++    public long bytesPastMark(DataPosition mark)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
 +    public void reBuffer()
 +    {
 +        totalChunkOffset += buffer.position();
 +        buffer = chunkProvider.nextChunk();
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/SSTableSimpleIterator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
index 674ed7f,0000000..9fcdfa4
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/RangeAwareSSTableWriter.java
@@@ -1,205 -1,0 +1,205 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *     http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing, software
 + * distributed under the License is distributed on an "AS IS" BASIS,
 + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 + * See the License for the specific language governing permissions and
 + * limitations under the License.
 + */
 +package org.apache.cassandra.io.sstable.format;
 +
 +import java.io.IOException;
 +import java.util.ArrayList;
 +import java.util.Collection;
 +import java.util.List;
 +import java.util.UUID;
 +
 +import org.apache.cassandra.db.ColumnFamilyStore;
 +import org.apache.cassandra.db.DecoratedKey;
 +import org.apache.cassandra.db.Directories;
 +import org.apache.cassandra.db.PartitionPosition;
 +import org.apache.cassandra.db.SerializationHeader;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 +import org.apache.cassandra.db.rows.UnfilteredRowIterator;
 +import org.apache.cassandra.io.sstable.Descriptor;
 +import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 +import org.apache.cassandra.service.StorageService;
 +
 +public class RangeAwareSSTableWriter implements SSTableMultiWriter
 +{
 +    private final List<PartitionPosition> boundaries;
 +    private final Directories.DataDirectory[] directories;
 +    private final int sstableLevel;
 +    private final long estimatedKeys;
 +    private final long repairedAt;
 +    private final SSTableFormat.Type format;
-     private final SerializationHeader.Component header;
++    private final SerializationHeader header;
 +    private final LifecycleTransaction txn;
 +    private int currentIndex = -1;
 +    public final ColumnFamilyStore cfs;
 +    private final List<SSTableMultiWriter> finishedWriters = new ArrayList<>();
 +    private final List<SSTableReader> finishedReaders = new ArrayList<>();
 +    private SSTableMultiWriter currentWriter = null;
 +
-     public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, SSTableFormat.Type format, int sstableLevel, long totalSize, LifecycleTransaction txn, SerializationHeader.Component header) throws IOException
++    public RangeAwareSSTableWriter(ColumnFamilyStore cfs, long estimatedKeys, long repairedAt, SSTableFormat.Type format, int sstableLevel, long totalSize, LifecycleTransaction txn, SerializationHeader header) throws IOException
 +    {
 +        directories = cfs.getDirectories().getWriteableLocations();
 +        this.sstableLevel = sstableLevel;
 +        this.cfs = cfs;
 +        this.estimatedKeys = estimatedKeys / directories.length;
 +        this.repairedAt = repairedAt;
 +        this.format = format;
 +        this.txn = txn;
 +        this.header = header;
 +        boundaries = StorageService.getDiskBoundaries(cfs, directories);
 +        if (boundaries == null)
 +        {
 +            Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize);
 +            if (localDir == null)
 +                throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
 +            Descriptor desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
-             currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), txn);
++            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header, txn);
 +        }
 +    }
 +
 +    private void maybeSwitchWriter(DecoratedKey key)
 +    {
 +        if (boundaries == null)
 +            return;
 +
 +        boolean switched = false;
 +        while (currentIndex < 0 || key.compareTo(boundaries.get(currentIndex)) > 0)
 +        {
 +            switched = true;
 +            currentIndex++;
 +        }
 +
 +        if (switched)
 +        {
 +            if (currentWriter != null)
 +                finishedWriters.add(currentWriter);
 +
 +            Descriptor desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(directories[currentIndex])), format);
-             currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header.toHeader(cfs.metadata), txn);
++            currentWriter = cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, header, txn);
 +        }
 +    }
 +
 +    public boolean append(UnfilteredRowIterator partition)
 +    {
 +        maybeSwitchWriter(partition.partitionKey());
 +        return currentWriter.append(partition);
 +    }
 +
 +    @Override
 +    public Collection<SSTableReader> finish(long repairedAt, long maxDataAge, boolean openResult)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter writer : finishedWriters)
 +        {
 +            if (writer.getFilePointer() > 0)
 +                finishedReaders.addAll(writer.finish(repairedAt, maxDataAge, openResult));
 +            else
 +                SSTableMultiWriter.abortOrDie(writer);
 +        }
 +        return finishedReaders;
 +    }
 +
 +    @Override
 +    public Collection<SSTableReader> finish(boolean openResult)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter writer : finishedWriters)
 +        {
 +            if (writer.getFilePointer() > 0)
 +                finishedReaders.addAll(writer.finish(openResult));
 +            else
 +                SSTableMultiWriter.abortOrDie(writer);
 +        }
 +        return finishedReaders;
 +    }
 +
 +    @Override
 +    public Collection<SSTableReader> finished()
 +    {
 +        return finishedReaders;
 +    }
 +
 +    @Override
 +    public SSTableMultiWriter setOpenResult(boolean openResult)
 +    {
 +        finishedWriters.forEach((w) -> w.setOpenResult(openResult));
 +        currentWriter.setOpenResult(openResult);
 +        return this;
 +    }
 +
 +    public String getFilename()
 +    {
 +        return String.join("/", cfs.keyspace.getName(), cfs.getTableName());
 +    }
 +
 +    @Override
 +    public long getFilePointer()
 +    {
 +        return currentWriter.getFilePointer();
 +    }
 +
 +    @Override
 +    public UUID getCfId()
 +    {
 +        return currentWriter.getCfId();
 +    }
 +
 +    @Override
 +    public Throwable commit(Throwable accumulate)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter writer : finishedWriters)
 +            accumulate = writer.commit(accumulate);
 +        return accumulate;
 +    }
 +
 +    @Override
 +    public Throwable abort(Throwable accumulate)
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        for (SSTableMultiWriter finishedWriter : finishedWriters)
 +            accumulate = finishedWriter.abort(accumulate);
 +
 +        return accumulate;
 +    }
 +
 +    @Override
 +    public void prepareToCommit()
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        finishedWriters.forEach(SSTableMultiWriter::prepareToCommit);
 +    }
 +
 +    @Override
 +    public void close()
 +    {
 +        if (currentWriter != null)
 +            finishedWriters.add(currentWriter);
 +        currentWriter = null;
 +        finishedWriters.forEach(SSTableMultiWriter::close);
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
index ab38ba9,5f35029..6aaf776
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableWriter.java
@@@ -83,9 -80,8 +83,9 @@@ public abstract class SSTableWriter ext
          this.keyCount = keyCount;
          this.repairedAt = repairedAt;
          this.metadataCollector = metadataCollector;
-         this.header = header;
+         this.header = header != null ? header : SerializationHeader.makeWithoutStats(metadata); //null header indicates streaming from pre-3.0 sstable
          this.rowIndexEntrySerializer = descriptor.version.getSSTableFormat().getIndexSerializer(metadata, descriptor.version, header);
 +        this.observers = observers == null ? Collections.emptySet() : observers;
      }
  
      public static SSTableWriter create(Descriptor descriptor,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/sstable/format/big/BigTableWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/util/RandomAccessReader.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/io/util/SequentialWriter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/streaming/StreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/StreamReader.java
index 7348027,f8db26b..7d7cf8a
--- a/src/java/org/apache/cassandra/streaming/StreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReader.java
@@@ -35,15 -35,18 +35,18 @@@ import org.apache.cassandra.config.CFMe
  import org.apache.cassandra.config.Schema;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.rows.*;
 -import org.apache.cassandra.io.sstable.Descriptor;
  import org.apache.cassandra.io.sstable.SSTableMultiWriter;
  import org.apache.cassandra.io.sstable.SSTableSimpleIterator;
 +import org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter;
  import org.apache.cassandra.io.sstable.format.SSTableFormat;
  import org.apache.cassandra.io.sstable.format.Version;
+ import org.apache.cassandra.io.util.RewindableDataInputStreamPlus;
  import org.apache.cassandra.io.util.DataInputPlus;
+ import org.apache.cassandra.net.MessagingService;
  import org.apache.cassandra.streaming.messages.FileMessageHeader;
  import org.apache.cassandra.utils.ByteBufferUtil;
- import org.apache.cassandra.utils.BytesReadTracker;
+ import org.apache.cassandra.io.util.TrackedInputStream;
+ import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.Pair;
  
  
@@@ -124,7 -129,7 +127,7 @@@ public class StreamReade
          {
              if (deserializer != null)
                  logger.warn("[Stream {}] Error while reading partition {} from stream on ks='{}' and table='{}'.",
--                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getColumnFamilyName());
++                            session.planId(), deserializer.partitionKey(), cfs.keyspace.getName(), cfs.getTableName());
              if (writer != null)
              {
                  writer.abort(e);
@@@ -142,10 -157,9 +155,10 @@@
          Directories.DataDirectory localDir = cfs.getDirectories().getWriteableLocation(totalSize);
          if (localDir == null)
              throw new IOException("Insufficient disk space to store " + totalSize + " bytes");
 -        desc = Descriptor.fromFilename(cfs.getSSTablePath(cfs.getDirectories().getLocationForDisk(localDir), format));
  
-         RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, format, sstableLevel, totalSize, session.getTransaction(cfId), header);
 -        return cfs.createSSTableMultiWriter(desc, estimatedKeys, repairedAt, sstableLevel, getHeader(cfs.metadata), session.getTransaction(cfId));
++        RangeAwareSSTableWriter writer = new RangeAwareSSTableWriter(cfs, estimatedKeys, repairedAt, format, sstableLevel, totalSize, session.getTransaction(cfId), getHeader(cfs.metadata));
 +        StreamHook.instance.reportIncomingFile(cfs, writer, session, fileSeqNum);
 +        return writer;
      }
  
      protected void drain(InputStream dis, long bytesRead) throws IOException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/587773fa/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
index 5a47787,9719587..318484f
--- a/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
+++ b/src/java/org/apache/cassandra/streaming/compress/CompressedStreamReader.java
@@@ -24,9 -24,7 +24,8 @@@ import java.nio.channels.ReadableByteCh
  
  import com.google.common.base.Throwables;
  
- import org.apache.cassandra.db.DecoratedKey;
  import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 +import org.apache.cassandra.io.sstable.format.RangeAwareSSTableWriter;
  
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;