You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2016/11/30 09:49:55 UTC

[02/11] cassandra git commit: Remove pre-3.0 compatibility code for 4.0

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
deleted file mode 100644
index a72d397..0000000
--- a/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
+++ /dev/null
@@ -1,179 +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.db.rows;
-
-import java.nio.ByteBuffer;
-import java.security.MessageDigest;
-
-import org.junit.Test;
-
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.CQLTester;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.db.context.CounterContext;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.CounterId;
-import org.apache.cassandra.utils.FBUtilities;
-
-import static org.junit.Assert.assertEquals;
-
-/**
- * Test that digest for pre-3.0 versions are properly computed (they match the value computed on pre-3.0 nodes).
- *
- * The concreted 'hard-coded' digests this file tests against have been generated on a 2.2 node using basically
- * the same test file but with 2 modifications:
- *   1. readAndDigest is modified to work on 2.2 (the actual modification is in the method as a comment)
- *   2. the assertions are replace by simple println() of the generated digest.
- *
- * Note that we only compare against 2.2 since digests should be fixed between version before 3.0 (this would be a bug
- * of previous version otherwise).
- */
-public class DigestBackwardCompatibilityTest extends CQLTester
-{
-    private ByteBuffer readAndDigest(String partitionKey)
-    {
-        /*
-         * In 2.2, this must be replaced by:
-         *   ColumnFamily partition = getCurrentColumnFamilyStore().getColumnFamily(QueryFilter.getIdentityFilter(Util.dk(partitionKey), currentTable(), System.currentTimeMillis()));
-         *   return ColumnFamily.digest(partition);
-         */
-
-        ReadCommand cmd = Util.cmd(getCurrentColumnFamilyStore(), partitionKey).build();
-        ImmutableBTreePartition partition = Util.getOnlyPartitionUnfiltered(cmd);
-        MessageDigest digest = FBUtilities.threadLocalMD5Digest();
-        UnfilteredRowIterators.digest(cmd, partition.unfilteredIterator(), digest, MessagingService.VERSION_22);
-        return ByteBuffer.wrap(digest.digest());
-    }
-
-    private void assertDigest(String expected, ByteBuffer actual)
-    {
-        String toTest = ByteBufferUtil.bytesToHex(actual);
-        assertEquals(String.format("[digest from 2.2] %s != %s [digest from 3.0]", expected, toTest), expected, toTest);
-    }
-
-    @Test
-    public void testCQLTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text, t int, v1 text, v2 int, PRIMARY KEY (k, t))");
-
-        String key = "someKey";
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s(k, t, v1, v2) VALUES (?, ?, ?, ?) USING TIMESTAMP ? AND TTL ?", key, i, "v" + i, i, 1L, 200);
-
-        // ColumnFamily(table_0 [0::false:0@1!200,0:v1:false:2@1!200,0:v2:false:4@1!200,1::false:0@1!200,1:v1:false:2@1!200,1:v2:false:4@1!200,2::false:0@1!200,2:v1:false:2@1!200,2:v2:false:4@1!200,3::false:0@1!200,3:v1:false:2@1!200,3:v2:false:4@1!200,4::false:0@1!200,4:v1:false:2@1!200,4:v2:false:4@1!200,5::false:0@1!200,5:v1:false:2@1!200,5:v2:false:4@1!200,6::false:0@1!200,6:v1:false:2@1!200,6:v2:false:4@1!200,7::false:0@1!200,7:v1:false:2@1!200,7:v2:false:4@1!200,8::false:0@1!200,8:v1:false:2@1!200,8:v2:false:4@1!200,9::false:0@1!200,9:v1:false:2@1!200,9:v2:false:4@1!200,])
-        assertDigest("aa608035cf6574a97061b5c166b64939", readAndDigest(key));
-
-        // This is a cell deletion
-        execute("DELETE v1 FROM %s USING TIMESTAMP ? WHERE k = ? AND t = ?", 2L, key, 2);
-
-        // This is a range tombstone
-        execute("DELETE FROM %s USING TIMESTAMP ? WHERE k = ? AND t = ?", 3L, key, 4);
-
-        // This is a partition level deletion (but we use an older tombstone so it doesn't get rid of everything and keeps the test interesting)
-        execute("DELETE FROM %s USING TIMESTAMP ? WHERE k = ?", 0L, key);
-
-        // ColumnFamily(table_0 -{deletedAt=0, localDeletion=1441012270, ranges=[4:_-4:!, deletedAt=3, localDeletion=1441012270]}- [0::false:0@1!200,0:v1:false:2@1!200,0:v2:false:4@1!200,1::false:0@1!200,1:v1:false:2@1!200,1:v2:false:4@1!200,2::false:0@1!200,2:v1:true:4@2,2:v2:false:4@1!200,3::false:0@1!200,3:v1:false:2@1!200,3:v2:false:4@1!200,5::false:0@1!200,5:v1:false:2@1!200,5:v2:false:4@1!200,6::false:0@1!200,6:v1:false:2@1!200,6:v2:false:4@1!200,7::false:0@1!200,7:v1:false:2@1!200,7:v2:false:4@1!200,8::false:0@1!200,8:v1:false:2@1!200,8:v2:false:4@1!200,9::false:0@1!200,9:v1:false:2@1!200,9:v2:false:4@1!200,])
-        assertDigest("b5f38d2dc7b917d221f98ab1641f82bf", readAndDigest(key));
-    }
-
-    @Test
-    public void testCompactTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text, t int, v text, PRIMARY KEY (k, t)) WITH COMPACT STORAGE");
-
-        String key = "someKey";
-
-        for (int i = 0; i < 10; i++)
-            execute("INSERT INTO %s(k, t, v) VALUES (?, ?, ?) USING TIMESTAMP ? AND TTL ?", key, i, "v" + i, 1L, 200);
-
-        assertDigest("44785ddd7c62c73287b448b6063645e5", readAndDigest(key));
-
-        // This is a cell deletion
-        execute("DELETE FROM %s USING TIMESTAMP ? WHERE k = ? AND t = ?", 2L, key, 2);
-
-        // This is a partition level deletion (but we use an older tombstone so it doesn't get rid of everything and keeps the test interesting)
-        execute("DELETE FROM %s USING TIMESTAMP ? WHERE k = ?", 0L, key);
-
-        assertDigest("55d9bd6335276395d83b18eb17f9abe7", readAndDigest(key));
-    }
-
-    @Test
-    public void testStaticCompactTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text PRIMARY KEY, v1 text, v2 int) WITH COMPACT STORAGE");
-
-        String key = "someKey";
-        execute("INSERT INTO %s(k, v1, v2) VALUES (?, ?, ?) USING TIMESTAMP ?", key, "v", 0, 1L);
-
-        assertDigest("d2080f9f57d6edf92da1fdaaa76573d3", readAndDigest(key));
-    }
-
-    @Test
-    public void testTableWithCollection() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k text PRIMARY KEY, m map<text, text>)");
-
-        String key = "someKey";
-
-        execute("INSERT INTO %s(k, m) VALUES (?, { 'foo' : 'value1', 'bar' : 'value2' }) USING TIMESTAMP ?", key, 1L);
-
-        // ColumnFamily(table_2 -{deletedAt=-9223372036854775808, localDeletion=2147483647, ranges=[m:_-m:!, deletedAt=0, localDeletion=1441012271]}- [:false:0@1,m:626172:false:6@1,m:666f6f:false:6@1,])
-        assertDigest("708f3fc8bc8149cc3513eef300bf0182", readAndDigest(key));
-
-        // This is a collection range tombstone
-        execute("DELETE m FROM %s USING TIMESTAMP ? WHERE k = ?", 2L, key);
-
-        // ColumnFamily(table_2 -{deletedAt=-9223372036854775808, localDeletion=2147483647, ranges=[m:_-m:!, deletedAt=2, localDeletion=1441012271]}- [:false:0@1,])
-        assertDigest("f39937fc3ed96956ef507e81717fa5cd", readAndDigest(key));
-    }
-
-    @Test
-    public void testCounterTable() throws Throwable
-    {
-        /*
-         * We can't use CQL to insert counters as both the timestamp and counter ID are automatically assigned and unpredictable.
-         * So we need to built it ourselves in a way that is totally equivalent between 2.2 and 3.0 which makes the test a little
-         * bit less readable. In any case, the code to generate the equivalent mutation on 2.2 is:
-         * ColumnFamily cf = ArrayBackedSortedColumns.factory.create(getCurrentColumnFamilyStore().metadata);
-         * ByteBuffer value = CounterContext.instance().createGlobal(CounterId.fromInt(1), 1L, 42L);
-         * cf.addColumn(new BufferCounterCell(CellNames.simpleSparse(new ColumnIdentifier("c", true)) , value, 0L, Long.MIN_VALUE));
-         * new Mutation(KEYSPACE, ByteBufferUtil.bytes(key), cf).applyUnsafe();
-         *
-         * Also note that we use COMPACT STORAGE only because it has no bearing on the test and was slightly easier in 2.2 to create
-         * the mutation.
-         */
-
-        createTable("CREATE TABLE %s (k text PRIMARY KEY, c counter) WITH COMPACT STORAGE");
-
-        String key = "someKey";
-
-        CFMetaData metadata = getCurrentColumnFamilyStore().metadata;
-        ColumnDefinition column = metadata.getColumnDefinition(ByteBufferUtil.bytes("c"));
-        ByteBuffer value = CounterContext.instance().createGlobal(CounterId.fromInt(1), 1L, 42L);
-        Row row = BTreeRow.singleCellRow(Clustering.STATIC_CLUSTERING, BufferCell.live(column, 0L, value));
-
-        new Mutation(PartitionUpdate.singleRowUpdate(metadata, Util.dk(key), row)).applyUnsafe();
-
-        assertDigest("3a5f7b48c320538b4cd2f829e05c6db3", readAndDigest(key));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java b/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
deleted file mode 100644
index 78849e3..0000000
--- a/test/unit/org/apache/cassandra/hints/LegacyHintsMigratorTest.java
+++ /dev/null
@@ -1,197 +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.hints;
-
-import java.io.File;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.file.Files;
-import java.util.*;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.config.SchemaConstants;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.marshal.UUIDType;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.db.rows.BTreeRow;
-import org.apache.cassandra.db.rows.BufferCell;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.net.MessagingService;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.UUIDGen;
-
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertTrue;
-
-import static org.apache.cassandra.hints.HintsTestUtil.assertMutationsEqual;
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
-// TODO: test split into several files
-@SuppressWarnings("deprecation")
-public class LegacyHintsMigratorTest
-{
-    private static final String KEYSPACE = "legacy_hints_migrator_test";
-    private static final String TABLE = "table";
-
-    @BeforeClass
-    public static void defineSchema()
-    {
-        SchemaLoader.prepareServer();
-        SchemaLoader.createKeyspace(KEYSPACE, KeyspaceParams.simple(1), SchemaLoader.standardCFMD(KEYSPACE, TABLE));
-    }
-
-    @Test
-    public void testNothingToMigrate() throws IOException
-    {
-        File directory = Files.createTempDirectory(null).toFile();
-        try
-        {
-            testNothingToMigrate(directory);
-        }
-        finally
-        {
-            directory.deleteOnExit();
-        }
-    }
-
-    private static void testNothingToMigrate(File directory)
-    {
-        // truncate system.hints to enseure nothing inside
-        Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.LEGACY_HINTS).truncateBlocking();
-        new LegacyHintsMigrator(directory, 128 * 1024 * 1024).migrate();
-        HintsCatalog catalog = HintsCatalog.load(directory, HintsService.EMPTY_PARAMS);
-        assertEquals(0, catalog.stores().count());
-    }
-
-    @Test
-    public void testMigrationIsComplete() throws IOException
-    {
-        File directory = Files.createTempDirectory(null).toFile();
-        try
-        {
-            testMigrationIsComplete(directory);
-        }
-        finally
-        {
-            directory.deleteOnExit();
-        }
-    }
-
-    private static void testMigrationIsComplete(File directory)
-    {
-        long timestamp = System.currentTimeMillis();
-
-        // write 100 mutations for each of the 10 generated endpoints
-        Map<UUID, Queue<Mutation>> mutations = new HashMap<>();
-        for (int i = 0; i < 10; i++)
-        {
-            UUID hostId = UUID.randomUUID();
-            Queue<Mutation> queue = new LinkedList<>();
-            mutations.put(hostId, queue);
-
-            for (int j = 0; j < 100; j++)
-            {
-                Mutation mutation = createMutation(j, timestamp + j);
-                queue.offer(mutation);
-                Mutation legacyHint = createLegacyHint(mutation, timestamp, hostId);
-                legacyHint.applyUnsafe();
-            }
-        }
-
-        // run the migration
-        new LegacyHintsMigrator(directory, 128 * 1024 * 1024).migrate();
-
-        // validate that the hints table is truncated now
-        assertTrue(Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.LEGACY_HINTS).isEmpty());
-
-        HintsCatalog catalog = HintsCatalog.load(directory, HintsService.EMPTY_PARAMS);
-
-        // assert that we've correctly loaded 10 hints stores
-        assertEquals(10, catalog.stores().count());
-
-        // for each of the 10 stores, make sure the mutations have been migrated correctly
-        for (Map.Entry<UUID, Queue<Mutation>> entry : mutations.entrySet())
-        {
-            HintsStore store = catalog.get(entry.getKey());
-            assertNotNull(store);
-
-            HintsDescriptor descriptor = store.poll();
-            assertNotNull(descriptor);
-
-            // read all the hints
-            Queue<Hint> actualHints = new LinkedList<>();
-            try (HintsReader reader = HintsReader.open(new File(directory, descriptor.fileName())))
-            {
-                for (HintsReader.Page page : reader)
-                    page.hintsIterator().forEachRemaining(actualHints::offer);
-            }
-
-            // assert the size matches
-            assertEquals(100, actualHints.size());
-
-            // compare expected hints to actual hints
-            for (int i = 0; i < 100; i++)
-            {
-                Hint hint = actualHints.poll();
-                Mutation mutation = entry.getValue().poll();
-                int ttl = mutation.smallestGCGS();
-
-                assertEquals(timestamp, hint.creationTime);
-                assertEquals(ttl, hint.gcgs);
-                assertTrue(mutation + " != " + hint.mutation, Util.sameContent(mutation, hint.mutation));
-            }
-        }
-    }
-
-    // legacy hint mutation creation code, copied more or less verbatim from the previous implementation
-    private static Mutation createLegacyHint(Mutation mutation, long now, UUID targetId)
-    {
-        int version = MessagingService.VERSION_21;
-        int ttl = mutation.smallestGCGS();
-        UUID hintId = UUIDGen.getTimeUUID();
-
-        ByteBuffer key = UUIDType.instance.decompose(targetId);
-        Clustering clustering = SystemKeyspace.LegacyHints.comparator.make(hintId, version);
-        ByteBuffer value = ByteBuffer.wrap(FBUtilities.serialize(mutation, Mutation.serializer, version));
-        Cell cell = BufferCell.expiring(SystemKeyspace.LegacyHints.compactValueColumn(),
-                                        now,
-                                        ttl,
-                                        FBUtilities.nowInSeconds(),
-                                        value);
-        return new Mutation(PartitionUpdate.singleRowUpdate(SystemKeyspace.LegacyHints,
-                                                            key,
-                                                            BTreeRow.singleCellRow(clustering, cell)));
-    }
-
-    private static Mutation createMutation(int index, long timestamp)
-    {
-        CFMetaData table = Schema.instance.getCFMetaData(KEYSPACE, TABLE);
-        return new RowUpdateBuilder(table, timestamp, bytes(index))
-               .clustering(bytes(index))
-               .add("val", bytes(index))
-               .build();
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
index f19d962..d940186 100644
--- a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
@@ -83,7 +83,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName("age");
 
         File directory = cfs.getDirectories().getDirectoryForNewSSTables();
-        Descriptor descriptor = Descriptor.fromFilename(cfs.getSSTablePath(directory));
+        Descriptor descriptor = cfs.newSSTableDescriptor(directory);
         PerSSTableIndexWriter indexWriter = (PerSSTableIndexWriter) sasi.getFlushObserver(descriptor, OperationType.FLUSH);
 
         SortedMap<DecoratedKey, Row> expectedKeys = new TreeMap<>(DecoratedKey.comparator);
@@ -175,7 +175,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
         SASIIndex sasi = (SASIIndex) cfs.indexManager.getIndexByName(columnName);
 
         File directory = cfs.getDirectories().getDirectoryForNewSSTables();
-        Descriptor descriptor = Descriptor.fromFilename(cfs.getSSTablePath(directory));
+        Descriptor descriptor = cfs.newSSTableDescriptor(directory);
         PerSSTableIndexWriter indexWriter = (PerSSTableIndexWriter) sasi.getFlushObserver(descriptor, OperationType.FLUSH);
 
         final long now = System.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/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 0d2a9fb..6359566 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedRandomAccessReaderTest.java
@@ -103,9 +103,7 @@ public class CompressedRandomAccessReaderTest
         }
 
         try (FileHandle.Builder builder = new FileHandle.Builder(filename)
-                                                              .withCompressionMetadata(new CompressionMetadata(filename + ".metadata",
-                                                                                                               f.length(),
-                                                                                                               ChecksumType.CRC32));
+                                                              .withCompressionMetadata(new CompressionMetadata(filename + ".metadata", f.length()));
              FileHandle fh = builder.complete();
              RandomAccessReader reader = fh.createReader())
         {
@@ -149,7 +147,7 @@ public class CompressedRandomAccessReaderTest
         }
         assert f.exists();
 
