You are viewing a plain text version of this content. The canonical link for it is here.
Posted to pr@cassandra.apache.org by GitBox <gi...@apache.org> on 2022/01/24 17:22:37 UTC

[GitHub] [cassandra] adelapena commented on a change in pull request #1405: Sstableverify unit test operate on SSTables

adelapena commented on a change in pull request #1405:
URL: https://github.com/apache/cassandra/pull/1405#discussion_r790981507



##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}
+ * Since the tool mainly exercises the {@link org.apache.cassandra.db.compaction.Verifier}, we elect to
+ * not run every conceivable option as many tests are already covered by {@link org.apache.cassandra.db.VerifyTest}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */
+public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
+{
+    @BeforeClass
+    public static void setup()
+    {
+        // since legacy tables test data uses ByteOrderedPartitioner that's what we need
+        // for the check version to work
+        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
+        System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing`
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
+    @Test
+    public void testCheckVersionValidVersion() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCheckVersionWorking";
+        String workingTable = "workingCheckTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable, "-c");
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCheckVersionWithWrongVersion() throws Exception {
+        String keyspace = "StandaloneVerifierTestWrongVersions";
+        String tableName = "legacy_ma_simple";
+
+        createAndPopulateTable(keyspace, tableName, cfs -> {
+            // lets just copy old version files from test data into the source dir

Review comment:
       ```suggestion
               // let's just copy old version files from test data into the source dir
   ```

##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}
+ * Since the tool mainly exercises the {@link org.apache.cassandra.db.compaction.Verifier}, we elect to
+ * not run every conceivable option as many tests are already covered by {@link org.apache.cassandra.db.VerifyTest}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */
+public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
+{
+    @BeforeClass
+    public static void setup()
+    {
+        // since legacy tables test data uses ByteOrderedPartitioner that's what we need
+        // for the check version to work
+        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
+        System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing`
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
+    @Test
+    public void testCheckVersionValidVersion() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCheckVersionWorking";
+        String workingTable = "workingCheckTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable, "-c");
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCheckVersionWithWrongVersion() throws Exception {
+        String keyspace = "StandaloneVerifierTestWrongVersions";
+        String tableName = "legacy_ma_simple";
+
+        createAndPopulateTable(keyspace, tableName, cfs -> {
+            // lets just copy old version files from test data into the source dir
+            File testDataDir = new File("test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple");
+            for (File cfsDir : cfs.getDirectories().getCFDirectories())
+            {
+                FileUtils.copyDirectory(testDataDir, cfsDir);
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspace, tableName, "-c");
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("is not the latest version, run upgradesstables");
+    }
+
+    @Test
+    public void testWorkingDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestWorkingDataKs";
+        String workingTable = "workingTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable);
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCorruptStatsFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptStatsKs";
+        String corruptStatsTable = "corruptStatsTable";
+        createAndPopulateTable(keyspaceName, corruptStatsTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.STATS), "rw"))
+            {
+                file.seek(0);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptStatsTable);
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStderr()).contains("Error Loading", corruptStatsTable);
+    }
+
+    @Test
+    public void testCorruptDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptDataKs";
+        String corruptDataTable = "corruptDataTable";
+
+        createAndPopulateTable(keyspaceName,corruptDataTable, cfs -> {

Review comment:
       ```suggestion
           createAndPopulateTable(keyspaceName, corruptDataTable, cfs -> {
   ```

##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}
+ * Since the tool mainly exercises the {@link org.apache.cassandra.db.compaction.Verifier}, we elect to
+ * not run every conceivable option as many tests are already covered by {@link org.apache.cassandra.db.VerifyTest}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */
+public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
+{
+    @BeforeClass
+    public static void setup()
+    {
+        // since legacy tables test data uses ByteOrderedPartitioner that's what we need
+        // for the check version to work
+        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
+        System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing`
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
+    @Test
+    public void testCheckVersionValidVersion() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCheckVersionWorking";
+        String workingTable = "workingCheckTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable, "-c");
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCheckVersionWithWrongVersion() throws Exception {
+        String keyspace = "StandaloneVerifierTestWrongVersions";
+        String tableName = "legacy_ma_simple";
+
+        createAndPopulateTable(keyspace, tableName, cfs -> {
+            // lets just copy old version files from test data into the source dir
+            File testDataDir = new File("test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple");
+            for (File cfsDir : cfs.getDirectories().getCFDirectories())
+            {
+                FileUtils.copyDirectory(testDataDir, cfsDir);
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspace, tableName, "-c");
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("is not the latest version, run upgradesstables");
+    }
+
+    @Test
+    public void testWorkingDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestWorkingDataKs";
+        String workingTable = "workingTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable);
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCorruptStatsFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptStatsKs";
+        String corruptStatsTable = "corruptStatsTable";
+        createAndPopulateTable(keyspaceName, corruptStatsTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.STATS), "rw"))
+            {
+                file.seek(0);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptStatsTable);
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStderr()).contains("Error Loading", corruptStatsTable);
+    }
+
+    @Test
+    public void testCorruptDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptDataKs";
+        String corruptDataTable = "corruptDataTable";
+
+        createAndPopulateTable(keyspaceName,corruptDataTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            long row0Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("0"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long row1Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("1"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long startPosition = Math.min(row0Start, row1Start);
+
+            try (RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw"))
+            {
+                file.seek(startPosition);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptDataTable);
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("Invalid SSTable", corruptDataTable);
+    }
+
+    /**
+     * Since we are testing a verifier, we'd like to corrupt files to verify code paths
+     * This function definition is used by {@link this#createAndPopulateTable}.
+     *
+     * CFS is the open ColumnFamilyStore for a given keyspace, table
+     */
+    private interface CorruptFunction {
+        public void apply(ColumnFamilyStore cfs) throws Exception;
+    }
+
+    /**
+     * This function sets up the keyspace, and table schema for a standardCFMD table.
+
+     * This will also populate the tableName with a few rows.  After completion the
+     * server will be shutdown.
+     *
+     * @param keyspace name the table should be created in

Review comment:
       Nit: Maybe something like:
   ```suggestion
        * @param keyspace the name of the keyspace in which the table should be created
   ```

##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}
+ * Since the tool mainly exercises the {@link org.apache.cassandra.db.compaction.Verifier}, we elect to
+ * not run every conceivable option as many tests are already covered by {@link org.apache.cassandra.db.VerifyTest}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */
+public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
+{
+    @BeforeClass
+    public static void setup()
+    {
+        // since legacy tables test data uses ByteOrderedPartitioner that's what we need
+        // for the check version to work
+        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
+        System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing`
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
+    @Test
+    public void testCheckVersionValidVersion() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCheckVersionWorking";
+        String workingTable = "workingCheckTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable, "-c");
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCheckVersionWithWrongVersion() throws Exception {
+        String keyspace = "StandaloneVerifierTestWrongVersions";
+        String tableName = "legacy_ma_simple";
+
+        createAndPopulateTable(keyspace, tableName, cfs -> {
+            // lets just copy old version files from test data into the source dir
+            File testDataDir = new File("test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple");
+            for (File cfsDir : cfs.getDirectories().getCFDirectories())
+            {
+                FileUtils.copyDirectory(testDataDir, cfsDir);
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspace, tableName, "-c");
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("is not the latest version, run upgradesstables");
+    }
+
+    @Test
+    public void testWorkingDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestWorkingDataKs";
+        String workingTable = "workingTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable);
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCorruptStatsFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptStatsKs";
+        String corruptStatsTable = "corruptStatsTable";
+        createAndPopulateTable(keyspaceName, corruptStatsTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.STATS), "rw"))
+            {
+                file.seek(0);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptStatsTable);
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStderr()).contains("Error Loading", corruptStatsTable);
+    }
+
+    @Test
+    public void testCorruptDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptDataKs";
+        String corruptDataTable = "corruptDataTable";
+
+        createAndPopulateTable(keyspaceName,corruptDataTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            long row0Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("0"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long row1Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("1"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long startPosition = Math.min(row0Start, row1Start);
+
+            try (RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw"))
+            {
+                file.seek(startPosition);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptDataTable);
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("Invalid SSTable", corruptDataTable);
+    }
+
+    /**
+     * Since we are testing a verifier, we'd like to corrupt files to verify code paths
+     * This function definition is used by {@link this#createAndPopulateTable}.
+     *
+     * CFS is the open ColumnFamilyStore for a given keyspace, table
+     */
+    private interface CorruptFunction {
+        public void apply(ColumnFamilyStore cfs) throws Exception;
+    }
+
+    /**
+     * This function sets up the keyspace, and table schema for a standardCFMD table.
+
+     * This will also populate the tableName with a few rows.  After completion the
+     * server will be shutdown.
+     *
+     * @param keyspace name the table should be created in
+     * @param tableName new table name of the standard CFMD table
+     * @param corruptionFn function called to corrupt or change the contents on disk, is passed the Cfs of the table name.
+     * @throws Exception on error.
+     */
+    private void createAndPopulateTable(String keyspace, String tableName, CorruptFunction corruptionFn) throws Exception
+    {
+        SchemaLoader.createKeyspace(keyspace,
+                                    KeyspaceParams.simple(1),
+                                    standardCFMD(keyspace, tableName));
+
+        CompactionManager.instance.disableAutoCompaction();
+
+        Keyspace k = Keyspace.open(keyspace);
+        ColumnFamilyStore cfs = k.getColumnFamilyStore(tableName);
+
+        populateTable(cfs, 2);
+
+        corruptionFn.apply(cfs);
+    }
+
+    private void populateTable(ColumnFamilyStore cfs, int partitionsPerSSTable)

Review comment:
       This method could be static. It could also be inlined into `createAndPopulateTable`, but I don't have a strong preference on this.

##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}

Review comment:
       Nit: we can consistently use JavaDoc links
   ```suggestion
    * Class that tests tables for {@link StandaloneVerifier} by updating using {@link SchemaLoader}
    * Similar in vein to other {@link SchemaLoader} type tests, as well as {@link StandaloneUpgraderOnSStablesTest}.
   ```

##########
File path: test/unit/org/apache/cassandra/db/VerifyTest.java
##########
@@ -69,6 +73,14 @@
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
+/**
+ * Test for {@link Verifier}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */

Review comment:
       Nice comment, it's useful to be aware of the other tests around. I'd use JavaDoc links mainly for the sake of future refactors. Also, being picky I think we can omit the mention of CI since one can be checking coverage without any CI:
   ```suggestion
    * Note: the complete coverage is composed of:
    * - {@link org.apache.cassandra.tools.StandaloneVerifierOnSSTablesTest}
    * - {@link org.apache.cassandra.tools.StandaloneVerifierTest}
    * - {@link VerifyTest}
    */
   ```

##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}
+ * Since the tool mainly exercises the {@link org.apache.cassandra.db.compaction.Verifier}, we elect to
+ * not run every conceivable option as many tests are already covered by {@link org.apache.cassandra.db.VerifyTest}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */
+public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
+{
+    @BeforeClass
+    public static void setup()
+    {
+        // since legacy tables test data uses ByteOrderedPartitioner that's what we need
+        // for the check version to work
+        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
+        System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing`
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
+    @Test
+    public void testCheckVersionValidVersion() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCheckVersionWorking";
+        String workingTable = "workingCheckTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable, "-c");
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCheckVersionWithWrongVersion() throws Exception {
+        String keyspace = "StandaloneVerifierTestWrongVersions";
+        String tableName = "legacy_ma_simple";
+
+        createAndPopulateTable(keyspace, tableName, cfs -> {
+            // lets just copy old version files from test data into the source dir
+            File testDataDir = new File("test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple");
+            for (File cfsDir : cfs.getDirectories().getCFDirectories())
+            {
+                FileUtils.copyDirectory(testDataDir, cfsDir);
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspace, tableName, "-c");
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("is not the latest version, run upgradesstables");
+    }
+
+    @Test
+    public void testWorkingDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestWorkingDataKs";
+        String workingTable = "workingTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable);
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCorruptStatsFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptStatsKs";
+        String corruptStatsTable = "corruptStatsTable";
+        createAndPopulateTable(keyspaceName, corruptStatsTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.STATS), "rw"))
+            {
+                file.seek(0);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptStatsTable);
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStderr()).contains("Error Loading", corruptStatsTable);
+    }
+
+    @Test
+    public void testCorruptDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptDataKs";
+        String corruptDataTable = "corruptDataTable";
+
+        createAndPopulateTable(keyspaceName,corruptDataTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            long row0Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("0"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long row1Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("1"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long startPosition = Math.min(row0Start, row1Start);
+
+            try (RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw"))
+            {
+                file.seek(startPosition);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptDataTable);
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("Invalid SSTable", corruptDataTable);
+    }
+
+    /**
+     * Since we are testing a verifier, we'd like to corrupt files to verify code paths
+     * This function definition is used by {@link this#createAndPopulateTable}.
+     *
+     * CFS is the open ColumnFamilyStore for a given keyspace, table
+     */
+    private interface CorruptFunction {

Review comment:
       Nit: we can add a `@FunctionalInterface` annotation

##########
File path: test/unit/org/apache/cassandra/tools/StandaloneVerifierOnSSTablesTest.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.tools;
+
+import java.io.File;
+import java.io.RandomAccessFile;
+
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.StringUtils;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import org.apache.cassandra.SchemaLoader;
+import org.apache.cassandra.UpdateBuilder;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.PartitionPosition;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.io.sstable.Component;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.tools.ToolRunner.ToolResult;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.assertj.core.api.Assertions;
+
+import static org.apache.cassandra.SchemaLoader.standardCFMD;
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Class that tests tables for StandaloneVerifier by updating using SchemaLoader
+ * Similar in vein to other SchemaLoader type tests, as well as {@link StandaloneUpgraderOnSStablesTest}
+ * Since the tool mainly exercises the {@link org.apache.cassandra.db.compaction.Verifier}, we elect to
+ * not run every conceivable option as many tests are already covered by {@link org.apache.cassandra.db.VerifyTest}.
+ * 
+ * Note: the complete coverage for CI is composed of:
+ * - StandaloneVerifierOnSSTablesTest
+ * - StandaloneVerifierTest
+ * - VerifyTest
+ */
+public class StandaloneVerifierOnSSTablesTest extends OfflineToolUtils
+{
+    @BeforeClass
+    public static void setup()
+    {
+        // since legacy tables test data uses ByteOrderedPartitioner that's what we need
+        // for the check version to work
+        System.setProperty("cassandra.partitioner", "org.apache.cassandra.dht.ByteOrderedPartitioner");
+        System.setProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST, "true"); // Necessary for testing`
+        SchemaLoader.loadSchema();
+        StorageService.instance.initServer();
+    }
+
+    @AfterClass
+    public static void teardown() throws Exception
+    {
+        SchemaLoader.cleanupAndLeaveDirs();
+        System.clearProperty(Util.ALLOW_TOOL_REINIT_FOR_TEST);
+    }
+
+    @Test
+    public void testCheckVersionValidVersion() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCheckVersionWorking";
+        String workingTable = "workingCheckTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable, "-c");
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCheckVersionWithWrongVersion() throws Exception {
+        String keyspace = "StandaloneVerifierTestWrongVersions";
+        String tableName = "legacy_ma_simple";
+
+        createAndPopulateTable(keyspace, tableName, cfs -> {
+            // lets just copy old version files from test data into the source dir
+            File testDataDir = new File("test/data/legacy-sstables/ma/legacy_tables/legacy_ma_simple");
+            for (File cfsDir : cfs.getDirectories().getCFDirectories())
+            {
+                FileUtils.copyDirectory(testDataDir, cfsDir);
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspace, tableName, "-c");
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("is not the latest version, run upgradesstables");
+    }
+
+    @Test
+    public void testWorkingDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestWorkingDataKs";
+        String workingTable = "workingTable";
+
+        createAndPopulateTable(keyspaceName, workingTable, x -> {});
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, workingTable);
+        assertEquals(0, tool.getExitCode());
+        assertCorrectEnvPostTest();
+        tool.assertOnCleanExit();
+    }
+
+    @Test
+    public void testCorruptStatsFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptStatsKs";
+        String corruptStatsTable = "corruptStatsTable";
+        createAndPopulateTable(keyspaceName, corruptStatsTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            try (RandomAccessFile file = new RandomAccessFile(sstable.descriptor.filenameFor(Component.STATS), "rw"))
+            {
+                file.seek(0);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptStatsTable);
+
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStderr()).contains("Error Loading", corruptStatsTable);
+    }
+
+    @Test
+    public void testCorruptDataFile() throws Exception
+    {
+        String keyspaceName = "StandaloneVerifierTestCorruptDataKs";
+        String corruptDataTable = "corruptDataTable";
+
+        createAndPopulateTable(keyspaceName,corruptDataTable, cfs -> {
+            SSTableReader sstable = cfs.getLiveSSTables().iterator().next();
+            long row0Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("0"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long row1Start = sstable.getPosition(PartitionPosition.ForKey.get(ByteBufferUtil.bytes("1"), cfs.getPartitioner()), SSTableReader.Operator.EQ).position;
+            long startPosition = Math.min(row0Start, row1Start);
+
+            try (RandomAccessFile file = new RandomAccessFile(sstable.getFilename(), "rw"))
+            {
+                file.seek(startPosition);
+                file.writeBytes(StringUtils.repeat('z', 2));
+            }
+        });
+
+        ToolResult tool = ToolRunner.invokeClass(StandaloneVerifier.class, keyspaceName, corruptDataTable);
+        assertEquals(1, tool.getExitCode());
+        Assertions.assertThat(tool.getStdout()).contains("Invalid SSTable", corruptDataTable);
+    }
+
+    /**
+     * Since we are testing a verifier, we'd like to corrupt files to verify code paths
+     * This function definition is used by {@link this#createAndPopulateTable}.
+     *
+     * CFS is the open ColumnFamilyStore for a given keyspace, table
+     */
+    private interface CorruptFunction {
+        public void apply(ColumnFamilyStore cfs) throws Exception;
+    }
+
+    /**
+     * This function sets up the keyspace, and table schema for a standardCFMD table.
+
+     * This will also populate the tableName with a few rows.  After completion the
+     * server will be shutdown.
+     *
+     * @param keyspace name the table should be created in
+     * @param tableName new table name of the standard CFMD table
+     * @param corruptionFn function called to corrupt or change the contents on disk, is passed the Cfs of the table name.
+     * @throws Exception on error.
+     */
+    private void createAndPopulateTable(String keyspace, String tableName, CorruptFunction corruptionFn) throws Exception

Review comment:
       This method could be static.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: pr-unsubscribe@cassandra.apache.org
For additional commands, e-mail: pr-help@cassandra.apache.org