-        CompressionMetadata compressionMetadata = compressed ? new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32) : null;
+        CompressionMetadata compressionMetadata = compressed ? new CompressionMetadata(filename + ".metadata", f.length()) : null;
         try (FileHandle.Builder builder = new FileHandle.Builder(filename).mmapped(usemmap).withCompressionMetadata(compressionMetadata);
              FileHandle fh = builder.complete();
              RandomAccessReader reader = fh.createReader())
@@ -197,7 +195,7 @@ public class CompressedRandomAccessReaderTest
         }
 
         // open compression metadata and get chunk information
-        CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length(), ChecksumType.CRC32);
+        CompressionMetadata meta = new CompressionMetadata(metadata.getPath(), file.length());
         CompressionMetadata.Chunk chunk = meta.chunkFor(0);
 
         try (FileHandle.Builder builder = new FileHandle.Builder(file.getPath()).withCompressionMetadata(meta);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/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 a088e20..ce95d0d 100644
--- a/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/compress/CompressedSequentialWriterTest.java
@@ -121,7 +121,7 @@ public class CompressedSequentialWriterTest extends SequentialWriterTest
         }
 
         assert f.exists();
-        try (FileHandle.Builder builder = new FileHandle.Builder(filename).withCompressionMetadata(new CompressionMetadata(filename + ".metadata", f.length(), ChecksumType.CRC32));
+        try (FileHandle.Builder builder = new FileHandle.Builder(filename).withCompressionMetadata(new CompressionMetadata(filename + ".metadata", f.length()));
              FileHandle fh = builder.complete();
              RandomAccessReader reader = fh.createReader())
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
index 78964f4..4985342 100644
--- a/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/BigTableWriterTest.java
@@ -64,19 +64,19 @@ public class BigTableWriterTest extends AbstractTransactionalTest
 
         private TestableBTW()
         {
-            this(cfs.getSSTablePath(cfs.getDirectories().getDirectoryForNewSSTables()));
+            this(cfs.newSSTableDescriptor(cfs.getDirectories().getDirectoryForNewSSTables()));
         }
 
-        private TestableBTW(String file)
+        private TestableBTW(Descriptor desc)
         {
-            this(file, SSTableTxnWriter.create(cfs, file, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)));
+            this(desc, SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)));
         }
 
-        private TestableBTW(String file, SSTableTxnWriter sw)
+        private TestableBTW(Descriptor desc, SSTableTxnWriter sw)
         {
             super(sw);
-            this.file = new File(file);
-            this.descriptor = Descriptor.fromFilename(file);
+            this.file = new File(desc.filenameFor(Component.DATA));
+            this.descriptor = desc;
             this.writer = sw;
 
             for (int i = 0; i < 100; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
index 64367dc..ef1b785 100644
--- a/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/DescriptorTest.java
@@ -75,27 +75,19 @@ public class DescriptorTest
 
     private void testFromFilenameFor(File dir)
     {
-        // normal
-        checkFromFilename(new Descriptor(dir, ksname, cfname, 1, SSTableFormat.Type.BIG), false);
-        // skip component (for streaming lock file)
-        checkFromFilename(new Descriptor(dir, ksname, cfname, 2, SSTableFormat.Type.BIG), true);
+        checkFromFilename(new Descriptor(dir, ksname, cfname, 1, SSTableFormat.Type.BIG));
 
         // secondary index
         String idxName = "myidx";
         File idxDir = new File(dir.getAbsolutePath() + File.separator + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName);
-        checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 4, SSTableFormat.Type.BIG), false);
-
-        // legacy version
-        checkFromFilename(new Descriptor("ja", dir, ksname, cfname, 1, SSTableFormat.Type.LEGACY), false);
-        // legacy secondary index
-        checkFromFilename(new Descriptor("ja", dir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 3, SSTableFormat.Type.LEGACY), false);
+        checkFromFilename(new Descriptor(idxDir, ksname, cfname + Directories.SECONDARY_INDEX_NAME_SEPARATOR + idxName, 4, SSTableFormat.Type.BIG));
     }
 
-    private void checkFromFilename(Descriptor original, boolean skipComponent)
+    private void checkFromFilename(Descriptor original)
     {
-        File file = new File(skipComponent ? original.baseFilename() : original.filenameFor(Component.DATA));
+        File file = new File(original.filenameFor(Component.DATA));
 
-        Pair<Descriptor, String> pair = Descriptor.fromFilename(file.getParentFile(), file.getName(), skipComponent);
+        Pair<Descriptor, Component> pair = Descriptor.fromFilenameWithComponent(file);
         Descriptor desc = pair.left;
 
         assertEquals(original.directory, desc.directory);
@@ -103,15 +95,7 @@ public class DescriptorTest
         assertEquals(original.cfname, desc.cfname);
         assertEquals(original.version, desc.version);
         assertEquals(original.generation, desc.generation);
-
-        if (skipComponent)
-        {
-            assertNull(pair.right);
-        }
-        else
-        {
-            assertEquals(Component.DATA.name(), pair.right);
-        }
+        assertEquals(Component.DATA, pair.right);
     }
 
     @Test
@@ -128,20 +112,10 @@ public class DescriptorTest
     @Test
     public void validateNames()
     {
-        // TODO tmp file name probably is not handled correctly after CASSANDRA-7066
         String[] names = {
-             // old formats
-             "system-schema_keyspaces-jb-1-Data.db",
-             //"system-schema_keyspaces-tmp-jb-1-Data.db",
-             "system-schema_keyspaces-ka-1-big-Data.db",
-             //"system-schema_keyspaces-tmp-ka-1-big-Data.db",
-             // 2ndary index
-             "keyspace1-standard1.idx1-ka-1-big-Data.db",
-             // new formats
-             "la-1-big-Data.db",
-             //"tmp-la-1-big-Data.db",
+             "ma-1-big-Data.db",
              // 2ndary index
-             ".idx1" + File.separator + "la-1-big-Data.db",
+             ".idx1" + File.separator + "ma-1-big-Data.db",
         };
 
         for (String name : names)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
index f3757a0..d21b3f8 100644
--- a/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/IndexSummaryTest.java
@@ -85,13 +85,13 @@ public class IndexSummaryTest
     {
         Pair<List<DecoratedKey>, IndexSummary> random = generateRandomIndex(100, 1);
         DataOutputBuffer dos = new DataOutputBuffer();
-        IndexSummary.serializer.serialize(random.right, dos, false);
+        IndexSummary.serializer.serialize(random.right, dos);
         // write junk
         dos.writeUTF("JUNK");
         dos.writeUTF("JUNK");
         FileUtils.closeQuietly(dos);
         DataInputStream dis = new DataInputStream(new ByteArrayInputStream(dos.toByteArray()));
-        IndexSummary is = IndexSummary.serializer.deserialize(dis, partitioner, false, 1, 1);
+        IndexSummary is = IndexSummary.serializer.deserialize(dis, partitioner, 1, 1);
         for (int i = 0; i < 100; i++)
             assertEquals(i, is.binarySearch(random.left.get(i)));
         // read the junk
@@ -115,9 +115,9 @@ public class IndexSummaryTest
             assertArrayEquals(new byte[0], summary.getKey(0));
 
             DataOutputBuffer dos = new DataOutputBuffer();
-            IndexSummary.serializer.serialize(summary, dos, false);
+            IndexSummary.serializer.serialize(summary, dos);
             DataInputStream dis = new DataInputStream(new ByteArrayInputStream(dos.toByteArray()));
-            IndexSummary loaded = IndexSummary.serializer.deserialize(dis, p, false, 1, 1);
+            IndexSummary loaded = IndexSummary.serializer.deserialize(dis, p, 1, 1);
 
             assertEquals(1, loaded.size());
             assertEquals(summary.getPosition(0), loaded.getPosition(0));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/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 ef993bc..8996f2a 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -71,7 +71,7 @@ public class LegacySSTableTest
      * See {@link #testGenerateSstables()} to generate sstables.
      * Take care on commit as you need to add the sstable files using {@code git add -f}
      */
-    public static final String[] legacyVersions = {"mc", "mb", "ma", "la", "ka", "jb"};
+    public static final String[] legacyVersions = {"mc", "mb", "ma"};
 
     // 1200 chars
     static final String longString = "0123456789012345678901234567890123456789012345678901234567890123456789012345678901234567890123456789" +
@@ -121,9 +121,12 @@ public class LegacySSTableTest
      */
     protected Descriptor getDescriptor(String legacyVersion, String table)
     {
-        return new Descriptor(legacyVersion, getTableDir(legacyVersion, table), "legacy_tables", table, 1,
-                              BigFormat.instance.getVersion(legacyVersion).hasNewFileName()?
-                              SSTableFormat.Type.BIG :SSTableFormat.Type.LEGACY);
+        return new Descriptor(SSTableFormat.Type.BIG.info.getVersion(legacyVersion),
+                              getTableDir(legacyVersion, table),
+                              "legacy_tables",
+                              table,
+                              1,
+                              SSTableFormat.Type.BIG);
     }
 
     @Test
@@ -242,10 +245,7 @@ public class LegacySSTableTest
         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());
+        Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
     }
 
     private static void verifyReads(String legacyVersion)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
index 942ebe9..238dbd0 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableRewriterTest.java
@@ -933,9 +933,9 @@ public class SSTableRewriterTest extends SSTableWriterTestBase
         for (int f = 0 ; f < fileCount ; f++)
         {
             File dir = cfs.getDirectories().getDirectoryForNewSSTables();
-            String filename = cfs.getSSTablePath(dir);
+            Descriptor desc = cfs.newSSTableDescriptor(dir);
 
-            try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, filename, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)))
+            try (SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, desc, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS)))
             {
                 int end = f == fileCount - 1 ? partitionCount : ((f + 1) * partitionCount) / fileCount;
                 for ( ; i < end ; i++)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
index df9d1aa..90b1857 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableUtils.java
@@ -217,7 +217,7 @@ public class SSTableUtils
             CFMetaData cfm = Schema.instance.getCFMetaData(ksname, cfname);
             ColumnFamilyStore cfs = Schema.instance.getColumnFamilyStoreInstance(cfm.cfId);
             SerializationHeader header = appender.header();
-            SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, datafile.getAbsolutePath(), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0, header);
+            SSTableTxnWriter writer = SSTableTxnWriter.create(cfs, Descriptor.fromFilename(datafile.getAbsolutePath()), expectedSize, ActiveRepairService.UNREPAIRED_SSTABLE, 0, header);
             while (appender.append(writer)) { /* pass */ }
             Collection<SSTableReader> readers = writer.finish(true);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
index a123a22..c1f11b6 100644
--- a/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
+++ b/test/unit/org/apache/cassandra/io/sstable/SSTableWriterTestBase.java
@@ -163,8 +163,8 @@ public class SSTableWriterTestBase extends SchemaLoader
 
     public static SSTableWriter getWriter(ColumnFamilyStore cfs, File directory, LifecycleTransaction txn)
     {
-        String filename = cfs.getSSTablePath(directory);
-        return SSTableWriter.create(filename, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS), cfs.indexManager.listIndexes(), txn);
+        Descriptor desc = cfs.newSSTableDescriptor(directory);
+        return SSTableWriter.create(desc, 0, 0, new SerializationHeader(true, cfs.metadata, cfs.metadata.partitionColumns(), EncodingStats.NO_STATS), cfs.indexManager.listIndexes(), txn);
     }
 
     public static ByteBuffer random(int i, int size)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/format/ClientModeSSTableTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/ClientModeSSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/format/ClientModeSSTableTest.java
deleted file mode 100644
index 7a741f9..0000000
--- a/test/unit/org/apache/cassandra/io/sstable/format/ClientModeSSTableTest.java
+++ /dev/null
@@ -1,133 +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.sstable.format;
-
-import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
-
-import java.io.File;
-import java.nio.ByteBuffer;
-
-import com.google.common.util.concurrent.Runnables;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.Slices;
-import org.apache.cassandra.db.filter.ColumnFilter;
-import org.apache.cassandra.db.marshal.BytesType;
-import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-import org.apache.cassandra.dht.ByteOrderedPartitioner;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.io.sstable.Descriptor;
-
-/**
- * Tests backwards compatibility for SSTables
- */
-public class ClientModeSSTableTest
-{
-    public static final String LEGACY_SSTABLE_PROP = "legacy-sstable-root";
-    public static final String KSNAME = "Keyspace1";
-    public static final String CFNAME = "Standard1";
-
-    public static File LEGACY_SSTABLE_ROOT;
-
-    static CFMetaData metadata;
-
-    @BeforeClass
-    public static void defineSchema() throws ConfigurationException
-    {
-        DatabaseDescriptor.toolInitialization();
-
-        metadata = CFMetaData.Builder.createDense(KSNAME, CFNAME, false, false)
-                                                .addPartitionKey("key", BytesType.instance)
-                                                .addClusteringColumn("column", BytesType.instance)
-                                                .addRegularColumn("value", BytesType.instance)
-                                                .withPartitioner(ByteOrderedPartitioner.instance)
-                                                .build();
-
-        String scp = System.getProperty(LEGACY_SSTABLE_PROP);
-        assert scp != null;
-        LEGACY_SSTABLE_ROOT = new File(scp).getAbsoluteFile();
-        assert LEGACY_SSTABLE_ROOT.isDirectory();
-    }
-
-    /**
-     * Get a descriptor for the legacy sstable at the given version.
-     */
-    protected Descriptor getDescriptor(String ver)
-    {
-        File directory = new File(LEGACY_SSTABLE_ROOT + File.separator + ver + File.separator + KSNAME);
-        return new Descriptor(ver, directory, KSNAME, CFNAME, 0, SSTableFormat.Type.LEGACY);
-    }
-
-    @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
-    {
-        SSTableReader reader = null;
-        try
-        {
-            reader = SSTableReader.openNoValidation(getDescriptor(version), metadata);
-
-            ByteBuffer key = bytes(Integer.toString(100));
-
-            try (UnfilteredRowIterator iter = reader.iterator(metadata.decorateKey(key), Slices.ALL, ColumnFilter.selection(metadata.partitionColumns()), false, false))
-            {
-                assert iter.next().clustering().get(0).equals(key);
-            }
-        }
-        catch (Throwable e)
-        {
-            System.err.println("Failed to read " + version);
-            throw e;
-        }
-        finally
-        {
-            if (reader != null)
-            {
-                int globalTidyCount = SSTableReader.GlobalTidy.lookup.size();
-                reader.selfRef().release();
-                assert reader.selfRef().globalCount() == 0;
-
-                // await clean-up to complete if started.
-                ScheduledExecutors.nonPeriodicTasks.submit(Runnables.doNothing()).get();
-                // Ensure clean-up completed.
-                assert SSTableReader.GlobalTidy.lookup.size() < globalTidyCount;
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
index f4c2f46..505d45d 100644
--- a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
@@ -89,7 +89,7 @@ public class SSTableFlushObserverTest
 
         SSTableFormat.Type sstableFormat = SSTableFormat.Type.current();
 
-        BigTableWriter writer = new BigTableWriter(new Descriptor(sstableFormat.info.getLatestVersion().version,
+        BigTableWriter writer = new BigTableWriter(new Descriptor(sstableFormat.info.getLatestVersion(),
                                                                   directory,
                                                                   KS_NAME, CF_NAME,
                                                                   0,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
index 9df3e11..79249b6 100644
--- a/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/metadata/MetadataSerializerTest.java
@@ -102,12 +102,6 @@ public class MetadataSerializerTest
     }
 
     @Test
-    public void testLaReadLb() throws IOException
-    {
-        testOldReadsNew("la", "lb");
-    }
-
-    @Test
     public void testMaReadMb() throws IOException
     {
         testOldReadsNew("ma", "mb");
@@ -134,7 +128,8 @@ public class MetadataSerializerTest
         File statsFileLb = serialize(originalMetadata, serializer, BigFormat.instance.getVersion(newV));
         File statsFileLa = serialize(originalMetadata, serializer, BigFormat.instance.getVersion(oldV));
         // Reading both as earlier version should yield identical results.
-        Descriptor desc = new Descriptor(oldV, statsFileLb.getParentFile(), "", "", 0, SSTableFormat.Type.current());
+        SSTableFormat.Type stype = SSTableFormat.Type.current();
+        Descriptor desc = new Descriptor(stype.info.getVersion(oldV), statsFileLb.getParentFile(), "", "", 0, stype);
         try (RandomAccessReader inLb = RandomAccessReader.open(statsFileLb);
              RandomAccessReader inLa = RandomAccessReader.open(statsFileLa))
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/4a246419/test/unit/org/apache/cassandra/io/util/MmappedRegionsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/util/MmappedRegionsTest.java b/test/unit/org/apache/cassandra/io/util/MmappedRegionsTest.java
index 39c9689..f6e97fb 100644
--- a/test/unit/org/apache/cassandra/io/util/MmappedRegionsTest.java
+++ b/test/unit/org/apache/cassandra/io/util/MmappedRegionsTest.java
@@ -314,7 +314,7 @@ public class MmappedRegionsTest
             writer.finish();
         }
 
-        CompressionMetadata metadata = new CompressionMetadata(cf.getAbsolutePath(), f.length(), ChecksumType.CRC32);
+        CompressionMetadata metadata = new CompressionMetadata(cf.getAbsolutePath(), f.length());
         try(ChannelProxy channel = new ChannelProxy(f);
             MmappedRegions regions = MmappedRegions.map(channel, metadata))
         {