You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by pa...@apache.org on 2018/02/11 13:25:46 UTC

[01/29] cassandra git commit: Protect against overflow of local expiration time

Repository: cassandra
Updated Branches:
  refs/heads/cassandra-2.1 5ba9e6da9 -> b2949439e
  refs/heads/cassandra-2.2 aa831c98f -> 1602e6063
  refs/heads/cassandra-3.0 21978bf9b -> c231ed5be
  refs/heads/cassandra-3.11 6eb65e5a2 -> 0a6b6f506
  refs/heads/trunk 161c37da8 -> 6034c268c


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
new file mode 100644
index 0000000..ab4ef21
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -0,0 +1,410 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.BufferExpiringCell;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+public class TTLTest extends CQLTester
+{
+    public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+
+    public static int MAX_TTL = ExpiringCell.MAX_TTL;
+
+    public static final String SIMPLE_NOCLUSTERING = "table1";
+    public static final String SIMPLE_CLUSTERING = "table2";
+    public static final String COMPLEX_NOCLUSTERING = "table3";
+    public static final String COMPLEX_CLUSTERING = "table4";
+
+    @Test
+    public void testTTLPerRequestLimit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        // insert with low TTL should not be denied
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+        execute("TRUNCATE %s");
+
+        // insert with low TTL should not be denied
+        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+    }
+
+
+    @Test
+    public void testTTLDefaultLimit() throws Throwable
+    {
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live cannot be smaller than 0"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                        + (MAX_TTL + 1));
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                  + (MAX_TTL + 1) + ")"));
+        }
+
+        // table with default low TTL should not be denied
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+    }
+
+    @Test
+    public void testRejectExpirationDateOverflowPolicy() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+            execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        checkTTLIsCapped("i");
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyPerRequest() throws Throwable
+    {
+        // Test cap policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+
+        // simple column, clustering, flush
+        baseCapExpirationDateOverflowTest(true, true, true);
+        // simple column, clustering, noflush
+        baseCapExpirationDateOverflowTest(true, true, false);
+        // simple column, noclustering, flush
+        baseCapExpirationDateOverflowTest(true, false, true);
+        // simple column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(true, false, false);
+        // complex column, clustering, flush
+        baseCapExpirationDateOverflowTest(false, true, true);
+        // complex column, clustering, noflush
+        baseCapExpirationDateOverflowTest(false, true, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, true);
+        // complex column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(false, false, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, false);
+
+        // Return to previous policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+    {
+        createTable(true, true);
+        createTable(true, false);
+        createTable(false, true);
+        createTable(false, false);
+
+        baseTestRecoverOverflowedExpiration(false, false);
+        baseTestRecoverOverflowedExpiration(true, false);
+        baseTestRecoverOverflowedExpiration(true, true);
+    }
+
+    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+    {
+        // Create Table
+        if (simple)
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b int)");
+        }
+        else
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b set<text>)");
+        }
+
+        // Insert data with INSERT and UPDATE
+        if (simple)
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+            if (clustering)
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+            else
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+        }
+        else
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+            if (clustering)
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+            else
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+        }
+
+        // Maybe Flush
+        Keyspace ks = Keyspace.open(keyspace());
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+        // Verify data
+        verifyData(simple);
+
+        // Maybe major compact
+        if (flush)
+        {
+            // Major compact and check data is still present
+            ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+
+            // Verify data again
+            verifyData(simple);
+        }
+    }
+
+    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        // simple column, clustering
+        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+        // simple column, noclustering
+        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+        // complex column, clustering
+        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+        // complex column, noclustering
+        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+    }
+
+    private void verifyData(boolean simple) throws Throwable
+    {
+        if (simple)
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+        }
+        else
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        // Cannot retrieve TTL from collections
+        if (simple)
+            checkTTLIsCapped("b");
+    }
+
+    /**
+     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+     */
+    private void checkTTLIsCapped(String field) throws Throwable
+    {
+
+        // TTL is computed dynamically from row expiration time, so if it is
+        // equal or higher to the minimum max TTL we compute before the query
+        // we are fine.
+        int minMaxTTL = computeMaxTTL();
+        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+        for (UntypedResultSet.Row row : execute)
+        {
+            int ttl = row.getInt("ttl(" + field + ")");
+            assertTrue(ttl >= minMaxTTL);
+        }
+    }
+
+    /**
+     * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+     */
+    private int computeMaxTTL()
+    {
+        int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+    }
+
+    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        if (reinsertOverflowedTTL)
+        {
+            assert runScrub;
+        }
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
+
+        assertEquals(0, cfs.getLiveSSTableCount());
+
+        copySSTablesToTableDir(simple, clustering);
+
+        cfs.loadNewSSTables();
+
+        if (runScrub)
+        {
+            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+        }
+
+        if (reinsertOverflowedTTL)
+        {
+            if (simple)
+            {
+                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
+                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
+
+            }
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+
+            cfs.forceMajorCompaction();
+
+            if (simple)
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        else
+        {
+            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
+        }
+        cfs.truncateBlocking(); //cleanup for next tests
+    }
+
+    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
+    {
+        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
+        File sourceDir = getTableDir(simple, clustering);
+        for (File file : sourceDir.listFiles())
+        {
+            copyFile(file, destDir);
+        }
+    }
+
+    private void createTable(boolean simple, boolean clustering) throws Throwable
+    {
+        if (simple)
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
+        }
+        else
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
+        }
+    }
+
+    private static File getTableDir(boolean simple, boolean clustering)
+    {
+        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+    }
+
+    private static void copyFile(File src, File dest) throws IOException
+    {
+        byte[] buf = new byte[65536];
+        if (src.isFile())
+        {
+            File target = new File(dest, src.getName());
+            int rd;
+            FileInputStream is = new FileInputStream(src);
+            FileOutputStream os = new FileOutputStream(target);
+            while ((rd = is.read(buf)) >= 0)
+                os.write(buf, 0, rd);
+        }
+    }
+
+    public static String getTableName(boolean simple, boolean clustering)
+    {
+        if (simple)
+            return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+        else
+            return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+    }
+}


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


[17/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LegacyLayout.java
index 2117dd6,0000000..912d591
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@@ -1,2504 -1,0 +1,2509 @@@
 +/*
 + * 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;
 +
 +import java.io.DataInput;
 +import java.io.IOException;
 +import java.io.IOError;
 +import java.nio.ByteBuffer;
 +import java.security.MessageDigest;
 +import java.util.*;
 +
 +import org.apache.cassandra.cql3.SuperColumnCompatibility;
++import org.apache.cassandra.thrift.Column;
 +import org.apache.cassandra.utils.AbstractIterator;
 +import com.google.common.collect.Iterators;
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.PeekingIterator;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.utils.*;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 +
 +/**
 + * Functions to deal with the old format.
 + */
 +public abstract class LegacyLayout
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LegacyLayout.class);
 +
 +    public final static int MAX_CELL_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
 +
 +    public final static int STATIC_PREFIX = 0xFFFF;
 +
 +    public final static int DELETION_MASK        = 0x01;
 +    public final static int EXPIRATION_MASK      = 0x02;
 +    public final static int COUNTER_MASK         = 0x04;
 +    public final static int COUNTER_UPDATE_MASK  = 0x08;
 +    private final static int RANGE_TOMBSTONE_MASK = 0x10;
 +
 +    private LegacyLayout() {}
 +
 +    public static AbstractType<?> makeLegacyComparator(CFMetaData metadata)
 +    {
 +        ClusteringComparator comparator = metadata.comparator;
 +        if (!metadata.isCompound())
 +        {
 +            assert comparator.size() == 1;
 +            return comparator.subtype(0);
 +        }
 +
 +        boolean hasCollections = metadata.hasCollectionColumns() || metadata.hasDroppedCollectionColumns();
 +        List<AbstractType<?>> types = new ArrayList<>(comparator.size() + (metadata.isDense() ? 0 : 1) + (hasCollections ? 1 : 0));
 +
 +        types.addAll(comparator.subtypes());
 +
 +        if (!metadata.isDense())
 +        {
 +            types.add(UTF8Type.instance);
 +
 +            if (hasCollections)
 +            {
 +                Map<ByteBuffer, CollectionType> defined = new HashMap<>();
 +
 +                for (CFMetaData.DroppedColumn def : metadata.getDroppedColumns().values())
 +                    if (def.type instanceof CollectionType && def.type.isMultiCell())
 +                        defined.put(bytes(def.name), (CollectionType) def.type);
 +
 +                for (ColumnDefinition def : metadata.partitionColumns())
 +                    if (def.type instanceof CollectionType && def.type.isMultiCell())
 +                        defined.put(def.name.bytes, (CollectionType) def.type);
 +
 +                types.add(ColumnToCollectionType.getInstance(defined));
 +            }
 +        }
 +        return CompositeType.getInstance(types);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer cellname)
 +    throws UnknownColumnException
 +    {
 +        assert cellname != null;
 +        if (metadata.isSuper())
 +        {
 +            assert superColumnName != null;
 +            return decodeForSuperColumn(metadata, new Clustering(superColumnName), cellname);
 +        }
 +
 +        assert superColumnName == null;
 +        return decodeCellName(metadata, cellname);
 +    }
 +
 +    private static LegacyCellName decodeForSuperColumn(CFMetaData metadata, Clustering clustering, ByteBuffer subcol)
 +    {
 +        ColumnDefinition def = metadata.getColumnDefinition(subcol);
 +        if (def != null)
 +        {
 +            // it's a statically defined subcolumn
 +            return new LegacyCellName(clustering, def, null);
 +        }
 +
 +        def = metadata.compactValueColumn();
 +        assert def != null && def.type instanceof MapType;
 +        return new LegacyCellName(clustering, def, subcol);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException
 +    {
 +        return decodeCellName(metadata, cellname, false);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException
 +    {
 +        Clustering clustering = decodeClustering(metadata, cellname);
 +
 +        if (metadata.isSuper())
 +            return decodeForSuperColumn(metadata, clustering, CompositeType.extractComponent(cellname, 1));
 +
 +        if (metadata.isDense() || (metadata.isCompactTable() && readAllAsDynamic))
 +            return new LegacyCellName(clustering, metadata.compactValueColumn(), null);
 +
 +        ByteBuffer column = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size()) : cellname;
 +        if (column == null)
 +        {
 +            // Tables for composite 2ndary indexes used to be compound but dense, but we've transformed them into regular tables
 +            // (non compact ones) but with no regular column (i.e. we only care about the clustering). So we'll get here
 +            // in that case, and what we want to return is basically a row marker.
 +            if (metadata.partitionColumns().isEmpty())
 +                return new LegacyCellName(clustering, null, null);
 +
 +            // Otherwise, we shouldn't get there
 +            throw new IllegalArgumentException("No column name component found in cell name");
 +        }
 +
 +        // Row marker, this is ok
 +        if (!column.hasRemaining())
 +            return new LegacyCellName(clustering, null, null);
 +
 +        ColumnDefinition def = metadata.getColumnDefinition(column);
 +        if ((def == null) || def.isPrimaryKeyColumn())
 +        {
 +            // If it's a compact table, it means the column is in fact a "dynamic" one
 +            if (metadata.isCompactTable())
 +                return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null);
 +
 +            if (def == null)
 +                throw new UnknownColumnException(metadata, column);
 +            else
 +                throw new IllegalArgumentException("Cannot add primary key column to partition update");
 +        }
 +
 +        ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null;
 +
 +        // Note that because static compact columns are translated to static defs in the new world order, we need to force a static
 +        // clustering if the definition is static (as it might not be in this case).
 +        return new LegacyCellName(def.isStatic() ? Clustering.STATIC_CLUSTERING : clustering, def, collectionElement);
 +    }
 +
 +    public static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, boolean isStart)
 +    {
 +        if (!bound.hasRemaining())
 +            return isStart ? LegacyBound.BOTTOM : LegacyBound.TOP;
 +
 +        if (!metadata.isCompound())
 +        {
 +            // The non compound case is a lot easier, in that there is no EOC nor collection to worry about, so dealing
 +            // with that first.
 +            return new LegacyBound(isStart ? Slice.Bound.inclusiveStartOf(bound) : Slice.Bound.inclusiveEndOf(bound), false, null);
 +        }
 +
 +        int clusteringSize = metadata.comparator.size();
 +
 +        List<ByteBuffer> components = CompositeType.splitName(bound);
 +        byte eoc = CompositeType.lastEOC(bound);
 +
 +        // There can be  more components than the clustering size only in the case this is the bound of a collection
 +        // range tombstone. In which case, there is exactly one more component, and that component is the name of the
 +        // collection being selected/deleted.
 +        assert components.size() <= clusteringSize || (!metadata.isCompactTable() && components.size() == clusteringSize + 1);
 +
 +        ColumnDefinition collectionName = null;
 +        if (components.size() > clusteringSize)
 +            collectionName = metadata.getColumnDefinition(components.remove(clusteringSize));
 +
 +        boolean isInclusive;
 +        if (isStart)
 +        {
 +            isInclusive = eoc <= 0;
 +        }
 +        else
 +        {
 +            isInclusive = eoc >= 0;
 +
 +            // for an end bound, if we only have a prefix of all the components and the final EOC is zero,
 +            // then it should only match up to the prefix but no further, that is, it is an inclusive bound
 +            // of the exact prefix but an exclusive bound of anything beyond it, so adding an empty
 +            // composite value ensures this behavior, see CASSANDRA-12423 for more details
 +            if (eoc == 0 && components.size() < clusteringSize)
 +            {
 +                components.add(ByteBufferUtil.EMPTY_BYTE_BUFFER);
 +                isInclusive = false;
 +            }
 +        }
 +
 +        Slice.Bound.Kind boundKind = Slice.Bound.boundKind(isStart, isInclusive);
 +        Slice.Bound sb = Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
 +        return new LegacyBound(sb, metadata.isCompound() && CompositeType.isStaticName(bound), collectionName);
 +    }
 +
 +    public static ByteBuffer encodeBound(CFMetaData metadata, Slice.Bound bound, boolean isStart)
 +    {
 +        if (bound == Slice.Bound.BOTTOM || bound == Slice.Bound.TOP || metadata.comparator.size() == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        ClusteringPrefix clustering = bound.clustering();
 +
 +        if (!metadata.isCompound())
 +        {
 +            assert clustering.size() == 1;
 +            return clustering.get(0);
 +        }
 +
 +        CompositeType ctype = CompositeType.getInstance(metadata.comparator.subtypes());
 +        CompositeType.Builder builder = ctype.builder();
 +        for (int i = 0; i < clustering.size(); i++)
 +            builder.add(clustering.get(i));
 +
 +        if (isStart)
 +            return bound.isInclusive() ? builder.build() : builder.buildAsEndOfRange();
 +        else
 +            return bound.isInclusive() ? builder.buildAsEndOfRange() : builder.build();
 +    }
 +
 +    public static ByteBuffer encodeCellName(CFMetaData metadata, ClusteringPrefix clustering, ByteBuffer columnName, ByteBuffer collectionElement)
 +    {
 +        boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
 +
 +        if (!metadata.isCompound())
 +        {
 +            if (isStatic)
 +                return columnName;
 +
 +            assert clustering.size() == 1 : "Expected clustering size to be 1, but was " + clustering.size();
 +            return clustering.get(0);
 +        }
 +
 +        // We use comparator.size() rather than clustering.size() because of static clusterings
 +        int clusteringSize = metadata.comparator.size();
 +        int size = clusteringSize + (metadata.isDense() ? 0 : 1) + (collectionElement == null ? 0 : 1);
 +        if (metadata.isSuper())
 +            size = clusteringSize + 1;
 +        ByteBuffer[] values = new ByteBuffer[size];
 +        for (int i = 0; i < clusteringSize; i++)
 +        {
 +            if (isStatic)
 +            {
 +                values[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +                continue;
 +            }
 +
 +            ByteBuffer v = clustering.get(i);
 +            // we can have null (only for dense compound tables for backward compatibility reasons) but that
 +            // means we're done and should stop there as far as building the composite is concerned.
 +            if (v == null)
 +                return CompositeType.build(Arrays.copyOfRange(values, 0, i));
 +
 +            values[i] = v;
 +        }
 +
 +        if (metadata.isSuper())
 +        {
 +            // We need to set the "column" (in thrift terms) name, i.e. the value corresponding to the subcomparator.
 +            // What it is depends if this a cell for a declared "static" column or a "dynamic" column part of the
 +            // super-column internal map.
 +            assert columnName != null; // This should never be null for supercolumns, see decodeForSuperColumn() above
 +            values[clusteringSize] = columnName.equals(SuperColumnCompatibility.SUPER_COLUMN_MAP_COLUMN)
 +                                   ? collectionElement
 +                                   : columnName;
 +        }
 +        else
 +        {
 +            if (!metadata.isDense())
 +                values[clusteringSize] = columnName;
 +            if (collectionElement != null)
 +                values[clusteringSize + 1] = collectionElement;
 +        }
 +
 +        return CompositeType.build(isStatic, values);
 +    }
 +
 +    public static Clustering decodeClustering(CFMetaData metadata, ByteBuffer value)
 +    {
 +        int csize = metadata.comparator.size();
 +        if (csize == 0)
 +            return Clustering.EMPTY;
 +
 +        if (metadata.isCompound() && CompositeType.isStaticName(value))
 +            return Clustering.STATIC_CLUSTERING;
 +
 +        List<ByteBuffer> components = metadata.isCompound()
 +                                    ? CompositeType.splitName(value)
 +                                    : Collections.singletonList(value);
 +
 +        return new Clustering(components.subList(0, Math.min(csize, components.size())).toArray(new ByteBuffer[csize]));
 +    }
 +
 +    public static ByteBuffer encodeClustering(CFMetaData metadata, ClusteringPrefix clustering)
 +    {
 +        if (clustering.size() == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        if (!metadata.isCompound())
 +        {
 +            assert clustering.size() == 1;
 +            return clustering.get(0);
 +        }
 +
 +        ByteBuffer[] values = new ByteBuffer[clustering.size()];
 +        for (int i = 0; i < clustering.size(); i++)
 +            values[i] = clustering.get(i);
 +        return CompositeType.build(values);
 +    }
 +
 +    /**
 +     * The maximum number of cells to include per partition when converting to the old format.
 +     * <p>
 +     * We already apply the limit during the actual query, but for queries that counts cells and not rows (thrift queries
 +     * and distinct queries as far as old nodes are concerned), we may still include a little bit more than requested
 +     * because {@link DataLimits} always include full rows. So if the limit ends in the middle of a queried row, the
 +     * full row will be part of our result. This would confuse old nodes however so we make sure to truncate it to
 +     * what's expected before writting it on the wire.
 +     *
 +     * @param command the read commmand for which to determine the maximum cells per partition. This can be {@code null}
 +     * in which case {@code Integer.MAX_VALUE} is returned.
 +     * @return the maximum number of cells per partition that should be enforced according to the read command if
 +     * post-query limitation are in order (see above). This will be {@code Integer.MAX_VALUE} if no such limits are
 +     * necessary.
 +     */
 +    private static int maxCellsPerPartition(ReadCommand command)
 +    {
 +        if (command == null)
 +            return Integer.MAX_VALUE;
 +
 +        DataLimits limits = command.limits();
 +
 +        // There is 2 types of DISTINCT queries: those that includes only the partition key, and those that include static columns.
 +        // On old nodes, the latter expects the first row in term of CQL count, which is what we already have and there is no additional
 +        // limit to apply. The former however expect only one cell per partition and rely on it (See CASSANDRA-10762).
 +        if (limits.isDistinct())
 +            return command.columnFilter().fetchedColumns().statics.isEmpty() ? 1 : Integer.MAX_VALUE;
 +
 +        switch (limits.kind())
 +        {
 +            case THRIFT_LIMIT:
 +            case SUPER_COLUMN_COUNTING_LIMIT:
 +                return limits.perPartitionCount();
 +            default:
 +                return Integer.MAX_VALUE;
 +        }
 +    }
 +
 +    // For serializing to old wire format
 +    public static LegacyUnfilteredPartition fromUnfilteredRowIterator(ReadCommand command, UnfilteredRowIterator iterator)
 +    {
 +        // we need to extract the range tombstone so materialize the partition. Since this is
 +        // used for the on-wire format, this is not worst than it used to be.
 +        final ImmutableBTreePartition partition = ImmutableBTreePartition.create(iterator);
 +        DeletionInfo info = partition.deletionInfo();
 +        Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> pair = fromRowIterator(partition.metadata(), partition.iterator(), partition.staticRow());
 +
 +        LegacyLayout.LegacyRangeTombstoneList rtl = pair.left;
 +
 +        // Processing the cell iterator results in the LegacyRangeTombstoneList being populated, so we do this
 +        // before we use the LegacyRangeTombstoneList at all
 +        List<LegacyLayout.LegacyCell> cells = Lists.newArrayList(pair.right);
 +
 +        int maxCellsPerPartition = maxCellsPerPartition(command);
 +        if (cells.size() > maxCellsPerPartition)
 +            cells = cells.subList(0, maxCellsPerPartition);
 +
 +        // The LegacyRangeTombstoneList already has range tombstones for the single-row deletions and complex
 +        // deletions.  Go through our normal range tombstones and add then to the LegacyRTL so that the range
 +        // tombstones all get merged and sorted properly.
 +        if (info.hasRanges())
 +        {
 +            Iterator<RangeTombstone> rangeTombstoneIterator = info.rangeIterator(false);
 +            while (rangeTombstoneIterator.hasNext())
 +            {
 +                RangeTombstone rt = rangeTombstoneIterator.next();
 +                Slice slice = rt.deletedSlice();
 +                LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(slice.start(), false, null);
 +                LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(slice.end(), false, null);
 +                rtl.add(start, end, rt.deletionTime().markedForDeleteAt(), rt.deletionTime().localDeletionTime());
 +            }
 +        }
 +
 +        return new LegacyUnfilteredPartition(info.getPartitionDeletion(), rtl, cells);
 +    }
 +
 +    public static void serializeAsLegacyPartition(ReadCommand command, UnfilteredRowIterator partition, DataOutputPlus out, int version) throws IOException
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        out.writeBoolean(true);
 +
 +        LegacyLayout.LegacyUnfilteredPartition legacyPartition = LegacyLayout.fromUnfilteredRowIterator(command, partition);
 +
 +        UUIDSerializer.serializer.serialize(partition.metadata().cfId, out, version);
 +        DeletionTime.serializer.serialize(legacyPartition.partitionDeletion, out);
 +
 +        legacyPartition.rangeTombstones.serialize(out, partition.metadata());
 +
 +        // begin cell serialization
 +        out.writeInt(legacyPartition.cells.size());
 +        for (LegacyLayout.LegacyCell cell : legacyPartition.cells)
 +        {
 +            ByteBufferUtil.writeWithShortLength(cell.name.encode(partition.metadata()), out);
 +            out.writeByte(cell.serializationFlags());
 +            if (cell.isExpiring())
 +            {
 +                out.writeInt(cell.ttl);
 +                out.writeInt(cell.localDeletionTime);
 +            }
 +            else if (cell.isTombstone())
 +            {
 +                out.writeLong(cell.timestamp);
 +                out.writeInt(TypeSizes.sizeof(cell.localDeletionTime));
 +                out.writeInt(cell.localDeletionTime);
 +                continue;
 +            }
 +            else if (cell.isCounterUpdate())
 +            {
 +                out.writeLong(cell.timestamp);
 +                long count = CounterContext.instance().getLocalCount(cell.value);
 +                ByteBufferUtil.writeWithLength(ByteBufferUtil.bytes(count), out);
 +                continue;
 +            }
 +            else if (cell.isCounter())
 +            {
 +                out.writeLong(Long.MIN_VALUE);  // timestampOfLastDelete (not used, and MIN_VALUE is the default)
 +            }
 +
 +            out.writeLong(cell.timestamp);
 +            ByteBufferUtil.writeWithLength(cell.value, out);
 +        }
 +    }
 +
 +    // For the old wire format
 +    // Note: this can return null if an empty partition is serialized!
 +    public static UnfilteredRowIterator deserializeLegacyPartition(DataInputPlus in, int version, SerializationHelper.Flag flag, ByteBuffer key) throws IOException
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        // This is only used in mutation, and mutation have never allowed "null" column families
 +        boolean present = in.readBoolean();
 +        if (!present)
 +            return null;
 +
 +        CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
 +        LegacyDeletionInfo info = LegacyDeletionInfo.deserialize(metadata, in);
 +        int size = in.readInt();
 +        Iterator<LegacyCell> cells = deserializeCells(metadata, in, flag, size);
 +        SerializationHelper helper = new SerializationHelper(metadata, version, flag);
 +        return onWireCellstoUnfilteredRowIterator(metadata, metadata.partitioner.decorateKey(key), info, cells, false, helper);
 +    }
 +
 +    // For the old wire format
 +    public static long serializedSizeAsLegacyPartition(ReadCommand command, UnfilteredRowIterator partition, int version)
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        if (partition.isEmpty())
 +            return TypeSizes.sizeof(false);
 +
 +        long size = TypeSizes.sizeof(true);
 +
 +        LegacyLayout.LegacyUnfilteredPartition legacyPartition = LegacyLayout.fromUnfilteredRowIterator(command, partition);
 +
 +        size += UUIDSerializer.serializer.serializedSize(partition.metadata().cfId, version);
 +        size += DeletionTime.serializer.serializedSize(legacyPartition.partitionDeletion);
 +        size += legacyPartition.rangeTombstones.serializedSize(partition.metadata());
 +
 +        // begin cell serialization
 +        size += TypeSizes.sizeof(legacyPartition.cells.size());
 +        for (LegacyLayout.LegacyCell cell : legacyPartition.cells)
 +        {
 +            size += ByteBufferUtil.serializedSizeWithShortLength(cell.name.encode(partition.metadata()));
 +            size += 1;  // serialization flags
 +            if (cell.isExpiring())
 +            {
 +                size += TypeSizes.sizeof(cell.ttl);
 +                size += TypeSizes.sizeof(cell.localDeletionTime);
 +            }
 +            else if (cell.isTombstone())
 +            {
 +                size += TypeSizes.sizeof(cell.timestamp);
 +                // localDeletionTime replaces cell.value as the body
 +                size += TypeSizes.sizeof(TypeSizes.sizeof(cell.localDeletionTime));
 +                size += TypeSizes.sizeof(cell.localDeletionTime);
 +                continue;
 +            }
 +            else if (cell.isCounterUpdate())
 +            {
 +                size += TypeSizes.sizeof(cell.timestamp);
 +                long count = CounterContext.instance().getLocalCount(cell.value);
 +                size += ByteBufferUtil.serializedSizeWithLength(ByteBufferUtil.bytes(count));
 +                continue;
 +            }
 +            else if (cell.isCounter())
 +            {
 +                size += TypeSizes.sizeof(Long.MIN_VALUE);  // timestampOfLastDelete
 +            }
 +
 +            size += TypeSizes.sizeof(cell.timestamp);
 +            size += ByteBufferUtil.serializedSizeWithLength(cell.value);
 +        }
 +
 +        return size;
 +    }
 +
 +    // For thrift sake
 +    public static UnfilteredRowIterator toUnfilteredRowIterator(CFMetaData metadata,
 +                                                                DecoratedKey key,
 +                                                                LegacyDeletionInfo delInfo,
 +                                                                Iterator<LegacyCell> cells)
 +    {
 +        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
 +        return toUnfilteredRowIterator(metadata, key, delInfo, cells, false, helper);
 +    }
 +
 +    // For deserializing old wire format
 +    public static UnfilteredRowIterator onWireCellstoUnfilteredRowIterator(CFMetaData metadata,
 +                                                                           DecoratedKey key,
 +                                                                           LegacyDeletionInfo delInfo,
 +                                                                           Iterator<LegacyCell> cells,
 +                                                                           boolean reversed,
 +                                                                           SerializationHelper helper)
 +    {
 +
 +        // If the table is a static compact, the "column_metadata" are now internally encoded as
 +        // static. This has already been recognized by decodeCellName, but it means the cells
 +        // provided are not in the expected order (the "static" cells are not necessarily at the front).
 +        // So sort them to make sure toUnfilteredRowIterator works as expected.
 +        // Further, if the query is reversed, then the on-wire format still has cells in non-reversed
 +        // order, but we need to have them reverse in the final UnfilteredRowIterator. So reverse them.
 +        if (metadata.isStaticCompactTable() || reversed)
 +        {
 +            List<LegacyCell> l = new ArrayList<>();
 +            Iterators.addAll(l, cells);
 +            Collections.sort(l, legacyCellComparator(metadata, reversed));
 +            cells = l.iterator();
 +        }
 +
 +        return toUnfilteredRowIterator(metadata, key, delInfo, cells, reversed, helper);
 +    }
 +
 +    private static UnfilteredRowIterator toUnfilteredRowIterator(CFMetaData metadata,
 +                                                                 DecoratedKey key,
 +                                                                 LegacyDeletionInfo delInfo,
 +                                                                 Iterator<LegacyCell> cells,
 +                                                                 boolean reversed,
 +                                                                 SerializationHelper helper)
 +    {
 +        // A reducer that basically does nothing, we know the 2 merged iterators can't have conflicting atoms (since we merge cells with range tombstones).
 +        MergeIterator.Reducer<LegacyAtom, LegacyAtom> reducer = new MergeIterator.Reducer<LegacyAtom, LegacyAtom>()
 +        {
 +            private LegacyAtom atom;
 +
 +            public void reduce(int idx, LegacyAtom current)
 +            {
 +                // We're merging cell with range tombstones, so we should always only have a single atom to reduce.
 +                assert atom == null;
 +                atom = current;
 +            }
 +
 +            protected LegacyAtom getReduced()
 +            {
 +                return atom;
 +            }
 +
 +            protected void onKeyChange()
 +            {
 +                atom = null;
 +            }
 +        };
 +        List<Iterator<LegacyAtom>> iterators = Arrays.asList(asLegacyAtomIterator(cells), asLegacyAtomIterator(delInfo.inRowRangeTombstones()));
 +        PeekingIterator<LegacyAtom> atoms = Iterators.peekingIterator(MergeIterator.get(iterators, legacyAtomComparator(metadata), reducer));
 +
 +        // Check if we have some static
 +        Row staticRow = atoms.hasNext() && atoms.peek().isStatic()
 +                      ? getNextRow(CellGrouper.staticGrouper(metadata, helper), atoms)
 +                      : Rows.EMPTY_STATIC_ROW;
 +
 +        Iterator<Row> rows = convertToRows(new CellGrouper(metadata, helper), atoms);
 +        Iterator<RangeTombstone> ranges = delInfo.deletionInfo.rangeIterator(reversed);
 +        return new RowAndDeletionMergeIterator(metadata,
 +                                               key,
 +                                               delInfo.deletionInfo.getPartitionDeletion(),
 +                                               ColumnFilter.all(metadata),
 +                                               staticRow,
 +                                               reversed,
 +                                               EncodingStats.NO_STATS,
 +                                               rows,
 +                                               ranges,
 +                                               true);
 +    }
 +
 +    public static Row extractStaticColumns(CFMetaData metadata, DataInputPlus in, Columns statics) throws IOException
 +    {
 +        assert !statics.isEmpty();
 +        assert metadata.isCompactTable();
 +
 +        if (metadata.isSuper())
 +            // TODO: there is in practice nothing to do here, but we need to handle the column_metadata for super columns somewhere else
 +            throw new UnsupportedOperationException();
 +
 +        Set<ByteBuffer> columnsToFetch = new HashSet<>(statics.size());
 +        for (ColumnDefinition column : statics)
 +            columnsToFetch.add(column.name.bytes);
 +
 +        Row.Builder builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
 +        builder.newRow(Clustering.STATIC_CLUSTERING);
 +
 +        boolean foundOne = false;
 +        LegacyAtom atom;
 +        while ((atom = readLegacyAtomSkippingUnknownColumn(metadata,in)) != null)
 +        {
 +            if (atom.isCell())
 +            {
 +                LegacyCell cell = atom.asCell();
 +                if (!columnsToFetch.contains(cell.name.encode(metadata)))
 +                    continue;
 +
 +                foundOne = true;
 +                builder.addCell(new BufferCell(cell.name.column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, null));
 +            }
 +            else
 +            {
 +                LegacyRangeTombstone tombstone = atom.asRangeTombstone();
 +                // TODO: we need to track tombstones and potentially ignore cells that are
 +                // shadowed (or even better, replace them by tombstones).
 +                throw new UnsupportedOperationException();
 +            }
 +        }
 +
 +        return foundOne ? builder.build() : Rows.EMPTY_STATIC_ROW;
 +    }
 +
 +    private static LegacyAtom readLegacyAtomSkippingUnknownColumn(CFMetaData metadata, DataInputPlus in)
 +    throws IOException
 +    {
 +        while (true)
 +        {
 +            try
 +            {
 +                return readLegacyAtom(metadata, in, false);
 +            }
 +            catch (UnknownColumnException e)
 +            {
 +                // Simply skip, as the method name implies.
 +            }
 +        }
 +
 +    }
 +
 +    private static Row getNextRow(CellGrouper grouper, PeekingIterator<? extends LegacyAtom> cells)
 +    {
 +        if (!cells.hasNext())
 +            return null;
 +
 +        grouper.reset();
 +        while (cells.hasNext() && grouper.addAtom(cells.peek()))
 +        {
 +            // We've added the cell already in the grouper, so just skip it
 +            cells.next();
 +        }
 +        return grouper.getRow();
 +    }
 +
 +    @SuppressWarnings("unchecked")
 +    private static Iterator<LegacyAtom> asLegacyAtomIterator(Iterator<? extends LegacyAtom> iter)
 +    {
 +        return (Iterator<LegacyAtom>)iter;
 +    }
 +
 +    private static Iterator<Row> convertToRows(final CellGrouper grouper, final PeekingIterator<LegacyAtom> atoms)
 +    {
 +        return new AbstractIterator<Row>()
 +        {
 +            protected Row computeNext()
 +            {
 +                if (!atoms.hasNext())
 +                    return endOfData();
 +
 +                return getNextRow(grouper, atoms);
 +            }
 +        };
 +    }
 +
 +    public static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRowIterator(final RowIterator iterator)
 +    {
 +        return fromRowIterator(iterator.metadata(), iterator, iterator.staticRow());
 +    }
 +
 +    private static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRowIterator(final CFMetaData metadata, final Iterator<Row> iterator, final Row staticRow)
 +    {
 +        LegacyRangeTombstoneList deletions = new LegacyRangeTombstoneList(new LegacyBoundComparator(metadata.comparator), 10);
 +        Iterator<LegacyCell> cells = new AbstractIterator<LegacyCell>()
 +        {
 +            private Iterator<LegacyCell> currentRow = initializeRow();
 +
 +            private Iterator<LegacyCell> initializeRow()
 +            {
 +                if (staticRow == null || staticRow.isEmpty())
 +                    return Collections.<LegacyLayout.LegacyCell>emptyIterator();
 +
 +                Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> row = fromRow(metadata, staticRow);
 +                deletions.addAll(row.left);
 +                return row.right;
 +            }
 +
 +            protected LegacyCell computeNext()
 +            {
 +                while (true)
 +                {
 +                    if (currentRow.hasNext())
 +                        return currentRow.next();
 +
 +                    if (!iterator.hasNext())
 +                        return endOfData();
 +
 +                    Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> row = fromRow(metadata, iterator.next());
 +                    deletions.addAll(row.left);
 +                    currentRow = row.right;
 +                }
 +            }
 +        };
 +
 +        return Pair.create(deletions, cells);
 +    }
 +
 +    private static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRow(final CFMetaData metadata, final Row row)
 +    {
 +        // convert any complex deletions or row deletion into normal range tombstones so that we can build and send a proper RangeTombstoneList
 +        // to legacy nodes
 +        LegacyRangeTombstoneList deletions = new LegacyRangeTombstoneList(new LegacyBoundComparator(metadata.comparator), 10);
 +
 +        if (!row.deletion().isLive())
 +        {
 +            Clustering clustering = row.clustering();
 +            Slice.Bound startBound = Slice.Bound.inclusiveStartOf(clustering);
 +            Slice.Bound endBound = Slice.Bound.inclusiveEndOf(clustering);
 +
 +            LegacyBound start = new LegacyLayout.LegacyBound(startBound, false, null);
 +            LegacyBound end = new LegacyLayout.LegacyBound(endBound, false, null);
 +
 +            deletions.add(start, end, row.deletion().time().markedForDeleteAt(), row.deletion().time().localDeletionTime());
 +        }
 +
 +        for (ColumnData cd : row)
 +        {
 +            ColumnDefinition col = cd.column();
 +            if (col.isSimple())
 +                continue;
 +
 +            DeletionTime delTime = ((ComplexColumnData)cd).complexDeletion();
 +            if (!delTime.isLive())
 +            {
 +                Clustering clustering = row.clustering();
 +
 +                Slice.Bound startBound = Slice.Bound.inclusiveStartOf(clustering);
 +                Slice.Bound endBound = Slice.Bound.inclusiveEndOf(clustering);
 +
 +                LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(startBound, col.isStatic(), col);
 +                LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(endBound, col.isStatic(), col);
 +
 +                deletions.add(start, end, delTime.markedForDeleteAt(), delTime.localDeletionTime());
 +            }
 +        }
 +
 +        Iterator<LegacyCell> cells = new AbstractIterator<LegacyCell>()
 +        {
 +            private final Iterator<Cell> cells = row.cellsInLegacyOrder(metadata, false).iterator();
 +            // we don't have (and shouldn't have) row markers for compact tables.
 +            private boolean hasReturnedRowMarker = metadata.isCompactTable();
 +
 +            protected LegacyCell computeNext()
 +            {
 +                if (!hasReturnedRowMarker)
 +                {
 +                    hasReturnedRowMarker = true;
 +
 +                    // don't include a row marker if there's no timestamp on the primary key; this is the 3.0+ equivalent
 +                    // of a row marker
 +                    if (!row.primaryKeyLivenessInfo().isEmpty())
 +                    {
 +                        LegacyCellName cellName = new LegacyCellName(row.clustering(), null, null);
 +                        LivenessInfo info = row.primaryKeyLivenessInfo();
 +                        return new LegacyCell(info.isExpiring() ? LegacyCell.Kind.EXPIRING : LegacyCell.Kind.REGULAR, cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER, info.timestamp(), info.localExpirationTime(), info.ttl());
 +                    }
 +                }
 +
 +                if (!cells.hasNext())
 +                    return endOfData();
 +
 +                return makeLegacyCell(row.clustering(), cells.next());
 +            }
 +        };
 +        return Pair.create(deletions, cells);
 +    }
 +
 +    private static LegacyCell makeLegacyCell(Clustering clustering, Cell cell)
 +    {
 +        LegacyCell.Kind kind;
 +        if (cell.isCounterCell())
 +            kind = LegacyCell.Kind.COUNTER;
 +        else if (cell.isTombstone())
 +            kind = LegacyCell.Kind.DELETED;
 +        else if (cell.isExpiring())
 +            kind = LegacyCell.Kind.EXPIRING;
 +        else
 +            kind = LegacyCell.Kind.REGULAR;
 +
 +        CellPath path = cell.path();
 +        assert path == null || path.size() == 1;
 +        LegacyCellName name = new LegacyCellName(clustering, cell.column(), path == null ? null : path.get(0));
 +        return new LegacyCell(kind, name, cell.value(), cell.timestamp(), cell.localDeletionTime(), cell.ttl());
 +    }
 +
 +    public static RowIterator toRowIterator(final CFMetaData metadata,
 +                                            final DecoratedKey key,
 +                                            final Iterator<LegacyCell> cells,
 +                                            final int nowInSec)
 +    {
 +        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
 +        return UnfilteredRowIterators.filter(toUnfilteredRowIterator(metadata, key, LegacyDeletionInfo.live(), cells, false, helper), nowInSec);
 +    }
 +
 +    public static Comparator<LegacyCell> legacyCellComparator(CFMetaData metadata)
 +    {
 +        return legacyCellComparator(metadata, false);
 +    }
 +
 +    public static Comparator<LegacyCell> legacyCellComparator(final CFMetaData metadata, final boolean reversed)
 +    {
 +        final Comparator<LegacyCellName> cellNameComparator = legacyCellNameComparator(metadata, reversed);
 +        return new Comparator<LegacyCell>()
 +        {
 +            public int compare(LegacyCell cell1, LegacyCell cell2)
 +            {
 +                LegacyCellName c1 = cell1.name;
 +                LegacyCellName c2 = cell2.name;
 +
 +                int c = cellNameComparator.compare(c1, c2);
 +                if (c != 0)
 +                    return c;
 +
 +                // The actual sorting when the cellname is equal doesn't matter, we just want to make
 +                // sure the cells are not considered equal.
 +                if (cell1.timestamp != cell2.timestamp)
 +                    return cell1.timestamp < cell2.timestamp ? -1 : 1;
 +
 +                if (cell1.localDeletionTime != cell2.localDeletionTime)
 +                    return cell1.localDeletionTime < cell2.localDeletionTime ? -1 : 1;
 +
 +                return cell1.value.compareTo(cell2.value);
 +            }
 +        };
 +    }
 +
 +    // Note that this doesn't exactly compare cells as they were pre-3.0 because within a row they sort columns like
 +    // in 3.0, that is, with simple columns before complex columns. In other words, this comparator makes sure cells
 +    // are in the proper order to convert them to actual 3.0 rows.
 +    public static Comparator<LegacyCellName> legacyCellNameComparator(final CFMetaData metadata, final boolean reversed)
 +    {
 +        return new Comparator<LegacyCellName>()
 +        {
 +            public int compare(LegacyCellName c1, LegacyCellName c2)
 +            {
 +                // Compare clustering first
 +                if (c1.clustering == Clustering.STATIC_CLUSTERING)
 +                {
 +                    if (c2.clustering != Clustering.STATIC_CLUSTERING)
 +                        return -1;
 +                }
 +                else if (c2.clustering == Clustering.STATIC_CLUSTERING)
 +                {
 +                    return 1;
 +                }
 +                else
 +                {
 +                    int c = metadata.comparator.compare(c1.clustering, c2.clustering);
 +                    if (c != 0)
 +                        return reversed ? -c : c;
 +                }
 +
 +                // Note that when reversed, we only care about the clustering being reversed, so it's ok
 +                // not to take reversed into account below.
 +
 +                // Then check the column name
 +                if (c1.column != c2.column)
 +                {
 +                    // A null for the column means it's a row marker
 +                    if (c1.column == null)
 +                        return -1;
 +                    if (c2.column == null)
 +                        return 1;
 +
 +                    assert c1.column.isRegular() || c1.column.isStatic();
 +                    assert c2.column.isRegular() || c2.column.isStatic();
 +                    int cmp = c1.column.compareTo(c2.column);
 +                    if (cmp != 0)
 +                        return cmp;
 +                }
 +
 +                assert (c1.collectionElement == null) == (c2.collectionElement == null);
 +
 +                if (c1.collectionElement != null)
 +                {
 +                    AbstractType<?> colCmp = ((CollectionType)c1.column.type).nameComparator();
 +                    return colCmp.compare(c1.collectionElement, c2.collectionElement);
 +                }
 +                return 0;
 +            }
 +        };
 +    }
 +
 +    private static boolean equalValues(ClusteringPrefix c1, ClusteringPrefix c2, ClusteringComparator comparator)
 +    {
 +        assert c1.size() == c2.size();
 +        for (int i = 0; i < c1.size(); i++)
 +        {
 +            if (comparator.compareComponent(i, c1.get(i), c2.get(i)) != 0)
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    private static Comparator<LegacyAtom> legacyAtomComparator(CFMetaData metadata)
 +    {
 +        return (o1, o2) ->
 +        {
 +            // First we want to compare by clustering, but we have to be careful with range tombstone, because
 +            // we can have collection deletion and we want those to sort properly just before the column they
 +            // delete, not before the whole row.
 +            // We also want to special case static so they sort before any non-static. Note in particular that
 +            // this special casing is important in the case of one of the Atom being Slice.Bound.BOTTOM: we want
 +            // it to sort after the static as we deal with static first in toUnfilteredAtomIterator and having
 +            // Slice.Bound.BOTTOM first would mess that up (note that static deletion is handled through a specific
 +            // static tombstone, see LegacyDeletionInfo.add()).
 +            if (o1.isStatic() != o2.isStatic())
 +                return o1.isStatic() ? -1 : 1;
 +
 +            ClusteringPrefix c1 = o1.clustering();
 +            ClusteringPrefix c2 = o2.clustering();
 +
 +            int clusteringComparison;
 +            if (c1.size() != c2.size() || (o1.isCell() == o2.isCell()) || !equalValues(c1, c2, metadata.comparator))
 +            {
 +                clusteringComparison = metadata.comparator.compare(c1, c2);
 +            }
 +            else
 +            {
 +                // one is a cell and one is a range tombstone, and both have the same prefix size (that is, the
 +                // range tombstone is either a row deletion or a collection deletion).
 +                LegacyRangeTombstone rt = o1.isCell() ? o2.asRangeTombstone() : o1.asRangeTombstone();
 +                clusteringComparison = rt.isCollectionTombstone()
 +                                       ? 0
 +                                       : metadata.comparator.compare(c1, c2);
 +            }
 +
 +            // Note that if both are range tombstones and have the same clustering, then they are equal.
 +            if (clusteringComparison != 0)
 +                return clusteringComparison;
 +
 +            if (o1.isCell())
 +            {
 +                LegacyCell cell1 = o1.asCell();
 +                if (o2.isCell())
 +                {
 +                    LegacyCell cell2 = o2.asCell();
 +                    // Check for row marker cells
 +                    if (cell1.name.column == null)
 +                        return cell2.name.column == null ? 0 : -1;
 +                    return cell2.name.column == null ? 1 : cell1.name.column.compareTo(cell2.name.column);
 +                }
 +
 +                LegacyRangeTombstone rt2 = o2.asRangeTombstone();
 +                assert rt2.isCollectionTombstone(); // otherwise, we shouldn't have got a clustering equality
 +                if (cell1.name.column == null)
 +                    return -1;
 +                int cmp = cell1.name.column.compareTo(rt2.start.collectionName);
 +                // If both are for the same column, then the RT should come first
 +                return cmp == 0 ? 1 : cmp;
 +            }
 +            else
 +            {
 +                assert o2.isCell();
 +                LegacyCell cell2 = o2.asCell();
 +
 +                LegacyRangeTombstone rt1 = o1.asRangeTombstone();
 +                assert rt1.isCollectionTombstone(); // otherwise, we shouldn't have got a clustering equality
 +
 +                if (cell2.name.column == null)
 +                    return 1;
 +
 +                int cmp = rt1.start.collectionName.compareTo(cell2.name.column);
 +                // If both are for the same column, then the RT should come first
 +                return cmp == 0 ? -1 : cmp;
 +            }
 +        };
 +    }
 +
 +    public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, boolean readAllAsDynamic)
 +    throws IOException, UnknownColumnException
 +    {
 +        ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
 +        if (!cellname.hasRemaining())
 +            return null; // END_OF_ROW
 +
 +        try
 +        {
 +            int b = in.readUnsignedByte();
 +            return (b & RANGE_TOMBSTONE_MASK) != 0
 +                   ? readLegacyRangeTombstoneBody(metadata, in, cellname)
 +                   : readLegacyCellBody(metadata, in, cellname, b, SerializationHelper.Flag.LOCAL, readAllAsDynamic);
 +        }
 +        catch (UnknownColumnException e)
 +        {
 +            // We legitimately can get here in 2 cases:
 +            // 1) for system tables, because we've unceremoniously removed columns (without registering them as dropped)
 +            // 2) for dropped columns.
 +            // In any other case, there is a mismatch between the schema and the data, and we complain loudly in
 +            // that case. Note that if we are in a legit case of an unknown column, we want to simply skip that cell,
 +            // but we don't do this here and re-throw the exception because the calling code sometimes has to know
 +            // about this happening. This does mean code calling this method should handle this case properly.
 +            if (!metadata.ksName.equals(SystemKeyspace.NAME) && metadata.getDroppedColumnDefinition(e.columnName) == null)
 +                throw new IllegalStateException(String.format("Got cell for unknown column %s in sstable of %s.%s: " +
 +                                                              "This suggest a problem with the schema which doesn't list " +
 +                                                              "this column. Even if that column was dropped, it should have " +
 +                                                              "been listed as such", metadata.ksName, metadata.cfName, UTF8Type.instance.compose(e.columnName)), e);
 +
 +            throw e;
 +        }
 +    }
 +
 +    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException
 +    {
 +        ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
 +        int b = in.readUnsignedByte();
 +        return readLegacyCellBody(metadata, in, cellname, b, flag, false);
 +    }
 +
 +    public static LegacyCell readLegacyCellBody(CFMetaData metadata, DataInput in, ByteBuffer cellname, int mask, SerializationHelper.Flag flag, boolean readAllAsDynamic)
 +    throws IOException, UnknownColumnException
 +    {
 +        // Note that we want to call decodeCellName only after we've deserialized other parts, since it can throw
 +        // and we want to throw only after having deserialized the full cell.
 +        if ((mask & COUNTER_MASK) != 0)
 +        {
 +            in.readLong(); // timestampOfLastDelete: this has been unused for a long time so we ignore it
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            if (flag == SerializationHelper.Flag.FROM_REMOTE || (flag == SerializationHelper.Flag.LOCAL && CounterContext.instance().shouldClearLocal(value)))
 +                value = CounterContext.instance().clearAllLocal(value);
 +            return new LegacyCell(LegacyCell.Kind.COUNTER, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +        else if ((mask & EXPIRATION_MASK) != 0)
 +        {
 +            int ttl = in.readInt();
 +            int expiration = in.readInt();
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            return new LegacyCell(LegacyCell.Kind.EXPIRING, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, expiration, ttl);
 +        }
 +        else
 +        {
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            LegacyCellName name = decodeCellName(metadata, cellname, readAllAsDynamic);
 +            return (mask & COUNTER_UPDATE_MASK) != 0
 +                ? new LegacyCell(LegacyCell.Kind.COUNTER, name, CounterContext.instance().createUpdate(ByteBufferUtil.toLong(value)), ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
 +                : ((mask & DELETION_MASK) == 0
 +                        ? new LegacyCell(LegacyCell.Kind.REGULAR, name, value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
 +                        : new LegacyCell(LegacyCell.Kind.DELETED, name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ts, ByteBufferUtil.toInt(value), Cell.NO_TTL));
 +        }
 +    }
 +
 +    public static LegacyRangeTombstone readLegacyRangeTombstoneBody(CFMetaData metadata, DataInputPlus in, ByteBuffer boundname) throws IOException
 +    {
 +        LegacyBound min = decodeBound(metadata, boundname, true);
 +        LegacyBound max = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
 +        DeletionTime dt = DeletionTime.serializer.deserialize(in);
 +        return new LegacyRangeTombstone(min, max, dt);
 +    }
 +
 +    public static Iterator<LegacyCell> deserializeCells(final CFMetaData metadata,
 +                                                        final DataInput in,
 +                                                        final SerializationHelper.Flag flag,
 +                                                        final int size)
 +    {
 +        return new AbstractIterator<LegacyCell>()
 +        {
 +            private int i = 0;
 +
 +            protected LegacyCell computeNext()
 +            {
 +                if (i >= size)
 +                    return endOfData();
 +
 +                ++i;
 +                try
 +                {
 +                    return readLegacyCell(metadata, in, flag);
 +                }
 +                catch (UnknownColumnException e)
 +                {
 +                    // We can get there if we read a cell for a dropped column, and if that is the case,
 +                    // then simply ignore the cell is fine. But also not that we ignore if it's the
 +                    // system keyspace because for those table we actually remove columns without registering
 +                    // them in the dropped columns
 +                    if (metadata.ksName.equals(SystemKeyspace.NAME) || metadata.getDroppedColumnDefinition(e.columnName) != null)
 +                        return computeNext();
 +                    else
 +                        throw new IOError(e);
 +                }
 +                catch (IOException e)
 +                {
 +                    throw new IOError(e);
 +                }
 +            }
 +        };
 +    }
 +
 +    public static class CellGrouper
 +    {
 +        /**
 +         * The fake TTL used for expired rows that have been compacted.
 +         */
 +        private static final int FAKE_TTL = 1;
 +
 +        public final CFMetaData metadata;
 +        private final boolean isStatic;
 +        private final SerializationHelper helper;
 +        private final Row.Builder builder;
 +        private Clustering clustering;
 +
 +        private LegacyRangeTombstone rowDeletion;
 +        private LegacyRangeTombstone collectionDeletion;
 +
 +        public CellGrouper(CFMetaData metadata, SerializationHelper helper)
 +        {
 +            this(metadata, helper, false);
 +        }
 +
 +        private CellGrouper(CFMetaData metadata, SerializationHelper helper, boolean isStatic)
 +        {
 +            this.metadata = metadata;
 +            this.isStatic = isStatic;
 +            this.helper = helper;
 +            // We cannot use a sorted builder because we don't have exactly the same ordering in 3.0 and pre-3.0. More precisely, within a row, we
 +            // store all simple columns before the complex ones in 3.0, which we use to sort everything sorted by the column name before. Note however
 +            // that the unsorted builder won't have to reconcile cells, so the exact value we pass for nowInSec doesn't matter.
 +            this.builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
 +        }
 +
 +        public static CellGrouper staticGrouper(CFMetaData metadata, SerializationHelper helper)
 +        {
 +            return new CellGrouper(metadata, helper, true);
 +        }
 +
 +        public void reset()
 +        {
 +            this.clustering = null;
 +            this.rowDeletion = null;
 +            this.collectionDeletion = null;
 +        }
 +
 +        public boolean addAtom(LegacyAtom atom)
 +        {
 +            return atom.isCell()
 +                 ? addCell(atom.asCell())
 +                 : addRangeTombstone(atom.asRangeTombstone());
 +        }
 +
 +        public boolean addCell(LegacyCell cell)
 +        {
 +            if (clustering == null)
 +            {
 +                clustering = cell.name.clustering;
 +                assert !isStatic || clustering == Clustering.STATIC_CLUSTERING;
 +                builder.newRow(clustering);
 +            }
 +            else if (!clustering.equals(cell.name.clustering))
 +            {
 +                return false;
 +            }
 +
 +            // Ignore shadowed cells
 +            if (rowDeletion != null && rowDeletion.deletionTime.deletes(cell.timestamp))
 +                return true;
 +
 +            ColumnDefinition column = cell.name.column;
 +            if (column == null)
 +            {
 +                // It's the row marker
 +                assert !cell.value.hasRemaining();
 +                // In 2.1, the row marker expired cell might have been converted into a deleted one by compaction.
 +                // If we do not set the primary key liveness info for this row and it does not contains any regular columns
 +                // the row will be empty. To avoid that, we reuse the localDeletionTime but use a fake TTL.
 +                // The only time in 2.x that we actually delete a row marker is in 2i tables, so in that case we do
 +                // want to actually propagate the row deletion. (CASSANDRA-13320)
 +                if (!cell.isTombstone())
 +                    builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, cell.ttl, cell.localDeletionTime));
 +                else if (metadata.isIndex())
 +                    builder.addRowDeletion(Row.Deletion.regular(new DeletionTime(cell.timestamp, cell.localDeletionTime)));
 +                else
 +                    builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, FAKE_TTL, cell.localDeletionTime));
 +            }
 +            else
 +            {
 +                if (collectionDeletion != null && collectionDeletion.start.collectionName.name.equals(column.name) && collectionDeletion.deletionTime.deletes(cell.timestamp))
 +                    return true;
 +
 +                if (helper.includes(column))
 +                {
 +                    CellPath path = null;
 +                    if (column.isComplex())
 +                    {
 +                        // Recalling startOfComplexColumn for every cell is a big inefficient, but it's ok in practice
 +                        // and it's simpler. And since 1) this only matter for super column selection in thrift in
 +                        // practice and 2) is only used during upgrade, it's probably worth keeping things simple.
 +                        helper.startOfComplexColumn(column);
 +                        path = cell.name.collectionElement == null ? null : CellPath.create(cell.name.collectionElement);
 +                        if (!helper.includes(path))
 +                            return true;
 +                    }
 +                    Cell c = new BufferCell(column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, path);
 +                    if (!helper.isDropped(c, column.isComplex()))
 +                        builder.addCell(c);
 +                    if (column.isComplex())
 +                    {
 +                        helper.endOfComplexColumn();
 +                    }
 +                }
 +            }
 +            return true;
 +        }
 +
 +        private boolean addRangeTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (tombstone.isRowDeletion(metadata))
 +                return addRowTombstone(tombstone);
 +            else if (tombstone.isCollectionTombstone())
 +                return addCollectionTombstone(tombstone);
 +            else
 +                return addGenericRangeTombstone(tombstone);
 +        }
 +
 +        private boolean addRowTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (clustering != null)
 +            {
 +                // If we're already in the row, there might be a chance that there were two range tombstones
 +                // written, as 2.x storage format does not guarantee just one range tombstone, unlike 3.x.
 +                // We have to make sure that clustering matches, which would mean that tombstone is for the
 +                // same row.
 +                if (rowDeletion != null && clustering.equals(tombstone.start.getAsClustering(metadata)))
 +                {
 +                    // If the tombstone superceeds the previous delete, we discard the previous one
 +                    if (tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
 +                    {
 +                        builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
 +                        rowDeletion = tombstone;
 +                    }
 +                    return true;
 +                }
 +
 +                // If we're already within a row and there was no delete written before that one, it can't be the same one
 +                return false;
 +            }
 +
 +            clustering = tombstone.start.getAsClustering(metadata);
 +            builder.newRow(clustering);
 +            builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
 +            rowDeletion = tombstone;
 +
 +            return true;
 +        }
 +
 +        private boolean addCollectionTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (!helper.includes(tombstone.start.collectionName))
 +                return false; // see CASSANDRA-13109
 +
 +            if (clustering == null)
 +            {
 +                clustering = tombstone.start.getAsClustering(metadata);
 +                builder.newRow(clustering);
 +            }
 +            else if (!clustering.equals(tombstone.start.getAsClustering(metadata)))
 +            {
 +                return false;
 +            }
 +
 +            builder.addComplexDeletion(tombstone.start.collectionName, tombstone.deletionTime);
 +            if (rowDeletion == null || tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
 +                collectionDeletion = tombstone;
 +
 +            return true;
 +        }
 +
 +        private boolean addGenericRangeTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            /*
 +             * We can see a non-collection, non-row deletion in two scenarios:
 +             *
 +             * 1. Most commonly, the tombstone's start bound is bigger than current row's clustering, which means that
 +             *    the current row is over, and we should move on to the next row or RT;
 +             *
 +             * 2. Less commonly, the tombstone's start bound is smaller than current row's clustering, which means that
 +             *    we've crossed an index boundary and are seeing a non-closed RT from the previous block, repeated;
 +             *    we should ignore it and stay in the current row.
 +             *
 +             *  In either case, clustering should be non-null, or we shouldn't have gotten to this method at all
 +             *  However, to be absolutely SURE we're in case two above, we check here.
 +             */
 +            return clustering != null && metadata.comparator.compare(clustering, tombstone.start.bound.clustering()) > 0;
 +        }
 +
 +        public Row getRow()
 +        {
 +            return builder.build();
 +        }
 +    }
 +
 +    public static class LegacyUnfilteredPartition
 +    {
 +        public final DeletionTime partitionDeletion;
 +        public final LegacyRangeTombstoneList rangeTombstones;
 +        public final List<LegacyCell> cells;
 +
 +        private LegacyUnfilteredPartition(DeletionTime partitionDeletion, LegacyRangeTombstoneList rangeTombstones, List<LegacyCell> cells)
 +        {
 +            this.partitionDeletion = partitionDeletion;
 +            this.rangeTombstones = rangeTombstones;
 +            this.cells = cells;
 +        }
 +
 +        public void digest(CFMetaData metadata, MessageDigest digest)
 +        {
 +            for (LegacyCell cell : cells)
 +            {
 +                digest.update(cell.name.encode(metadata).duplicate());
 +
 +                if (cell.isCounter())
 +                    CounterContext.instance().updateDigest(digest, cell.value);
 +                else
 +                    digest.update(cell.value.duplicate());
 +
 +                FBUtilities.updateWithLong(digest, cell.timestamp);
 +                FBUtilities.updateWithByte(digest, cell.serializationFlags());
 +
 +                if (cell.isExpiring())
 +                    FBUtilities.updateWithInt(digest, cell.ttl);
 +
 +                if (cell.isCounter())
 +                {
 +                    // Counters used to have the timestampOfLastDelete field, which we stopped using long ago and has been hard-coded
 +                    // to Long.MIN_VALUE but was still taken into account in 2.2 counter digests (to maintain backward compatibility
 +                    // in the first place).
 +                    FBUtilities.updateWithLong(digest, Long.MIN_VALUE);
 +                }
 +            }
 +
 +            if (partitionDeletion.markedForDeleteAt() != Long.MIN_VALUE)
 +                digest.update(ByteBufferUtil.bytes(partitionDeletion.markedForDeleteAt()));
 +
 +            if (!rangeTombstones.isEmpty())
 +                rangeTombstones.updateDigest(digest);
 +        }
 +    }
 +
 +    public static class LegacyCellName
 +    {
 +        public final Clustering clustering;
 +        public final ColumnDefinition column;
 +        public final ByteBuffer collectionElement;
 +
 +        private LegacyCellName(Clustering clustering, ColumnDefinition column, ByteBuffer collectionElement)
 +        {
 +            this.clustering = clustering;
 +            this.column = column;
 +            this.collectionElement = collectionElement;
 +        }
 +
 +        public ByteBuffer encode(CFMetaData metadata)
 +        {
 +            return encodeCellName(metadata, clustering, column == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : column.name.bytes, collectionElement);
 +        }
 +
 +        public ByteBuffer superColumnSubName()
 +        {
 +            assert collectionElement != null;
 +            return collectionElement;
 +        }
 +
 +        public ByteBuffer superColumnName()
 +        {
 +            return clustering.get(0);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            for (int i = 0; i < clustering.size(); i++)
 +                sb.append(i > 0 ? ":" : "").append(clustering.get(i) == null ? "null" : ByteBufferUtil.bytesToHex(clustering.get(i)));
 +            return String.format("Cellname(clustering=%s, column=%s, collElt=%s)", sb.toString(), column == null ? "null" : column.name, collectionElement == null ? "null" : ByteBufferUtil.bytesToHex(collectionElement));
 +        }
 +    }
 +
 +    public static class LegacyBound
 +    {
 +        public static final LegacyBound BOTTOM = new LegacyBound(Slice.Bound.BOTTOM, false, null);
 +        public static final LegacyBound TOP = new LegacyBound(Slice.Bound.TOP, false, null);
 +
 +        public final Slice.Bound bound;
 +        public final boolean isStatic;
 +        public final ColumnDefinition collectionName;
 +
 +        public LegacyBound(Slice.Bound bound, boolean isStatic, ColumnDefinition collectionName)
 +        {
 +            this.bound = bound;
 +            this.isStatic = isStatic;
 +            this.collectionName = collectionName;
 +        }
 +
 +        public Clustering getAsClustering(CFMetaData metadata)
 +        {
 +            if (isStatic)
 +                return Clustering.STATIC_CLUSTERING;
 +
 +            assert bound.size() == metadata.comparator.size();
 +            ByteBuffer[] values = new ByteBuffer[bound.size()];
 +            for (int i = 0; i < bound.size(); i++)
 +                values[i] = bound.get(i);
 +            return new Clustering(values);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            sb.append(bound.kind()).append('(');
 +            for (int i = 0; i < bound.size(); i++)
 +                sb.append(i > 0 ? ":" : "").append(bound.get(i) == null ? "null" : ByteBufferUtil.bytesToHex(bound.get(i)));
 +            sb.append(')');
 +            return String.format("Bound(%s, collection=%s)", sb.toString(), collectionName == null ? "null" : collectionName.name);
 +        }
 +    }
 +
 +    public interface LegacyAtom
 +    {
 +        public boolean isCell();
 +
 +        public ClusteringPrefix clustering();
 +        public boolean isStatic();
 +
 +        public LegacyCell asCell();
 +        public LegacyRangeTombstone asRangeTombstone();
 +    }
 +
 +    /**
 +     * A legacy cell.
 +     * <p>
 +     * This is used as a temporary object to facilitate dealing with the legacy format, this
 +     * is not meant to be optimal.
 +     */
 +    public static class LegacyCell implements LegacyAtom
 +    {
 +        private final static int DELETION_MASK        = 0x01;
 +        private final static int EXPIRATION_MASK      = 0x02;
 +        private final static int COUNTER_MASK         = 0x04;
 +        private final static int COUNTER_UPDATE_MASK  = 0x08;
 +        private final static int RANGE_TOMBSTONE_MASK = 0x10;
 +
 +        public enum Kind { REGULAR, EXPIRING, DELETED, COUNTER }
 +
 +        public final Kind kind;
 +
 +        public final LegacyCellName name;
 +        public final ByteBuffer value;
 +
 +        public final long timestamp;
 +        public final int localDeletionTime;
 +        public final int ttl;
 +
 +        private LegacyCell(Kind kind, LegacyCellName name, ByteBuffer value, long timestamp, int localDeletionTime, int ttl)
 +        {
 +            this.kind = kind;
 +            this.name = name;
 +            this.value = value;
 +            this.timestamp = timestamp;
 +            this.localDeletionTime = localDeletionTime;
 +            this.ttl = ttl;
 +        }
 +
 +        public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp)
 +        throws UnknownColumnException
 +        {
 +            return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +
 +        public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp, int ttl, int nowInSec)
 +        throws UnknownColumnException
 +        {
-             return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, nowInSec + ttl, ttl);
++            /*
++             * CASSANDRA-14092: Max expiration date capping is maybe performed here, expiration overflow policy application
++             * is done at {@link org.apache.cassandra.thrift.ThriftValidation#validateTtl(CFMetaData, Column)}
++             */
++            return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), ttl);
 +        }
 +
 +        public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
 +        throws UnknownColumnException
 +        {
 +            return new LegacyCell(Kind.DELETED, decodeCellName(metadata, superColumnName, name), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, nowInSec, LivenessInfo.NO_TTL);
 +        }
 +
 +        public static LegacyCell counterUpdate(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long value)
 +        throws UnknownColumnException
 +        {
 +            // See UpdateParameters.addCounter() for more details on this
 +            ByteBuffer counterValue = CounterContext.instance().createUpdate(value);
 +            return counter(decodeCellName(metadata, superColumnName, name), counterValue);
 +        }
 +
 +        public static LegacyCell counter(LegacyCellName name, ByteBuffer value)
 +        {
 +            return new LegacyCell(Kind.COUNTER, name, value, FBUtilities.timestampMicros(), Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +
 +        public byte serializationFlags()
 +        {
 +            if (isExpiring())
 +                return EXPIRATION_MASK;
 +            if (isTombstone())
 +                return DELETION_MASK;
 +            if (isCounterUpdate())
 +                return COUNTER_UPDATE_MASK;
 +            if (isCounter())
 +                return COUNTER_MASK;
 +            return 0;
 +        }
 +
 +        public boolean isCounterUpdate()
 +        {
 +            // See UpdateParameters.addCounter() for more details on this
 +            return isCounter() && CounterContext.instance().isUpdate(value);
 +        }
 +
 +        public ClusteringPrefix clustering()
 +        {
 +            return name.clustering;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return name.clustering == Clustering.STATIC_CLUSTERING;
 +        }
 +
 +        public boolean isCell()
 +        {
 +            return true;
 +        }
 +
 +        public LegacyCell asCell()
 +        {
 +            return this;
 +        }
 +
 +        public LegacyRangeTombstone asRangeTombstone()
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        public boolean isCounter()
 +        {
 +            return kind == Kind.COUNTER;
 +        }
 +
 +        public boolean isExpiring()
 +        {
 +            return kind == Kind.EXPIRING;
 +        }
 +
 +        public boolean isTombstone()
 +        {
 +            return kind == Kind.DELETED;
 +        }
 +
 +        public boolean isLive(int nowInSec)
 +        {
 +            if (isTombstone())
 +                return false;
 +
 +            return !isExpiring() || nowInSec < localDeletionTime;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("LegacyCell(%s, name=%s, v=%s, ts=%s, ldt=%s, ttl=%s)", kind, name, ByteBufferUtil.bytesToHex(value), timestamp, localDeletionTime, ttl);
 +        }
 +    }
 +
 +    /**
 +     * A legacy range tombstone.
 +     * <p>
 +     * This is used as a temporary object to facilitate dealing with the legacy format, this
 +     * is not meant to be optimal.
 +     */
 +    public static class LegacyRangeTombstone implements LegacyAtom
 +    {
 +        public final LegacyBound start;
 +        public final LegacyBound stop;
 +        public final DeletionTime deletionTime;
 +
 +        public LegacyRangeTombstone(LegacyBound start, LegacyBound stop, DeletionTime deletionTime)
 +        {
 +            // Because of the way RangeTombstoneList work, we can have a tombstone where only one of
 +            // the bound has a collectionName. That happens if we have a big tombstone A (spanning one
 +            // or multiple rows) and a collection tombstone B. In that case, RangeTombstoneList will
 +            // split this into 3 RTs: the first one from the beginning of A to the beginning of B,
 +            // then B, then a third one from the end of B to the end of A. To make this simpler, if
 +            // we detect that case we transform the 1st and 3rd tombstone so they don't end in the middle
 +            // of a row (which is still correct).
 +            if ((start.collectionName == null) != (stop.collectionName == null))
 +            {
 +                if (start.collectionName == null)
 +                    stop = new LegacyBound(stop.bound, stop.isStatic, null);
 +                else
 +                    start = new LegacyBound(start.bound, start.isStatic, null);
 +            }
 +            else if (!Objects.equals(start.collectionName, stop.collectionName))
 +            {
 +                // We're in the similar but slightly more complex case where on top of the big tombstone
 +                // A, we have 2 (or more) collection tombstones B and C within A. So we also end up with
 +                // a tombstone that goes between the end of B and the start of C.
 +                start = new LegacyBound(start.bound, start.isStatic, null);
 +                stop = new LegacyBound(stop.bound, stop.isStatic, null);
 +            }
 +
 +            this.start = start;
 +            this.stop = stop;
 +            this.deletionTime = deletionTime;
 +        }
 +
 +        public ClusteringPrefix clustering()
 +        {
 +            return start.bound;
 +        }
 +
 +        public LegacyRangeTombstone withNewStart(LegacyBound newStart)
 +        {
 +            return new LegacyRangeTombstone(newStart, stop, deletionTime);
 +        }
 +
 +        public LegacyRangeTombstone withNewEnd(LegacyBound newStop)
 +        {
 +            return new LegacyRangeTombstone(start, newStop, deletionTime);
 +        }
 +
 +        public boolean isCell()
 +        {
 +            return false;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return start.isStatic || stop.isStatic;
 +        }
 +
 +        public LegacyCell asCell()
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        public LegacyRangeTombstone asRangeTombstone()
 +        {
 +            return this;
 +        }
 +
 +        public boolean isCollectionTombstone()
 +        {
 +            return start.collectionName != null;
 +        }
 +
 +        public boolean isRowDeletion(CFMetaData metadata)
 +        {
 +            if (start.collectionName != null
 +                || stop.collectionName != null
 +                || start.bound.size() != metadata.comparator.size()
 +                || stop.bound.size() != metadata.comparator.size())
 +                return false;
 +
 +            for (int i = 0; i < start.bound.size(); i++)
 +                if (!Objects.equals(start.bound.get(i), stop.bound.get(i)))
 +                    return false;
 +            return true;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("RT(%s-%s, %s)", start, stop, deletionTime);
 +        }
 +    }
 +
 +    public static class LegacyDeletionInfo
 +    {
 +        public final MutableDeletionInfo deletionInfo;
 +        public final List<LegacyRangeTombstone> inRowTombstones = new ArrayList<>();
 +
 +        private LegacyDeletionInfo(MutableDeletionInfo deletionInfo)
 +        {
 +            this.deletionInfo = deletionInfo;
 +        }
 +
 +        public static LegacyDeletionInfo live()
 +        {
 +            return new LegacyDeletionInfo(MutableDeletionInfo.live());
 +        }
 +
 +        public void add(DeletionTime topLevel)
 +        {
 +            deletionInfo.add(topLevel);
 +        }
 +
 +        private static Slice.Bound staticBound(CFMetaData metadata, boolean isStart)
 +        {
 +            // In pre-3.0 nodes, static row started by a clustering with all empty values so we
 +            // preserve that here. Note that in practice, it doesn't really matter since the rest
 +            // of the code will ignore the bound for RT that have their static flag set.
 +            ByteBuffer[] values = new ByteBuffer[metadata.comparator.size()];
 +            for (int i = 0; i < values.length; i++)
 +                values[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +            return isStart
 +                 ? Slice.Bound.inclusiveStartOf(values)
 +                 : Slice.Bound.inclusiveEndOf(values);
 +        }
 +
 +        public void add(CFMetaData metadata, LegacyRangeTombstone tombstone)
 +        {
 +            if (metadata.hasStaticColumns())
 +            {
 +                /*
 +                 * For table having static columns we have to deal with the following cases:
 +                 *  1. the end of the tombstone is static (in which case either the start is static or is BOTTOM, which is the same
 +                 *     for our consideration). This mean that either the range only delete the static row, or that it's a collection
 +                 *     tombstone of a static collection. In both case, we just add the tombstone to the inRowTombstones.
 +                 *  2. only the start is static. There is then 2 subcase: either the start is inclusive, and that mean we include the
 +                 *     static row and more (so we add an inRowTombstone for the static and deal with the rest normally). Or the start
 +                 *     is exclusive, and that means we explicitely exclude the static (in which case we can just add the tombstone
 +                 *     as if it started at BOTTOM).
 +                 *  3. none of the bound are static but the start is BOTTOM. This means we intended to delete the static row so we
 +                 *     need to add it to the inRowTombstones (and otherwise handle the range normally).
 +                 */
 +                if (tombstone.stop.isStatic)
 +                {
 +                    // If the start is BOTTOM, we replace it by the beginning of the starting row so as to not confuse the
 +                    // RangeTombstone.isRowDeletion() method
 +                    if (tombstone.start == LegacyBound.BOTTOM)
 +                        tombstone = tombstone.withNewStart(new LegacyBound(staticBound(metadata, true), true, null));
 +                    inRowTombstones.add(tombstone);
 +                    return;
 +                }
 +
 +                if (tombstone.start.isStatic)
 +                {
 +                    if (tombstone.start.bound.isInclusive())
 +                        inRowTombstones.add(tombstone.withNewEnd(new LegacyBound(staticBound(metadata, false), true, null)));
 +
 +                    tombstone = tombstone.withNewStart(LegacyBound.BOTTOM);
 +                }
 +                else if (tombstone.start == LegacyBound.BOTTOM)
 +                {
 +                    inRowTombstones.add(new LegacyRangeTombstone(new LegacyBound(staticBound(metadata, true), true, null),
 +                                                                 new LegacyBound(staticBound(metadata, false), true, null),
 +                                                                 tombstone.deletionTime));
 +                }
 +            }
 +
 +            if (tombstone.isCollectionTombstone() || tombstone.isRowDeletion(metadata))
 +                inRowTombstones.add(tombstone);
 +            else
 +                add(metadata, new RangeTombstone(Slice.make(tombstone.start.bound, tombstone.stop.bound), tombstone.deletionTime));
 +        }
 +
 +        public void add(CFMetaData metadata, RangeTombstone tombstone)
 +        {
 +            deletionInfo.add(tombstone, metadata.comparator);
 +        }
 +
 +        public Iterator<LegacyRangeTombstone> inRowRangeTombstones()
 +        {
 +            return inRowTombstones.iterator();
 +        }
 +
 +        public static LegacyDeletionInfo deserialize(CFMetaData metadata, DataInputPlus in) throws IOException
 +        {
 +            DeletionTime topLevel = DeletionTime.serializer.deserialize(in);
 +
 +            int rangeCount = in.readInt();
 +            if (rangeCount == 0)
 +                return new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
 +
 +            LegacyDeletionInfo delInfo = new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
 +            for (int i = 0; i < rangeCount; i++)
 +            {
 +                LegacyBound start = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
 +                LegacyBound end = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
 +                int delTime =  in.readInt();
 +                long markedAt = in.readLong();
 +
 +                delInfo.add(metadata, new LegacyRangeTombstone(start, end, new DeletionTime(markedAt, delTime)));
 +            }
 +            return delInfo;
 +        }
 +    }
 +
 +    /**
 +     * A helper class for LegacyRangeTombstoneList.  This replaces the Comparator<Composite> that RTL used before 3.0.
 +     */
 +    private static class LegacyBoundComparator implements Comparator<LegacyBound>
 +    {
 +        ClusteringComparator clusteringComparator;
 +
 +        public LegacyBoundComparator(ClusteringComparator clusteringComparator)
 +        {
 +            this.clusteringComparator = clusteringComparator;
 +        }
 +
 +        public int compare(LegacyBound a, LegacyBound b)
 +        {
 +            // In the legacy sorting, BOTTOM comes before anything else
 +            if (a == LegacyBound.BOTTOM)
 +                return b == LegacyBound.BOTTOM ? 0 : -1;
 +            if (b == LegacyBound.BOTTOM)
 +                return 1;
 +
 +            // Excluding BOTTOM, statics are always before anything else.
 +            if (a.isStatic != b.isStatic)
 +                return a.isStatic ? -1 : 1;
 +
 +            // We have to be careful with bound comparison because of collections. Namely, if the 2 bounds represent the
 +            // same prefix, then we should take the collectionName into account before taking the bounds kind
 +            // (ClusteringPrefix.Kind). This means we can't really call ClusteringComparator.compare() directly.
 +            // For instance, if
 +            //    a is (bound=INCL_START_BOUND('x'), collectionName='d')
 +            //    b is (bound=INCL_END_BOUND('x'),   collectionName='c')
 +            // Ten b < a since the element 'c' of collection 'x' comes before element 'd', but calling
 +            // clusteringComparator.compare(a.bound, b.bound) returns -1.
 +            // See CASSANDRA-13125 for details.
 +            int sa = a.bound.size();
 +            int sb = b.bound.size();
 +            for (int i = 0; i < Math.min(sa, sb); i++)
 +            {
 +                int cmp = clusteringComparator.compareComponent(i, a.bound.get(i), b.bound.get(i));
 +                if (cmp != 0)
 +                    return cmp;
 +            }
 +
 +            if (sa != sb)
 +                return sa < sb ? a.bound.kind().comparedToClustering : -b.bound.kind().comparedToClustering;
 +
 +            // Both bound represent the same prefix, compare the collection names
 +            // If one has a collection name and the other doesn't, the other comes before as it points to the beginning of the row.
 +            if ((a.collectionName == null) != (b.collectionName == null))
 +                return a.collectionName == null ? -1 : 1;
 +
 +            // If they both have a collection, compare that first
 +            if (a.collectionName != null)
 +            {
 +                int cmp = UTF8Type.instance.compare(a.collectionName.name.bytes, b.collectionName.name.bytes);
 +                if (cmp != 0)
 +                    return cmp;
 +            }
 +
 +            // Lastly, if everything so far is equal, compare their clustering kind
 +            return ClusteringPrefix.Kind.compare(a.bound.kind(), b.bound.kind());
 +        }
 +    }
 +
 +    /**
 +     * Almost an entire copy of RangeTombstoneList from C* 2.1.  The main difference is that LegacyBoundComparator
 +     * is used in place of Comparator<Composite> (because Composite doesn't exist any more).
 +     *
 +     * This class is needed to allow us to convert single-row deletions and complex deletions into range tombstones
 +     * and properly merge them into the normal set of range tombstones.
 +     */
 +    public static class LegacyRangeTombstoneList
 +    {
 +        private final LegacyBoundComparator comparator;
 +
 +        // Note: we don't want to use a List for the markedAts and delTimes to avoid boxin

<TRUNCATED>

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


[29/29] cassandra git commit: Merge branch 'cassandra-3.11' into trunk

Posted by pa...@apache.org.
Merge branch 'cassandra-3.11' into trunk


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

Branch: refs/heads/trunk
Commit: 6034c268c389fc40a2f96c2746a09997906b93ba
Parents: 161c37d 0a6b6f5
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 15:01:25 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 15:03:59 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 +++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 ++
 conf/jvm.options                                |  11 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  15 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../db/ExpirationDateOverflowHandling.java      | 121 +++++++
 .../org/apache/cassandra/db/LivenessInfo.java   |  12 +-
 .../db/compaction/CompactionManager.java        |  16 +-
 .../cassandra/db/compaction/Scrubber.java       | 196 ++++++++++-
 .../apache/cassandra/db/rows/BufferCell.java    |   8 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |   3 +
 .../apache/cassandra/db/rows/NativeCell.java    |   5 +
 .../cassandra/service/StorageService.java       |   7 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |   6 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   7 +-
 .../table1/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/mc-1-big-Data.db                     | Bin 0 -> 58 bytes
 .../table1/mc-1-big-Digest.crc32                |   1 +
 .../table1/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Statistics.db               | Bin 0 -> 4676 bytes
 .../table1/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table1/mc-1-big-TOC.txt                     |   8 +
 .../table2/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/mc-1-big-Data.db                     | Bin 0 -> 60 bytes
 .../table2/mc-1-big-Digest.crc32                |   1 +
 .../table2/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Statistics.db               | Bin 0 -> 4686 bytes
 .../table2/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table2/mc-1-big-TOC.txt                     |   8 +
 .../table3/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/mc-1-big-Data.db                     | Bin 0 -> 111 bytes
 .../table3/mc-1-big-Digest.crc32                |   1 +
 .../table3/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Statistics.db               | Bin 0 -> 4732 bytes
 .../table3/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table3/mc-1-big-TOC.txt                     |   8 +
 .../table4/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/mc-1-big-Data.db                     | Bin 0 -> 108 bytes
 .../table4/mc-1-big-Digest.crc32                |   1 +
 .../table4/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Statistics.db               | Bin 0 -> 4742 bytes
 .../table4/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table4/mc-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 327 ++++++++++++++++++-
 test/unit/org/apache/cassandra/db/CellTest.java |  22 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 56 files changed, 860 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0f7b7fe,5b49f48..4c48287
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -237,8 -50,9 +237,9 @@@ Merged from 2.2
   * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
   * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
  Merged from 2.1:
+  * Protect against overflow of local expiration time (CASSANDRA-14092)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 - * More PEP8 compiance for cqlsh (CASSANDRA-14021)
 + * More PEP8 compliance for cqlsh
  
  
  3.11.1

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/NEWS.txt
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/conf/jvm.options
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/debian/rules
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/redhat/cassandra.spec
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index d915560,d4e230f..262a510
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -20,11 -20,13 +20,13 @@@ package org.apache.cassandra.cql3
  import java.nio.ByteBuffer;
  import java.util.List;
  
 -import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.functions.Function;
+ import org.apache.cassandra.db.ExpirationDateOverflowHandling;
  import org.apache.cassandra.db.LivenessInfo;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.schema.TableMetadata;
  import org.apache.cassandra.serializers.MarshalException;
  import org.apache.cassandra.utils.ByteBufferUtil;
  
@@@ -98,10 -100,13 +100,13 @@@ public class Attribute
          return LongType.instance.compose(tval);
      }
  
-     public int getTimeToLive(QueryOptions options, int defaultTimeToLive) throws InvalidRequestException
 -    public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
++    public int getTimeToLive(QueryOptions options, TableMetadata metadata) throws InvalidRequestException
      {
          if (timeToLive == null)
-             return defaultTimeToLive;
+         {
+             ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
+             return metadata.params.defaultTimeToLive;
+         }
  
          ByteBuffer tval = timeToLive.bindAndGet(options);
          if (tval == null)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f0cfd0d,8a896e9..31aa80c
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@@ -215,7 -203,7 +215,7 @@@ public abstract class ModificationState
  
      public int getTimeToLive(QueryOptions options) throws InvalidRequestException
      {
-         return attrs.getTimeToLive(options, metadata().params.defaultTimeToLive);
 -        return attrs.getTimeToLive(options, cfm);
++        return attrs.getTimeToLive(options, metadata);
      }
  
      public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
index 0000000,852dcb1..fea729f
mode 000000,100644..100644
--- a/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
+++ b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
@@@ -1,0 -1,121 +1,121 @@@
+ /*
+  * 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;
+ 
+ import java.util.concurrent.TimeUnit;
+ 
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ import org.slf4j.helpers.MessageFormatter;
+ 
 -import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.cql3.Attributes;
+ import org.apache.cassandra.db.rows.BufferCell;
+ import org.apache.cassandra.db.rows.Cell;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.schema.TableMetadata;
+ import org.apache.cassandra.service.ClientWarn;
+ import org.apache.cassandra.utils.NoSpamLogger;
+ 
+ public class ExpirationDateOverflowHandling
+ {
+     private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
+ 
+     private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
+ 
+     public enum ExpirationDateOverflowPolicy
+     {
+         REJECT, CAP_NOWARN, CAP
+     }
+ 
+     @VisibleForTesting
+     public static ExpirationDateOverflowPolicy policy;
+ 
+     static {
+         String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
+         try
+         {
+             policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
+         }
+         catch (RuntimeException e)
+         {
+             logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
+             policy = ExpirationDateOverflowPolicy.REJECT;
+         }
+     }
+ 
+     public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
+                                                                           "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
+                                                                           "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
+                                                                           "is fixed. See CASSANDRA-14092 for more details.";
+ 
+     public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
+                                                                                  "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
+                                                                                  "the expiration date overflow policy or upgrade to a version where this limitation " +
+                                                                                  "is fixed. See CASSANDRA-14092 for more details.";
+ 
 -    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
++    public static void maybeApplyExpirationDateOverflowPolicy(TableMetadata metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
+     {
+         if (ttl == BufferCell.NO_TTL)
+             return;
+ 
+         // Check for localExpirationTime overflow (CASSANDRA-14092)
+         int nowInSecs = (int)(System.currentTimeMillis() / 1000);
+         if (ttl + nowInSecs < 0)
+         {
+             switch (policy)
+             {
+                 case CAP:
 -                    ClientWarn.instance.warn(MessageFormatter.arrayFormat(MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING, new Object[] { metadata.ksName,
 -                                                                                                                                   metadata.cfName,
++                    ClientWarn.instance.warn(MessageFormatter.arrayFormat(MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING, new Object[] { metadata.keyspace,
++                                                                                                                                   metadata.name,
+                                                                                                                                    isDefaultTTL? "default " : "", ttl })
+                                                              .getMessage());
+                 case CAP_NOWARN:
+                     /**
+                      * Capping at this stage is basically not rejecting the request. The actual capping is done
+                      * by {@link #computeLocalExpirationTime(int, int)}, which converts the negative TTL
+                      * to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
+                      */
+                     NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES, TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
 -                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
++                                     metadata.keyspace, metadata.name, isDefaultTTL? "default " : "", ttl);
+                     return;
+ 
+                 default:
 -                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
++                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.keyspace, metadata.name,
+                                                                     isDefaultTTL? "default " : "", ttl));
+             }
+         }
+     }
+ 
+     /**
+      * This method computes the {@link Cell#localDeletionTime()}, maybe capping to the maximum representable value
+      * which is {@link Cell#MAX_DELETION_TIME}.
+      *
+      * Please note that the {@link ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy} is applied
 -     * during {@link ExpirationDateOverflowHandling#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
++     * during {@link ExpirationDateOverflowHandling#maybeApplyExpirationDateOverflowPolicy(org.apache.cassandra.schema.TableMetadata, int, boolean)},
+      * so if the request was not denied it means its expiration date should be capped.
+      *
+      * See CASSANDRA-14092
+      */
+     public static int computeLocalExpirationTime(int nowInSec, int timeToLive)
+     {
+         int localExpirationTime = nowInSec + timeToLive;
+         return localExpirationTime >= 0? localExpirationTime : Cell.MAX_DELETION_TIME;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LivenessInfo.java
index a8c33b0,c2a2291..1340c00
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@@ -18,11 -18,12 +18,12 @@@
  package org.apache.cassandra.db;
  
  import java.util.Objects;
 -import java.security.MessageDigest;
  
 -import org.apache.cassandra.config.CFMetaData;
 +import com.google.common.hash.Hasher;
 +
+ import org.apache.cassandra.db.rows.Cell;
  import org.apache.cassandra.serializers.MarshalException;
 -import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.HashingUtils;
  
  /**
   * Stores the information relating to the liveness of the primary key columns of a row.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index c5e5958,809e741..01fc188
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -391,9 -387,9 +399,9 @@@ public class CompactionManager implemen
              }
  
              @Override
 -            public void execute(LifecycleTransaction input) throws IOException
 +            public void execute(LifecycleTransaction input)
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
+                 scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTL);
              }
          }, jobs, OperationType.SCRUB);
      }
@@@ -968,7 -977,7 +976,7 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData)
 -    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL)
      {
          CompactionInfo.Holder scrubInfo = null;
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 42dd614,f8fa548..ce749f4
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -77,9 -80,15 +81,15 @@@ public class Scrubber implements Closea
      };
      private final SortedSet<Partition> outOfOrder = new TreeSet<>(partitionComparator);
  
 -    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
 +    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData)
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
+         this(cfs, transaction, skipCorrupted, checkData, false);
+     }
+ 
+     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
 -                    boolean reinsertOverflowedTTLRows) throws IOException
++                    boolean reinsertOverflowedTTLRows)
+     {
+         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
      @SuppressWarnings("resource")
@@@ -87,14 -96,16 +97,16 @@@
                      LifecycleTransaction transaction,
                      boolean skipCorrupted,
                      OutputHandler outputHandler,
-                     boolean checkData)
+                     boolean checkData,
 -                    boolean reinsertOverflowedTTLRows) throws IOException
++                    boolean reinsertOverflowedTTLRows)
      {
          this.cfs = cfs;
          this.transaction = transaction;
          this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 -        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(cfs.metadata(),
                                                                                                          sstable.descriptor.version,
                                                                                                          sstable.header);
  
@@@ -308,8 -324,8 +325,8 @@@
          // OrderCheckerIterator will check, at iteration time, that the rows are in the proper order. If it detects
          // that one row is out of order, it will stop returning them. The remaining rows will be sorted and added
          // to the outOfOrder set that will be later written to a new SSTable.
-         OrderCheckerIterator sstableIterator = new OrderCheckerIterator(new RowMergingSSTableIterator(SSTableIdentityIterator.create(sstable, dataFile, key)),
+         OrderCheckerIterator sstableIterator = new OrderCheckerIterator(getIterator(key),
 -                                                                        cfs.metadata.comparator);
 +                                                                        cfs.metadata().comparator);
  
          try (UnfilteredRowIterator iterator = withValidation(sstableIterator, dataFile.getPath()))
          {
@@@ -621,6 -654,153 +655,152 @@@
              previous = next;
              return next;
          }
+     }
+ 
+     /**
+      * This iterator converts negative {@link AbstractCell#localDeletionTime()} into {@link AbstractCell#MAX_DELETION_TIME}
+      *
+      * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+      */
+     private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
+     {
+         /**
+          * The decorated iterator.
+          */
+         private final UnfilteredRowIterator iterator;
+ 
+         private final OutputHandler outputHandler;
+         private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+ 
+         public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, OutputHandler outputHandler,
+                                                     NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+         {
+             this.iterator = iterator;
+             this.outputHandler = outputHandler;
+             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+         }
+ 
 -        public CFMetaData metadata()
++        public TableMetadata metadata()
+         {
+             return iterator.metadata();
+         }
+ 
+         public boolean isReverseOrder()
+         {
+             return iterator.isReverseOrder();
+         }
+ 
 -        public PartitionColumns columns()
++        public RegularAndStaticColumns columns()
+         {
+             return iterator.columns();
+         }
+ 
+         public DecoratedKey partitionKey()
+         {
+             return iterator.partitionKey();
+         }
+ 
+         public Row staticRow()
+         {
+             return iterator.staticRow();
+         }
+ 
+         @Override
+         public boolean isEmpty()
+         {
+             return iterator.isEmpty();
+         }
+ 
+         public void close()
+         {
+             iterator.close();
+         }
+ 
+         public DeletionTime partitionLevelDeletion()
+         {
+             return iterator.partitionLevelDeletion();
+         }
+ 
+         public EncodingStats stats()
+         {
+             return iterator.stats();
+         }
+ 
+         protected Unfiltered computeNext()
+         {
+             if (!iterator.hasNext())
+                 return endOfData();
+ 
+             Unfiltered next = iterator.next();
+             if (!next.isRow())
+                 return next;
+ 
+             if (hasNegativeLocalExpirationTime((Row) next))
+             {
+                 outputHandler.debug(String.format("Found row with negative local expiration time: %s", next.toString(metadata(), false)));
+                 negativeLocalExpirationTimeMetrics.fixedRows++;
+                 return fixNegativeLocalExpirationTime((Row) next);
+             }
+ 
+             return next;
+         }
+ 
+         private boolean hasNegativeLocalExpirationTime(Row next)
+         {
+             Row row = next;
+             if (row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0)
+             {
+                 return true;
+             }
+ 
+             for (ColumnData cd : row)
+             {
+                 if (cd.column().isSimple())
+                 {
+                     Cell cell = (Cell)cd;
+                     if (cell.isExpiring() && cell.localDeletionTime() < 0)
+                         return true;
+                 }
+                 else
+                 {
+                     ComplexColumnData complexData = (ComplexColumnData)cd;
+                     for (Cell cell : complexData)
+                     {
+                         if (cell.isExpiring() && cell.localDeletionTime() < 0)
+                             return true;
+                     }
+                 }
+             }
+ 
+             return false;
+         }
  
+         private Unfiltered fixNegativeLocalExpirationTime(Row row)
+         {
+             Row.Builder builder = HeapAllocator.instance.cloningBTreeRowBuilder();
+             builder.newRow(row.clustering());
+             builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0 ?
+                                               row.primaryKeyLivenessInfo().withUpdatedTimestampAndLocalDeletionTime(row.primaryKeyLivenessInfo().timestamp() + 1, AbstractCell.MAX_DELETION_TIME)
+                                               :row.primaryKeyLivenessInfo());
+             builder.addRowDeletion(row.deletion());
+             for (ColumnData cd : row)
+             {
+                 if (cd.column().isSimple())
+                 {
+                     Cell cell = (Cell)cd;
+                     builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
+                 }
+                 else
+                 {
+                     ComplexColumnData complexData = (ComplexColumnData)cd;
+                     builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
+                     for (Cell cell : complexData)
+                     {
+                         builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
+                     }
+                 }
+             }
+             return builder.build();
+         }
      }
 -
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BufferCell.java
index 76c6d3e,b62d95a..8bf8f7d
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@@ -19,7 -19,8 +19,8 @@@ package org.apache.cassandra.db.rows
  
  import java.nio.ByteBuffer;
  
 -import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.db.ExpirationDateOverflowHandling;
 +import org.apache.cassandra.schema.ColumnMetadata;
  import org.apache.cassandra.db.marshal.ByteType;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.ObjectSizes;
@@@ -63,13 -64,13 +64,13 @@@ public class BufferCell extends Abstrac
          return expiring(column, timestamp, ttl, nowInSec, value, null);
      }
  
 -    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
 +    public static BufferCell expiring(ColumnMetadata column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
      {
          assert ttl != NO_TTL;
-         return new BufferCell(column, timestamp, ttl, nowInSec + ttl, value, path);
+         return new BufferCell(column, timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), value, path);
      }
  
 -    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec)
 +    public static BufferCell tombstone(ColumnMetadata column, long timestamp, int nowInSec)
      {
          return tombstone(column, timestamp, nowInSec, null);
      }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/db/rows/NativeCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/NativeCell.java
index 10ccf88,31ce0b7..c4cb6c1
--- a/src/java/org/apache/cassandra/db/rows/NativeCell.java
+++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java
@@@ -143,7 -143,12 +143,12 @@@ public class NativeCell extends Abstrac
          throw new UnsupportedOperationException();
      }
  
+     public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+     {
+         return new BufferCell(column, newTimestamp, ttl(), newLocalDeletionTime, value(), path());
+     }
+ 
 -    public Cell withUpdatedColumn(ColumnDefinition column)
 +    public Cell withUpdatedColumn(ColumnMetadata column)
      {
          return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), value(), path());
      }

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6034c268/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------


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


[05/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
new file mode 100644
index 0000000..ab4ef21
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -0,0 +1,410 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.BufferExpiringCell;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+public class TTLTest extends CQLTester
+{
+    public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+
+    public static int MAX_TTL = ExpiringCell.MAX_TTL;
+
+    public static final String SIMPLE_NOCLUSTERING = "table1";
+    public static final String SIMPLE_CLUSTERING = "table2";
+    public static final String COMPLEX_NOCLUSTERING = "table3";
+    public static final String COMPLEX_CLUSTERING = "table4";
+
+    @Test
+    public void testTTLPerRequestLimit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        // insert with low TTL should not be denied
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+        execute("TRUNCATE %s");
+
+        // insert with low TTL should not be denied
+        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+    }
+
+
+    @Test
+    public void testTTLDefaultLimit() throws Throwable
+    {
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live cannot be smaller than 0"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                        + (MAX_TTL + 1));
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                  + (MAX_TTL + 1) + ")"));
+        }
+
+        // table with default low TTL should not be denied
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+    }
+
+    @Test
+    public void testRejectExpirationDateOverflowPolicy() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+            execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        checkTTLIsCapped("i");
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyPerRequest() throws Throwable
+    {
+        // Test cap policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+
+        // simple column, clustering, flush
+        baseCapExpirationDateOverflowTest(true, true, true);
+        // simple column, clustering, noflush
+        baseCapExpirationDateOverflowTest(true, true, false);
+        // simple column, noclustering, flush
+        baseCapExpirationDateOverflowTest(true, false, true);
+        // simple column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(true, false, false);
+        // complex column, clustering, flush
+        baseCapExpirationDateOverflowTest(false, true, true);
+        // complex column, clustering, noflush
+        baseCapExpirationDateOverflowTest(false, true, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, true);
+        // complex column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(false, false, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, false);
+
+        // Return to previous policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+    {
+        createTable(true, true);
+        createTable(true, false);
+        createTable(false, true);
+        createTable(false, false);
+
+        baseTestRecoverOverflowedExpiration(false, false);
+        baseTestRecoverOverflowedExpiration(true, false);
+        baseTestRecoverOverflowedExpiration(true, true);
+    }
+
+    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+    {
+        // Create Table
+        if (simple)
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b int)");
+        }
+        else
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b set<text>)");
+        }
+
+        // Insert data with INSERT and UPDATE
+        if (simple)
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+            if (clustering)
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+            else
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+        }
+        else
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+            if (clustering)
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+            else
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+        }
+
+        // Maybe Flush
+        Keyspace ks = Keyspace.open(keyspace());
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+        // Verify data
+        verifyData(simple);
+
+        // Maybe major compact
+        if (flush)
+        {
+            // Major compact and check data is still present
+            ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+
+            // Verify data again
+            verifyData(simple);
+        }
+    }
+
+    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        // simple column, clustering
+        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+        // simple column, noclustering
+        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+        // complex column, clustering
+        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+        // complex column, noclustering
+        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+    }
+
+    private void verifyData(boolean simple) throws Throwable
+    {
+        if (simple)
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+        }
+        else
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        // Cannot retrieve TTL from collections
+        if (simple)
+            checkTTLIsCapped("b");
+    }
+
+    /**
+     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+     */
+    private void checkTTLIsCapped(String field) throws Throwable
+    {
+
+        // TTL is computed dynamically from row expiration time, so if it is
+        // equal or higher to the minimum max TTL we compute before the query
+        // we are fine.
+        int minMaxTTL = computeMaxTTL();
+        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+        for (UntypedResultSet.Row row : execute)
+        {
+            int ttl = row.getInt("ttl(" + field + ")");
+            assertTrue(ttl >= minMaxTTL);
+        }
+    }
+
+    /**
+     * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+     */
+    private int computeMaxTTL()
+    {
+        int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+    }
+
+    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        if (reinsertOverflowedTTL)
+        {
+            assert runScrub;
+        }
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
+
+        assertEquals(0, cfs.getLiveSSTableCount());
+
+        copySSTablesToTableDir(simple, clustering);
+
+        cfs.loadNewSSTables();
+
+        if (runScrub)
+        {
+            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+        }
+
+        if (reinsertOverflowedTTL)
+        {
+            if (simple)
+            {
+                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
+                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
+
+            }
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+
+            cfs.forceMajorCompaction();
+
+            if (simple)
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        else
+        {
+            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
+        }
+        cfs.truncateBlocking(); //cleanup for next tests
+    }
+
+    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
+    {
+        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
+        File sourceDir = getTableDir(simple, clustering);
+        for (File file : sourceDir.listFiles())
+        {
+            copyFile(file, destDir);
+        }
+    }
+
+    private void createTable(boolean simple, boolean clustering) throws Throwable
+    {
+        if (simple)
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
+        }
+        else
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
+        }
+    }
+
+    private static File getTableDir(boolean simple, boolean clustering)
+    {
+        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+    }
+
+    private static void copyFile(File src, File dest) throws IOException
+    {
+        byte[] buf = new byte[65536];
+        if (src.isFile())
+        {
+            File target = new File(dest, src.getName());
+            int rd;
+            FileInputStream is = new FileInputStream(src);
+            FileOutputStream os = new FileOutputStream(target);
+            while ((rd = is.read(buf)) >= 0)
+                os.write(buf, 0, rd);
+        }
+    }
+
+    public static String getTableName(boolean simple, boolean clustering)
+    {
+        if (simple)
+            return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+        else
+            return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+    }
+}


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


[18/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/trunk
Commit: c231ed5be0f47b030181185f544132523a2cf908
Parents: 21978bf 1602e60
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:55:27 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:57:53 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 +++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 ++
 conf/jvm.options                                |  11 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  14 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../db/ExpirationDateOverflowHandling.java      | 121 +++++++
 .../org/apache/cassandra/db/LegacyLayout.java   |   7 +-
 .../org/apache/cassandra/db/LivenessInfo.java   |  12 +-
 .../db/compaction/CompactionManager.java        |  16 +-
 .../cassandra/db/compaction/Scrubber.java       | 198 ++++++++++-
 .../apache/cassandra/db/rows/BufferCell.java    |   7 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |   9 +
 .../cassandra/service/StorageService.java       |   7 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/SSTableMetadataViewer.java  |   1 +
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/mc-1-big-Data.db                     | Bin 0 -> 58 bytes
 .../table1/mc-1-big-Digest.crc32                |   1 +
 .../table1/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Statistics.db               | Bin 0 -> 4676 bytes
 .../table1/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table1/mc-1-big-TOC.txt                     |   8 +
 .../table2/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/mc-1-big-Data.db                     | Bin 0 -> 60 bytes
 .../table2/mc-1-big-Digest.crc32                |   1 +
 .../table2/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Statistics.db               | Bin 0 -> 4686 bytes
 .../table2/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table2/mc-1-big-TOC.txt                     |   8 +
 .../table3/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/mc-1-big-Data.db                     | Bin 0 -> 111 bytes
 .../table3/mc-1-big-Digest.crc32                |   1 +
 .../table3/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Statistics.db               | Bin 0 -> 4732 bytes
 .../table3/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table3/mc-1-big-TOC.txt                     |   8 +
 .../table4/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/mc-1-big-Data.db                     | Bin 0 -> 108 bytes
 .../table4/mc-1-big-Digest.crc32                |   1 +
 .../table4/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Statistics.db               | Bin 0 -> 4742 bytes
 .../table4/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table4/mc-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 327 ++++++++++++++++++-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 57 files changed, 867 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --cc CASSANDRA-14092.txt
index 0000000,5ac872c..f95380b
mode 000000,100644..100644
--- a/CASSANDRA-14092.txt
+++ b/CASSANDRA-14092.txt
@@@ -1,0 -1,81 +1,81 @@@
+ CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+ ---------------------------------------------
+ 
+ The maximum expiration timestamp that can be represented by the storage engine is
+ 2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+ after this date are not currently supported.
+ 
+ # Expiration Date Overflow Policy
+ 
+ We plan to lift this limitation in newer versions, but while the fix is not available,
+ operators can decide which policy to apply when dealing with inserts with TTL exceeding
+ the maximum supported expiration date:
+   -     REJECT: this is the default policy and will reject any requests with expiration
+                 date timestamp after 2038-01-19T03:14:06+00:00.
+   -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                 2038-01-19T03:14:06+00:00 and the client will receive a warning.
+   - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+ 
+ These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
 -startup option.
++startup option in the jvm.options configuration file.
+ 
+ # Potential data loss on earlier versions
+ 
+ Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+ INSERTS with TTL expiring after the maximum supported date, causing the expiration
+ time field to overflow and the records to expire immediately. Expired records due
+ to overflow will not be queryable and will be permanently removed after a compaction.
+ 
+ 2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+ since an AssertionError is thrown during INSERT when the expiration time field overflows
+ on these versions. When assertions are disabled then it is possible to INSERT entries
+ with overflowed local expiration time and even the earlier versions are subject to data
+ loss due to this bug.
+ 
+ This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+ of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+ the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+ For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+ will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+ where this issue is addressed as soon as possible.
+ 
+ # Data Recovery
+ 
+ SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+ with overflowed expiration time that were backed up or did not go through compaction can
+ be recovered by reinserting overflowed entries with a valid expiration time and a higher
+ timestamp, since tombstones may have been generated with the original timestamp.
+ 
+ To find out if an SSTable has an entry with overflowed expiration, inspect it with the
 -sstable2json tool and look for a negative "local deletion time" field. SSTables in this
 -condition should be backed up immediately, as they are subject to data loss during
++'sstablemetadata' tool and look for a negative "Minimum local deletion time" field. SSTables
++in this condition should be backed up immediately, as they are subject to data loss during
+ compaction.
+ 
+ A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+ rows with overflowed expiration time with the maximum expiration date of
+ 2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+ for recovery of SSTables via scrub:
+ 
+ - Offline scrub:
+    - Clone the data directory tree to another location, keeping only the folders and the
+      contents of the system tables.
+    - Clone the configuration directory to another location, setting the data_file_directories
+      property to the cloned data directory in the cloned cassandra.yaml.
+    - Copy the affected SSTables to the cloned data location of the affected table.
+    - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+    - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+          WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+          compaction, so the data with overflowed will be removed - make sure to back up
+          your SSTables before running scrub.
+    - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+    - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+ 
+ - Online scrub:
+    - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+      as otherwise, the data may be removed permanently during compaction.
+    - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+    - run "nodetool refresh" to load the SSTables.
+    - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+    - Re-enable compactions after verifying that scrub recovered the missing entries.
+ 
 -See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
++See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index f42f3f4,82da6ad..a492c42
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,29 -1,4 +1,30 @@@
 -2.2.12
 +3.0.16
++ * Protect against overflow of local expiration time (CASSANDRA-14092)
 + * Use the correct digest file and reload sstable metadata in nodetool verify (CASSANDRA-14217)
 + * Handle failure when mutating repaired status in Verifier (CASSANDRA-13933)
 + * Close socket on error during connect on OutboundTcpConnection (CASSANDRA-9630)
 + * Set encoding for javadoc generation (CASSANDRA-14154)
 + * Fix index target computation for dense composite tables with dropped compact storage (CASSANDRA-14104)
 + * Improve commit log chain marker updating (CASSANDRA-14108)
 + * Extra range tombstone bound creates double rows (CASSANDRA-14008)
 + * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
 + * Accept role names containing forward-slash (CASSANDRA-14088)
 + * Optimize CRC check chance probability calculations (CASSANDRA-14094)
 + * Fix cleanup on keyspace with no replicas (CASSANDRA-13526)
 + * Fix updating base table rows with TTL not removing materialized view entries (CASSANDRA-14071)
 + * Reduce garbage created by DynamicSnitch (CASSANDRA-14091)
 + * More frequent commitlog chained markers (CASSANDRA-13987)
 + * Fix serialized size of DataLimits (CASSANDRA-14057)
 + * Add flag to allow dropping oversized read repair mutations (CASSANDRA-13975)
 + * Fix SSTableLoader logger message (CASSANDRA-14003)
 + * Fix repair race that caused gossip to block (CASSANDRA-13849)
 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964)
 + * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959)
 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813)
 + * Provide a JMX call to sync schema with local storage (CASSANDRA-13954)
 + * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939)
 + * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922)
 +Merged from 2.2:
   * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
   * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
   * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 1bbe1aa,4fe3508..f574c33
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,71 -33,7 +33,72 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.12
 +3.0.16
 +=====
 +
 +Upgrading
 +---------
++   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +     options are used. See CASSANDRA-13006 for more details.
 +   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +     for more details.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
 +3.0.15
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous upgrading sections,
 +     especially if you are upgrading from 2.2.
 +
 +Compact Storage
 +---------------
 +    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
 +      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
 +      exposing internal structure of Thrift/Compact Tables. You can find more details
 +      on exposed internal structure under: 
 +      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
 +
 +      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
 +      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
 +      current connection.
 +
 +      In order to upgrade, clients supporting a non-compact schema view can be rolled out
 +      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
 +      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
 +      after that.
 +
 +Materialized Views
 +-------------------
 +    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
 +    - A change was made in the way the Materialized View timestamp is computed, which
 +      may cause an old deletion to a base column which is view primary key (PK) column
 +      to not be reflected in the view when repairing the base table post-upgrade. This
 +      condition is only possible when a column deletion to an MV primary key (PK) column
 +      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
 +      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
 +      If such column deletions are done on a view PK column which is not a base PK, it's advisable
 +      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
 +      to fix potential inconsistencies by running repair on the views after upgrade or drop and
 +      re-create the views. See CASSANDRA-11500 for more details.
 +    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
 +      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
 +      situations so we advise against doing deletions on base columns not selected in views
 +      until this is fixed on CASSANDRA-13826.
 +
 +3.0.14
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/conf/jvm.options
----------------------------------------------------------------------
diff --cc conf/jvm.options
index a7b3bd8,0000000..eb2ad19
mode 100644,000000..100644
--- a/conf/jvm.options
+++ b/conf/jvm.options
@@@ -1,108 -1,0 +1,119 @@@
 +###########################################################################
 +#                             jvm.options                                 #
 +#                                                                         #
 +# - all flags defined here will be used by cassandra to startup the JVM   #
 +# - one flag should be specified per line                                 #
 +# - lines that do not start with '-' will be ignored                      #
 +# - only static flags are accepted (no variables or parameters)           #
 +# - dynamic flags will be appended to these on cassandra-env              #
 +###########################################################################
 +
 +#################
 +# HEAP SETTINGS #
 +#################
 +
 +# Heap size is automatically calculated by cassandra-env based on this
 +# formula: max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
 +# That is:
 +# - calculate 1/2 ram and cap to 1024MB
 +# - calculate 1/4 ram and cap to 8192MB
 +# - pick the max
 +#
 +# For production use you may wish to adjust this for your environment.
 +# If that's the case, uncomment the -Xmx and Xms options below to override the
 +# automatic calculation of JVM heap memory.
 +#
 +# It is recommended to set min (-Xms) and max (-Xmx) heap sizes to
 +# the same value to avoid stop-the-world GC pauses during resize, and
 +# so that we can lock the heap in memory on startup to prevent any
 +# of it from being swapped out.
 +#-Xms4G
 +#-Xmx4G
 +
 +# Young generation size is automatically calculated by cassandra-env
 +# based on this formula: min(100 * num_cores, 1/4 * heap size)
 +#
 +# The main trade-off for the young generation is that the larger it
 +# is, the longer GC pause times will be. The shorter it is, the more
 +# expensive GC will be (usually).
 +#
 +# It is not recommended to set the young generation size if using the
 +# G1 GC, since that will override the target pause-time goal.
 +# More info: http://www.oracle.com/technetwork/articles/java/g1gc-1984535.html
 +#
 +# The example below assumes a modern 8-core+ machine for decent
 +# times. If in doubt, and if you do not particularly want to tweak, go
 +# 100 MB per physical CPU core.
 +#-Xmn800M
 +
++###################################
++# EXPIRATION DATE OVERFLOW POLICY #
++###################################
++
++# Defines how to handle INSERT requests with TTL exceeding the maximum supported expiration date:
++# * REJECT: this is the default policy and will reject any requests with expiration date timestamp after 2038-01-19T03:14:06+00:00.
++# * CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on 2038-01-19T03:14:06+00:00 and the client will receive a warning.
++# * CAP_NOWARN: same as previous, except that the client warning will not be emitted.
++#
++#-Dcassandra.expiration_date_overflow_policy=REJECT
++
 +#################
 +#  GC SETTINGS  #
 +#################
 +
 +### CMS Settings
 +
 +-XX:+UseParNewGC
 +-XX:+UseConcMarkSweepGC
 +-XX:+CMSParallelRemarkEnabled
 +-XX:SurvivorRatio=8
 +-XX:MaxTenuringThreshold=1
 +-XX:CMSInitiatingOccupancyFraction=75
 +-XX:+UseCMSInitiatingOccupancyOnly
 +-XX:CMSWaitDuration=10000
 +-XX:+CMSParallelInitialMarkEnabled
 +-XX:+CMSEdenChunksRecordAlways
 +# some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541
 +-XX:+CMSClassUnloadingEnabled
 +
 +### G1 Settings (experimental, comment previous section and uncomment section below to enable)
 +
 +## Use the Hotspot garbage-first collector.
 +#-XX:+UseG1GC
 +#
 +## Have the JVM do less remembered set work during STW, instead
 +## preferring concurrent GC. Reduces p99.9 latency.
 +#-XX:G1RSetUpdatingPauseTimePercent=5
 +#
 +## Main G1GC tunable: lowering the pause target will lower throughput and vise versa.
 +## 200ms is the JVM default and lowest viable setting
 +## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml.
 +#-XX:MaxGCPauseMillis=500
 +
 +## Optional G1 Settings
 +
 +# Save CPU time on large (>= 16GB) heaps by delaying region scanning
 +# until the heap is 70% full. The default in Hotspot 8u40 is 40%.
 +#-XX:InitiatingHeapOccupancyPercent=70
 +
 +# For systems with > 8 cores, the default ParallelGCThreads is 5/8 the number of logical cores.
 +# Otherwise equal to the number of cores when 8 or less.
 +# Machines with > 10 cores should try setting these to <= full cores.
 +#-XX:ParallelGCThreads=16
 +# By default, ConcGCThreads is 1/4 of ParallelGCThreads.
 +# Setting both to the same value can reduce STW durations.
 +#-XX:ConcGCThreads=16
 +
 +### GC logging options -- uncomment to enable
 +
 +-XX:+PrintGCDetails
 +-XX:+PrintGCDateStamps
 +-XX:+PrintHeapAtGC
 +-XX:+PrintTenuringDistribution
 +-XX:+PrintGCApplicationStoppedTime
 +-XX:+PrintPromotionFailure
 +#-XX:PrintFLSStatistics=1
 +#-Xloggc:/var/log/cassandra/gc.log
 +-XX:+UseGCLogFileRotation
 +-XX:NumberOfGCLogFiles=10
 +-XX:GCLogFileSize=10M

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/redhat/cassandra.spec
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 4ed0f83,84f423a..832d0a7
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,12 -18,17 +18,11 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
--import java.util.Collections;
 -import java.util.concurrent.TimeUnit;
 -
 -import com.google.common.collect.Iterables;
 -import com.google.common.annotations.VisibleForTesting;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 +import java.util.List;
  
- import com.google.common.collect.Iterables;
- 
+ import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.functions.Function;
 -import org.apache.cassandra.db.ExpiringCell;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
@@@ -100,10 -137,13 +99,13 @@@ public class Attribute
          return LongType.instance.compose(tval);
      }
  
-     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+     public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
      {
          if (timeToLive == null)
-             return 0;
+         {
 -            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
 -            return metadata.getDefaultTimeToLive();
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
++            return metadata.params.defaultTimeToLive;
+         }
  
          ByteBuffer tval = timeToLive.bindAndGet(options);
          if (tval == null)
@@@ -125,9 -165,11 +127,11 @@@
          if (ttl < 0)
              throw new InvalidRequestException("A TTL must be greater or equal to 0, but was " + ttl);
  
 -        if (ttl > ExpiringCell.MAX_TTL)
 -            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 +        if (ttl > MAX_TTL)
 +            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, MAX_TTL));
  
 -        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
++        ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+ 
          return ttl;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 536681f,45908de..b5946bb
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1379,13 -1610,12 +1379,13 @@@ public class ColumnFamilyStore implemen
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
++        return scrub(disableSnapshot, skipCorrupted, reinsertOverflowedTTL, false, checkData, jobs);
      }
  
 -    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean reinsertOverflowedTTL, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
@@@ -1393,7 -1623,7 +1393,7 @@@
  
          try
          {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 -            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTL, jobs);
          }
          catch(Throwable t)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
index 0000000,0000000..852dcb1
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
@@@ -1,0 -1,0 +1,121 @@@
++/*
++ * 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;
++
++import java.util.concurrent.TimeUnit;
++
++import com.google.common.annotations.VisibleForTesting;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++import org.slf4j.helpers.MessageFormatter;
++
++import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.cql3.Attributes;
++import org.apache.cassandra.db.rows.BufferCell;
++import org.apache.cassandra.db.rows.Cell;
++import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.service.ClientWarn;
++import org.apache.cassandra.utils.NoSpamLogger;
++
++public class ExpirationDateOverflowHandling
++{
++    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
++
++    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
++
++    public enum ExpirationDateOverflowPolicy
++    {
++        REJECT, CAP_NOWARN, CAP
++    }
++
++    @VisibleForTesting
++    public static ExpirationDateOverflowPolicy policy;
++
++    static {
++        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
++        try
++        {
++            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
++        }
++        catch (RuntimeException e)
++        {
++            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
++            policy = ExpirationDateOverflowPolicy.REJECT;
++        }
++    }
++
++    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
++                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
++                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
++                                                                          "is fixed. See CASSANDRA-14092 for more details.";
++
++    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
++                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
++                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
++                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
++
++    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
++    {
++        if (ttl == BufferCell.NO_TTL)
++            return;
++
++        // Check for localExpirationTime overflow (CASSANDRA-14092)
++        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
++        if (ttl + nowInSecs < 0)
++        {
++            switch (policy)
++            {
++                case CAP:
++                    ClientWarn.instance.warn(MessageFormatter.arrayFormat(MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING, new Object[] { metadata.ksName,
++                                                                                                                                   metadata.cfName,
++                                                                                                                                   isDefaultTTL? "default " : "", ttl })
++                                                             .getMessage());
++                case CAP_NOWARN:
++                    /**
++                     * Capping at this stage is basically not rejecting the request. The actual capping is done
++                     * by {@link #computeLocalExpirationTime(int, int)}, which converts the negative TTL
++                     * to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
++                     */
++                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES, TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
++                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
++                    return;
++
++                default:
++                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
++                                                                    isDefaultTTL? "default " : "", ttl));
++            }
++        }
++    }
++
++    /**
++     * This method computes the {@link Cell#localDeletionTime()}, maybe capping to the maximum representable value
++     * which is {@link Cell#MAX_DELETION_TIME}.
++     *
++     * Please note that the {@link ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy} is applied
++     * during {@link ExpirationDateOverflowHandling#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
++     * so if the request was not denied it means its expiration date should be capped.
++     *
++     * See CASSANDRA-14092
++     */
++    public static int computeLocalExpirationTime(int nowInSec, int timeToLive)
++    {
++        int localExpirationTime = nowInSec + timeToLive;
++        return localExpirationTime >= 0? localExpirationTime : Cell.MAX_DELETION_TIME;
++    }
++}


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


[20/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LivenessInfo.java
index 89e0578,0000000..f6c9b62
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@@ -1,369 -1,0 +1,375 @@@
 +/*
 + * 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;
 +
 +import java.util.Objects;
 +import java.security.MessageDigest;
 +
 +import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.db.rows.Cell;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * Stores the information relating to the liveness of the primary key columns of a row.
 + * <p>
 + * A {@code LivenessInfo} can first be empty. If it isn't, it contains at least a timestamp,
 + * which is the timestamp for the row primary key columns. On top of that, the info can be
 + * ttl'ed, in which case the {@code LivenessInfo} also has both a ttl and a local expiration time.
 + * <p>
 + * Please note that if a liveness info is ttl'ed, that expiration is <b>only</b> an expiration
 + * of the liveness info itself (so, of the timestamp), and once the info expires it becomes
 + * {@code EMPTY}. But if a row has a liveness info which expires, the rest of the row data is
 + * unaffected (of course, the rest of said row data might be ttl'ed on its own but this is
 + * separate).
 + */
 +public class LivenessInfo
 +{
 +    public static final long NO_TIMESTAMP = Long.MIN_VALUE;
-     public static final int NO_TTL = 0;
++    public static final int NO_TTL = Cell.NO_TTL;
 +    /**
 +     * Used as flag for representing an expired liveness.
 +     *
 +     * TTL per request is at most 20 yrs, so this shouldn't conflict
 +     * (See {@link org.apache.cassandra.cql3.Attributes#MAX_TTL})
 +     */
 +    public static final int EXPIRED_LIVENESS_TTL = Integer.MAX_VALUE;
-     public static final int NO_EXPIRATION_TIME = Integer.MAX_VALUE;
++    public static final int NO_EXPIRATION_TIME = Cell.NO_DELETION_TIME;
 +
 +    public static final LivenessInfo EMPTY = new LivenessInfo(NO_TIMESTAMP);
 +
 +    protected final long timestamp;
 +
 +    protected LivenessInfo(long timestamp)
 +    {
 +        this.timestamp = timestamp;
 +    }
 +
 +    public static LivenessInfo create(CFMetaData metadata, long timestamp, int nowInSec)
 +    {
 +        int defaultTTL = metadata.params.defaultTimeToLive;
 +        if (defaultTTL != NO_TTL)
 +            return expiring(timestamp, defaultTTL, nowInSec);
 +
 +        return new LivenessInfo(timestamp);
 +    }
 +
 +    public static LivenessInfo expiring(long timestamp, int ttl, int nowInSec)
 +    {
 +        assert ttl != EXPIRED_LIVENESS_TTL;
-         return new ExpiringLivenessInfo(timestamp, ttl, nowInSec + ttl);
++        return new ExpiringLivenessInfo(timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl));
 +    }
 +
 +    public static LivenessInfo create(CFMetaData metadata, long timestamp, int ttl, int nowInSec)
 +    {
 +        return ttl == NO_TTL
 +             ? create(metadata, timestamp, nowInSec)
 +             : expiring(timestamp, ttl, nowInSec);
 +    }
 +
 +    // Note that this ctor ignores the default table ttl and takes the expiration time, not the current time.
 +    // Use when you know that's what you want.
 +    public static LivenessInfo create(long timestamp, int ttl, int localExpirationTime)
 +    {
 +        if (ttl == EXPIRED_LIVENESS_TTL)
 +            return new ExpiredLivenessInfo(timestamp, ttl, localExpirationTime);
 +        return ttl == NO_TTL ? new LivenessInfo(timestamp) : new ExpiringLivenessInfo(timestamp, ttl, localExpirationTime);
 +    }
 +
 +    /**
 +     * Whether this liveness info is empty (has no timestamp).
 +     *
 +     * @return whether this liveness info is empty or not.
 +     */
 +    public boolean isEmpty()
 +    {
 +        return timestamp == NO_TIMESTAMP;
 +    }
 +
 +    /**
 +     * The timestamp for this liveness info.
 +     *
 +     * @return the liveness info timestamp (or {@link #NO_TIMESTAMP} if the info is empty).
 +     */
 +    public long timestamp()
 +    {
 +        return timestamp;
 +    }
 +
 +    /**
 +     * Whether the info has a ttl.
 +     */
 +    public boolean isExpiring()
 +    {
 +        return false;
 +    }
 +
 +    /**
 +     * The ttl (if any) on the row primary key columns or {@link #NO_TTL} if it is not
 +     * expiring.
 +     *
 +     * Please note that this value is the TTL that was set originally and is thus not
 +     * changing.
 +     */
 +    public int ttl()
 +    {
 +        return NO_TTL;
 +    }
 +
 +    /**
 +     * The expiration time (in seconds) if the info is expiring ({@link #NO_EXPIRATION_TIME} otherwise).
 +     *
 +     */
 +    public int localExpirationTime()
 +    {
 +        return NO_EXPIRATION_TIME;
 +    }
 +
 +    /**
 +     * Whether that info is still live.
 +     *
 +     * A {@code LivenessInfo} is live if it is either not expiring, or if its expiration time if after
 +     * {@code nowInSec}.
 +     *
 +     * @param nowInSec the current time in seconds.
 +     * @return whether this liveness info is live or not.
 +     */
 +    public boolean isLive(int nowInSec)
 +    {
 +        return !isEmpty();
 +    }
 +
 +    /**
 +     * Adds this liveness information to the provided digest.
 +     *
 +     * @param digest the digest to add this liveness information to.
 +     */
 +    public void digest(MessageDigest digest)
 +    {
 +        FBUtilities.updateWithLong(digest, timestamp());
 +    }
 +
 +    /**
 +     * Validate the data contained by this liveness information.
 +     *
 +     * @throws MarshalException if some of the data is corrupted.
 +     */
 +    public void validate()
 +    {
 +    }
 +
 +    /**
 +     * The size of the (useful) data this liveness information contains.
 +     *
 +     * @return the size of the data this liveness information contains.
 +     */
 +    public int dataSize()
 +    {
 +        return TypeSizes.sizeof(timestamp());
 +    }
 +
 +    /**
 +     * Whether this liveness information supersedes another one (that is
 +     * whether is has a greater timestamp than the other or not).
 +     *
 +     * </br>
 +     *
 +     * If timestamps are the same and none of them are expired livenessInfo,
 +     * livenessInfo with greater TTL supersedes another. It also means, if timestamps are the same,
 +     * ttl superseders no-ttl. This is the same rule as {@link Conflicts#resolveRegular}
 +     *
 +     * If timestamps are the same and one of them is expired livenessInfo. Expired livenessInfo
 +     * supersedes, ie. tombstone supersedes.
 +     *
 +     * If timestamps are the same and both of them are expired livenessInfo(Ideally it shouldn't happen),
 +     * greater localDeletionTime wins.
 +     *
 +     * @param other
 +     *            the {@code LivenessInfo} to compare this info to.
 +     *
 +     * @return whether this {@code LivenessInfo} supersedes {@code other}.
 +     */
 +    public boolean supersedes(LivenessInfo other)
 +    {
 +        if (timestamp != other.timestamp)
 +            return timestamp > other.timestamp;
 +        if (isExpired() ^ other.isExpired())
 +            return isExpired();
 +        if (isExpiring() == other.isExpiring())
 +            return localExpirationTime() > other.localExpirationTime();
 +        return isExpiring();
 +    }
 +
 +    protected boolean isExpired()
 +    {
 +        return false;
 +    }
 +
 +    /**
 +     * Returns a copy of this liveness info updated with the provided timestamp.
 +     *
 +     * @param newTimestamp the timestamp for the returned info.
 +     * @return if this liveness info has a timestamp, a copy of it with {@code newTimestamp}
 +     * as timestamp. If it has no timestamp however, this liveness info is returned
 +     * unchanged.
 +     */
 +    public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +    {
 +        return new LivenessInfo(newTimestamp);
 +    }
 +
++    public LivenessInfo withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return LivenessInfo.create(newTimestamp, ttl(), newLocalDeletionTime);
++    }
++
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("[ts=%d]", timestamp);
 +    }
 +
 +    @Override
 +    public boolean equals(Object other)
 +    {
 +        if(!(other instanceof LivenessInfo))
 +            return false;
 +
 +        LivenessInfo that = (LivenessInfo)other;
 +        return this.timestamp() == that.timestamp()
 +            && this.ttl() == that.ttl()
 +            && this.localExpirationTime() == that.localExpirationTime();
 +    }
 +
 +    @Override
 +    public int hashCode()
 +    {
 +        return Objects.hash(timestamp(), ttl(), localExpirationTime());
 +    }
 +
 +    /**
 +     * Effectively acts as a PK tombstone. This is used for Materialized Views to shadow
 +     * updated entries while co-existing with row tombstones.
 +     *
 +     * See {@link org.apache.cassandra.db.view.ViewUpdateGenerator#deleteOldEntryInternal}.
 +     */
 +    private static class ExpiredLivenessInfo extends ExpiringLivenessInfo
 +    {
 +        private ExpiredLivenessInfo(long timestamp, int ttl, int localExpirationTime)
 +        {
 +            super(timestamp, ttl, localExpirationTime);
 +            assert ttl == EXPIRED_LIVENESS_TTL;
 +            assert timestamp != NO_TIMESTAMP;
 +        }
 +
 +        @Override
 +        public boolean isExpired()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean isLive(int nowInSec)
 +        {
 +            // used as tombstone to shadow entire PK
 +            return false;
 +        }
 +
 +        @Override
 +        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +        {
 +            return new ExpiredLivenessInfo(newTimestamp, ttl(), localExpirationTime());
 +        }
 +    }
 +
 +    private static class ExpiringLivenessInfo extends LivenessInfo
 +    {
 +        private final int ttl;
 +        private final int localExpirationTime;
 +
 +        private ExpiringLivenessInfo(long timestamp, int ttl, int localExpirationTime)
 +        {
 +            super(timestamp);
 +            assert ttl != NO_TTL && localExpirationTime != NO_EXPIRATION_TIME;
 +            this.ttl = ttl;
 +            this.localExpirationTime = localExpirationTime;
 +        }
 +
 +        @Override
 +        public int ttl()
 +        {
 +            return ttl;
 +        }
 +
 +        @Override
 +        public int localExpirationTime()
 +        {
 +            return localExpirationTime;
 +        }
 +
 +        @Override
 +        public boolean isExpiring()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean isLive(int nowInSec)
 +        {
 +            return nowInSec < localExpirationTime;
 +        }
 +
 +        @Override
 +        public void digest(MessageDigest digest)
 +        {
 +            super.digest(digest);
 +            FBUtilities.updateWithInt(digest, localExpirationTime);
 +            FBUtilities.updateWithInt(digest, ttl);
 +        }
 +
 +        @Override
 +        public void validate()
 +        {
 +            if (ttl < 0)
 +                throw new MarshalException("A TTL should not be negative");
 +            if (localExpirationTime < 0)
 +                throw new MarshalException("A local expiration time should not be negative");
 +        }
 +
 +        @Override
 +        public int dataSize()
 +        {
 +            return super.dataSize()
 +                 + TypeSizes.sizeof(ttl)
 +                 + TypeSizes.sizeof(localExpirationTime);
 +
 +        }
 +
 +        @Override
 +        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +        {
 +            return new ExpiringLivenessInfo(newTimestamp, ttl, localExpirationTime);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("[ts=%d ttl=%d, let=%d]", timestamp, ttl, localExpirationTime);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index eaf6dab,d90abe9..1d54667
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -363,7 -348,13 +363,15 @@@ public class CompactionManager implemen
          }
      }
  
--    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
++                                           int jobs)
+     throws InterruptedException, ExecutionException
+     {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
++                                           final boolean reinsertOverflowedTTL, int jobs)
      throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@@ -377,7 -368,7 +385,7 @@@
              @Override
              public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
 -                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
++                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTL);
              }
          }, jobs, OperationType.SCRUB);
      }
@@@ -745,11 -736,11 +753,11 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL) throws IOException
      {
          CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTL))
          {
              scrubInfo = scrubber.getScrubInfo();
              metrics.beginCompaction(scrubInfo);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index c8e0c53,affee11..bc11504
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -35,8 -37,12 +35,9 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
 -import org.apache.cassandra.utils.ByteBufferUtil;
 -import org.apache.cassandra.utils.JVMStabilityInspector;
 -import org.apache.cassandra.utils.OutputHandler;
 -import org.apache.cassandra.utils.memory.HeapAllocator;
 -import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.*;
  import org.apache.cassandra.utils.concurrent.Refs;
++import org.apache.cassandra.utils.memory.HeapAllocator;
  
  public class Scrubber implements Closeable
  {
@@@ -45,7 -51,9 +46,8 @@@
      private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
+     private final boolean reinsertOverflowedTTLRows;
  
 -    private final CompactionController controller;
      private final boolean isCommutative;
      private final boolean isIndex;
      private final boolean checkData;
@@@ -65,38 -76,41 +67,47 @@@
      long currentRowPositionFromIndex;
      long nextRowPositionFromIndex;
  
 -    private final OutputHandler outputHandler;
+     private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
+ 
 -    private static final Comparator<Row> rowComparator = new Comparator<Row>()
 +    private final OutputHandler outputHandler;
 +
 +    private static final Comparator<Partition> partitionComparator = new Comparator<Partition>()
      {
 -         public int compare(Row r1, Row r2)
 +         public int compare(Partition r1, Partition r2)
           {
 -             return r1.key.compareTo(r2.key);
 +             return r1.partitionKey().compareTo(r2.partitionKey());
           }
      };
 -    private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
 +    private final SortedSet<Partition> outOfOrder = new TreeSet<>(partitionComparator);
  
      public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
++        this(cfs, transaction, skipCorrupted, checkData, false);
+     }
+ 
+     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
+         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
      @SuppressWarnings("resource")
 -    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
 +    public Scrubber(ColumnFamilyStore cfs,
 +                    LifecycleTransaction transaction,
 +                    boolean skipCorrupted,
 +                    OutputHandler outputHandler,
-                     boolean checkData) throws IOException
++                    boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
          this.transaction = transaction;
          this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 -
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
 +                                                                                                        sstable.descriptor.version,
 +                                                                                                        sstable.header);
- 
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
          // Calculate the expected compacted filesize
@@@ -134,19 -150,15 +145,22 @@@
  
          this.currentRowPositionFromIndex = 0;
          this.nextRowPositionFromIndex = 0;
+ 
+         if (reinsertOverflowedTTLRows)
+             outputHandler.output("Starting scrub with reinsert overflowed TTL option");
      }
  
 +    private UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
 +    {
 +        return checkData ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
 +    }
 +
      public void scrub()
      {
 +        List<SSTableReader> finished = new ArrayList<>();
 +        boolean completed = false;
          outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
 -        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, transaction.isOffline());
 +        try (SSTableRewriter writer = SSTableRewriter.construct(cfs, transaction, false, sstable.maxDataAge, transaction.isOffline());
               Refs<SSTableReader> refs = Refs.ref(Collections.singleton(sstable)))
          {
              nextIndexKey = indexAvailable() ? ByteBufferUtil.readWithShortLength(indexFile) : null;
@@@ -285,57 -311,66 +299,71 @@@
          }
          finally
          {
 -            controller.close();
 -            if (transaction.isOffline() && newSstable != null)
 -                newSstable.selfRef().release();
 +            if (transaction.isOffline())
 +                finished.forEach(sstable -> sstable.selfRef().release());
          }
  
 -        if (newSstable == null)
 -        {
 -            if (badRows > 0)
 -                outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
 -            else
 -                outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
 -        }
 -        else
 +        if (completed)
          {
+             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+             if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                 outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
              if (badRows > 0)
-                 outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
-             else
-                 outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
+                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
          }
 +        else
 +        {
-             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
 +            if (badRows > 0)
-                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
++                outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
++            else
++                outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
 +        }
      }
  
      @SuppressWarnings("resource")
      private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, SSTableRewriter writer)
      {
 -        // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
 -        // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
 -        // to the outOfOrderRows that will be later written to a new SSTable.
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
 -                                                              cfs.metadata.comparator.onDiskAtomComparator());
 -        if (prevKey != null && prevKey.compareTo(key) > 0)
 -        {
 -            saveOutOfOrderRow(prevKey, key, atoms);
 -            return false;
 -        }
 +        // OrderCheckerIterator will check, at iteration time, that the rows are in the proper order. If it detects
 +        // that one row is out of order, it will stop returning them. The remaining rows will be sorted and added
 +        // to the outOfOrder set that will be later written to a new SSTable.
-         OrderCheckerIterator sstableIterator = new OrderCheckerIterator(new RowMergingSSTableIterator(sstable, dataFile, key),
++        OrderCheckerIterator sstableIterator = new OrderCheckerIterator(getIterator(key),
 +                                                                        cfs.metadata.comparator);
  
 -        AbstractCompactedRow compactedRow = new LazilyCompactedRow(controller, Collections.singletonList(atoms));
 -        if (writer.tryAppend(compactedRow) == null)
 -            emptyRows++;
 -        else
 -            goodRows++;
 +        try (UnfilteredRowIterator iterator = withValidation(sstableIterator, dataFile.getPath()))
 +        {
 +            if (prevKey != null && prevKey.compareTo(key) > 0)
 +            {
 +                saveOutOfOrderRow(prevKey, key, iterator);
 +                return false;
 +            }
  
 -        if (atoms.hasOutOfOrderCells())
 -            saveOutOfOrderRow(key, atoms);
 +            if (writer.tryAppend(iterator) == null)
 +                emptyRows++;
 +            else
 +                goodRows++;
 +        }
 +
 +        if (sstableIterator.hasRowsOutOfOrder())
 +        {
 +            outputHandler.warn(String.format("Out of order rows found in partition: %s", key));
 +            outOfOrder.add(sstableIterator.getRowsOutOfOrder());
 +        }
  
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key)
++    private UnfilteredRowIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
 -        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
++        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(sstable, dataFile, key);
++        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(rowMergingIterator,
+                                                                                     outputHandler,
 -                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
++                                                                                    negativeLocalDeletionInfoMetrics) : rowMergingIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;
@@@ -477,49 -555,12 +505,55 @@@
          }
      }
  
+     public class NegativeLocalDeletionInfoMetrics
+     {
+         public volatile int fixedRows = 0;
+     }
+ 
      /**
 +     * During 2.x migration, under some circumstances rows might have gotten duplicated.
 +     * Merging iterator merges rows with same clustering.
 +     *
 +     * For more details, refer to CASSANDRA-12144.
 +     */
 +    private static class RowMergingSSTableIterator extends SSTableIdentityIterator
 +    {
 +        RowMergingSSTableIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key)
 +        {
 +            super(sstable, file, key);
 +        }
 +
 +        @Override
 +        protected Unfiltered doCompute()
 +        {
 +            if (!iterator.hasNext())
 +                return endOfData();
 +
 +            Unfiltered next = iterator.next();
 +            if (!next.isRow())
 +                return next;
 +
 +            while (iterator.hasNext())
 +            {
 +                Unfiltered peek = iterator.peek();
 +                // If there was a duplicate row, merge it.
 +                if (next.clustering().equals(peek.clustering()) && peek.isRow())
 +                {
 +                    iterator.next(); // Make sure that the peeked item was consumed.
 +                    next = Rows.merge((Row) next, (Row) peek, FBUtilities.nowInSeconds());
 +                }
 +                else
 +                {
 +                    break;
 +                }
 +            }
 +
 +            return next;
 +        }
++
 +    }
 +
 +    /**
       * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
       * cells order and collect the out of order cells to correct the problem.
       */
@@@ -571,53 -608,98 +605,199 @@@
          }
  
          @Override
 -        protected OnDiskAtom computeNext()
 +        public boolean isEmpty()
          {
 -            if (!iterator.hasNext())
 -                return endOfData();
 +            return iterator.isEmpty();
 +        }
  
 -            OnDiskAtom next = iterator.next();
 +        public void close()
 +        {
 +            iterator.close();
 +        }
  
 -            // If we detect that some cells are out of order we will store and sort the remaining once to insert them
 -            // in a separate SSTable.
 -            if (previous != null && comparator.compare(next, previous) < 0)
 -            {
 -                outOfOrderCells = collectOutOfOrderCells(next, iterator);
 -                return endOfData();
 -            }
 -            previous = next;
 -            return next;
 +        public DeletionTime partitionLevelDeletion()
 +        {
 +            return iterator.partitionLevelDeletion();
          }
  
 -        public boolean hasOutOfOrderCells()
 +        public EncodingStats stats()
          {
 -            return outOfOrderCells != null;
 +            return iterator.stats();
          }
  
 -        public ColumnFamily getOutOfOrderCells()
 +        public boolean hasRowsOutOfOrder()
          {
 -            return outOfOrderCells;
 +            return rowsOutOfOrder != null;
          }
  
 -        private static ColumnFamily collectOutOfOrderCells(OnDiskAtom atom, OnDiskAtomIterator iterator)
 +        public Partition getRowsOutOfOrder()
          {
 -            ColumnFamily cf = iterator.getColumnFamily().cloneMeShallow(ArrayBackedSortedColumns.factory, false);
 -            cf.addAtom(atom);
 -            while (iterator.hasNext())
 -                cf.addAtom(iterator.next());
 -            return cf;
 +            return rowsOutOfOrder;
 +        }
 +
 +        protected Unfiltered computeNext()
 +        {
 +            if (!iterator.hasNext())
 +                return endOfData();
 +
 +            Unfiltered next = iterator.next();
 +
 +            // If we detect that some rows are out of order we will store and sort the remaining ones to insert them
 +            // in a separate SSTable.
 +            if (previous != null && comparator.compare(next, previous) < 0)
 +            {
 +                rowsOutOfOrder = ImmutableBTreePartition.create(UnfilteredRowIterators.concat(next, iterator), false);
 +                return endOfData();
 +            }
 +            previous = next;
 +            return next;
          }
+     }
+ 
+     /**
 -     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
++     * This iterator converts negative {@link AbstractCell#localDeletionTime()} into {@link AbstractCell#MAX_DELETION_TIME}
+      *
+      * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+      */
 -    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
++    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
+     {
+         /**
+          * The decorated iterator.
+          */
 -        private final OnDiskAtomIterator iterator;
++        private final UnfilteredRowIterator iterator;
+ 
+         private final OutputHandler outputHandler;
+         private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+ 
 -        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
++        public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, OutputHandler outputHandler,
+                                                     NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+         {
+             this.iterator = iterator;
+             this.outputHandler = outputHandler;
+             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+         }
+ 
 -        public ColumnFamily getColumnFamily()
++        public CFMetaData metadata()
+         {
 -            return iterator.getColumnFamily();
++            return iterator.metadata();
+         }
+ 
 -        public DecoratedKey getKey()
++        public boolean isReverseOrder()
+         {
 -            return iterator.getKey();
++            return iterator.isReverseOrder();
+         }
+ 
 -        public void close() throws IOException
++        public PartitionColumns columns()
+         {
 -            iterator.close();
++            return iterator.columns();
++        }
++
++        public DecoratedKey partitionKey()
++        {
++            return iterator.partitionKey();
++        }
++
++        public Row staticRow()
++        {
++            return iterator.staticRow();
+         }
+ 
+         @Override
 -        protected OnDiskAtom computeNext()
++        public boolean isEmpty()
++        {
++            return iterator.isEmpty();
++        }
++
++        public void close()
++        {
++            iterator.close();
++        }
++
++        public DeletionTime partitionLevelDeletion()
++        {
++            return iterator.partitionLevelDeletion();
++        }
++
++        public EncodingStats stats()
++        {
++            return iterator.stats();
++        }
++
++        protected Unfiltered computeNext()
+         {
+             if (!iterator.hasNext())
+                 return endOfData();
+ 
 -            OnDiskAtom next = iterator.next();
++            Unfiltered next = iterator.next();
++            if (!next.isRow())
++                return next;
+ 
 -            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
++            if (hasNegativeLocalExpirationTime((Row) next))
+             {
 -                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
++                outputHandler.debug(String.format("Found row with negative local expiration time: %s", next.toString(metadata(), false)));
+                 negativeLocalExpirationTimeMetrics.fixedRows++;
 -                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
++                return fixNegativeLocalExpirationTime((Row) next);
+             }
+ 
+             return next;
+         }
++
++        private boolean hasNegativeLocalExpirationTime(Row next)
++        {
++            Row row = next;
++            if (row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0)
++            {
++                return true;
++            }
++
++            for (ColumnData cd : row)
++            {
++                if (cd.column().isSimple())
++                {
++                    Cell cell = (Cell)cd;
++                    if (cell.isExpiring() && cell.localDeletionTime() < 0)
++                        return true;
++                }
++                else
++                {
++                    ComplexColumnData complexData = (ComplexColumnData)cd;
++                    for (Cell cell : complexData)
++                    {
++                        if (cell.isExpiring() && cell.localDeletionTime() < 0)
++                            return true;
++                    }
++                }
++            }
++
++            return false;
++        }
 +
++        private Unfiltered fixNegativeLocalExpirationTime(Row row)
++        {
++            Row.Builder builder = HeapAllocator.instance.cloningBTreeRowBuilder();
++            builder.newRow(row.clustering());
++            builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0 ?
++                                              row.primaryKeyLivenessInfo().withUpdatedTimestampAndLocalDeletionTime(row.primaryKeyLivenessInfo().timestamp() + 1, AbstractCell.MAX_DELETION_TIME)
++                                              :row.primaryKeyLivenessInfo());
++            builder.addRowDeletion(row.deletion());
++            for (ColumnData cd : row)
++            {
++                if (cd.column().isSimple())
++                {
++                    Cell cell = (Cell)cd;
++                    builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
++                }
++                else
++                {
++                    ComplexColumnData complexData = (ComplexColumnData)cd;
++                    builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
++                    for (Cell cell : complexData)
++                    {
++                        builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
++                    }
++                }
++            }
++            return builder.build();
++        }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BufferCell.java
index 82ae02c,0000000..df2619c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@@ -1,365 -1,0 +1,370 @@@
 +/*
 + * 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.io.IOException;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.ByteType;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +
 +public class BufferCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnDefinition.regularDef("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
 +
 +    private final long timestamp;
 +    private final int ttl;
 +    private final int localDeletionTime;
 +
 +    private final ByteBuffer value;
 +    private final CellPath path;
 +
 +    public BufferCell(ColumnDefinition column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
 +    {
 +        super(column);
 +        assert column.isComplex() == (path != null);
 +        this.timestamp = timestamp;
 +        this.ttl = ttl;
 +        this.localDeletionTime = localDeletionTime;
 +        this.value = value;
 +        this.path = path;
 +    }
 +
 +    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value)
 +    {
 +        return live(metadata, column, timestamp, value, null);
 +    }
 +
 +    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value, CellPath path)
 +    {
 +        if (metadata.params.defaultTimeToLive != NO_TTL)
 +            return expiring(column, timestamp, metadata.params.defaultTimeToLive, FBUtilities.nowInSeconds(), value, path);
 +
 +        return new BufferCell(column, timestamp, NO_TTL, NO_DELETION_TIME, value, path);
 +    }
 +
 +    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value)
 +    {
 +        return expiring(column, timestamp, ttl, nowInSec, value, null);
 +    }
 +
 +    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
 +    {
 +        assert ttl != NO_TTL;
-         return new BufferCell(column, timestamp, ttl, nowInSec + ttl, value, path);
++        return new BufferCell(column, timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), value, path);
 +    }
 +
 +    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec)
 +    {
 +        return tombstone(column, timestamp, nowInSec, null);
 +    }
 +
 +    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec, CellPath path)
 +    {
 +        return new BufferCell(column, timestamp, NO_TTL, nowInSec, ByteBufferUtil.EMPTY_BYTE_BUFFER, path);
 +    }
 +
 +    public boolean isCounterCell()
 +    {
 +        return !isTombstone() && column.isCounterColumn();
 +    }
 +
 +    public boolean isLive(int nowInSec)
 +    {
 +        return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && nowInSec < localDeletionTime);
 +    }
 +
 +    public boolean isTombstone()
 +    {
 +        return localDeletionTime != NO_DELETION_TIME && ttl == NO_TTL;
 +    }
 +
 +    public boolean isExpiring()
 +    {
 +        return ttl != NO_TTL;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return timestamp;
 +    }
 +
 +    public int ttl()
 +    {
 +        return ttl;
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return localDeletionTime;
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        return value;
 +    }
 +
 +    public CellPath path()
 +    {
 +        return path;
 +    }
 +
 +    public Cell withUpdatedColumn(ColumnDefinition newColumn)
 +    {
 +        return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);
 +    }
 +
++    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return new BufferCell(column, newTimestamp, ttl, newLocalDeletionTime, value, path);
++    }
++
 +    public Cell copy(AbstractAllocator allocator)
 +    {
 +        if (!value.hasRemaining())
 +            return this;
 +
 +        return new BufferCell(column, timestamp, ttl, localDeletionTime, allocator.clone(value), path == null ? null : path.copy(allocator));
 +    }
 +
 +    public Cell markCounterLocalToBeCleared()
 +    {
 +        if (!isCounterCell())
 +            return this;
 +
 +        ByteBuffer marked = CounterContext.instance().markLocalToBeCleared(value());
 +        return marked == value() ? this : new BufferCell(column, timestamp, ttl, localDeletionTime, marked, path);
 +    }
 +
 +    public Cell purge(DeletionPurger purger, int nowInSec)
 +    {
 +        if (!isLive(nowInSec))
 +        {
 +            if (purger.shouldPurge(timestamp, localDeletionTime))
 +                return null;
 +
 +            // We slightly hijack purging to convert expired but not purgeable columns to tombstones. The reason we do that is
 +            // that once a column has expired it is equivalent to a tombstone but actually using a tombstone is more compact since
 +            // we don't keep the column value. The reason we do it here is that 1) it's somewhat related to dealing with tombstones
 +            // so hopefully not too surprising and 2) we want to this and purging at the same places, so it's simpler/more efficient
 +            // to do both here.
 +            if (isExpiring())
 +            {
 +                // Note that as long as the expiring column and the tombstone put together live longer than GC grace seconds,
 +                // we'll fulfil our responsibility to repair. See discussion at
 +                // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
 +                return BufferCell.tombstone(column, timestamp, localDeletionTime - ttl, path).purge(purger, nowInSec);
 +            }
 +        }
 +        return this;
 +    }
 +
 +    public Cell updateAllTimestamp(long newTimestamp)
 +    {
 +        return new BufferCell(column, isTombstone() ? newTimestamp - 1 : newTimestamp, ttl, localDeletionTime, value, path);
 +    }
 +
 +    public int dataSize()
 +    {
 +        return TypeSizes.sizeof(timestamp)
 +             + TypeSizes.sizeof(ttl)
 +             + TypeSizes.sizeof(localDeletionTime)
 +             + value.remaining()
 +             + (path == null ? 0 : path.dataSize());
 +    }
 +
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value) + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
 +    }
 +
 +    /**
 +     * The serialization format for cell is:
 +     *     [ flags ][ timestamp ][ deletion time ][    ttl    ][ path size ][ path ][ value size ][ value ]
 +     *     [   1b  ][ 8b (vint) ][   4b (vint)   ][ 4b (vint) ][ 4b (vint) ][  arb ][  4b (vint) ][  arb  ]
 +     *
 +     * where not all field are always present (in fact, only the [ flags ] are guaranteed to be present). The fields have the following
 +     * meaning:
 +     *   - [ flags ] is the cell flags. It is a byte for which each bit represents a flag whose meaning is explained below (*_MASK constants)
 +     *   - [ timestamp ] is the cell timestamp. Present unless the cell has the USE_TIMESTAMP_MASK.
 +     *   - [ deletion time]: the local deletion time for the cell. Present if either the cell is deleted (IS_DELETED_MASK)
 +     *       or it is expiring (IS_EXPIRING_MASK) but doesn't have the USE_ROW_TTL_MASK.
 +     *   - [ ttl ]: the ttl for the cell. Present if the row is expiring (IS_EXPIRING_MASK) but doesn't have the
 +     *       USE_ROW_TTL_MASK.
 +     *   - [ value size ] is the size of the [ value ] field. It's present unless either the cell has the HAS_EMPTY_VALUE_MASK, or the value
 +     *       for columns of this type have a fixed length.
 +     *   - [ path size ] is the size of the [ path ] field. Present iff this is the cell of a complex column.
 +     *   - [ value ]: the cell value, unless it has the HAS_EMPTY_VALUE_MASK.
 +     *   - [ path ]: the cell path if the column this is a cell of is complex.
 +     */
 +    static class Serializer implements Cell.Serializer
 +    {
 +        private final static int IS_DELETED_MASK             = 0x01; // Whether the cell is a tombstone or not.
 +        private final static int IS_EXPIRING_MASK            = 0x02; // Whether the cell is expiring.
 +        private final static int HAS_EMPTY_VALUE_MASK        = 0x04; // Wether the cell has an empty value. This will be the case for tombstone in particular.
 +        private final static int USE_ROW_TIMESTAMP_MASK      = 0x08; // Wether the cell has the same timestamp than the row this is a cell of.
 +        private final static int USE_ROW_TTL_MASK            = 0x10; // Wether the cell has the same ttl than the row this is a cell of.
 +
 +        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException
 +        {
 +            assert cell != null;
 +            boolean hasValue = cell.value().hasRemaining();
 +            boolean isDeleted = cell.isTombstone();
 +            boolean isExpiring = cell.isExpiring();
 +            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
 +            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
 +            int flags = 0;
 +            if (!hasValue)
 +                flags |= HAS_EMPTY_VALUE_MASK;
 +
 +            if (isDeleted)
 +                flags |= IS_DELETED_MASK;
 +            else if (isExpiring)
 +                flags |= IS_EXPIRING_MASK;
 +
 +            if (useRowTimestamp)
 +                flags |= USE_ROW_TIMESTAMP_MASK;
 +            if (useRowTTL)
 +                flags |= USE_ROW_TTL_MASK;
 +
 +            out.writeByte((byte)flags);
 +
 +            if (!useRowTimestamp)
 +                header.writeTimestamp(cell.timestamp(), out);
 +
 +            if ((isDeleted || isExpiring) && !useRowTTL)
 +                header.writeLocalDeletionTime(cell.localDeletionTime(), out);
 +            if (isExpiring && !useRowTTL)
 +                header.writeTTL(cell.ttl(), out);
 +
 +            if (column.isComplex())
 +                column.cellPathSerializer().serialize(cell.path(), out);
 +
 +            if (hasValue)
 +                header.getType(column).writeValue(cell.value(), out);
 +        }
 +
 +        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException
 +        {
 +            int flags = in.readUnsignedByte();
 +            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
 +            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
 +            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
 +            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
 +            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
 +
 +            long timestamp = useRowTimestamp ? rowLiveness.timestamp() : header.readTimestamp(in);
 +
 +            int localDeletionTime = useRowTTL
 +                                  ? rowLiveness.localExpirationTime()
 +                                  : (isDeleted || isExpiring ? header.readLocalDeletionTime(in) : NO_DELETION_TIME);
 +
 +            int ttl = useRowTTL ? rowLiveness.ttl() : (isExpiring ? header.readTTL(in) : NO_TTL);
 +
 +            CellPath path = column.isComplex()
 +                          ? column.cellPathSerializer().deserialize(in)
 +                          : null;
 +
 +            boolean isCounter = localDeletionTime == NO_DELETION_TIME && column.type.isCounter();
 +
 +            ByteBuffer value = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +            if (hasValue)
 +            {
 +                if (helper.canSkipValue(column) || (path != null && helper.canSkipValue(path)))
 +                {
 +                    header.getType(column).skipValue(in);
 +                }
 +                else
 +                {
 +                    value = header.getType(column).readValue(in, DatabaseDescriptor.getMaxValueSize());
 +                    if (isCounter)
 +                        value = helper.maybeClearCounterValue(value);
 +                }
 +            }
 +
 +            return new BufferCell(column, timestamp, ttl, localDeletionTime, value, path);
 +        }
 +
 +        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header)
 +        {
 +            long size = 1; // flags
 +            boolean hasValue = cell.value().hasRemaining();
 +            boolean isDeleted = cell.isTombstone();
 +            boolean isExpiring = cell.isExpiring();
 +            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
 +            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
 +
 +            if (!useRowTimestamp)
 +                size += header.timestampSerializedSize(cell.timestamp());
 +
 +            if ((isDeleted || isExpiring) && !useRowTTL)
 +                size += header.localDeletionTimeSerializedSize(cell.localDeletionTime());
 +            if (isExpiring && !useRowTTL)
 +                size += header.ttlSerializedSize(cell.ttl());
 +
 +            if (column.isComplex())
 +                size += column.cellPathSerializer().serializedSize(cell.path());
 +
 +            if (hasValue)
 +                size += header.getType(column).writtenLength(cell.value());
 +
 +            return size;
 +        }
 +
 +        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
 +        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException
 +        {
 +            int flags = in.readUnsignedByte();
 +            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
 +            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
 +            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
 +            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
 +            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
 +
 +            if (!useRowTimestamp)
 +                header.skipTimestamp(in);
 +
 +            if (!useRowTTL && (isDeleted || isExpiring))
 +                header.skipLocalDeletionTime(in);
 +
 +            if (!useRowTTL && isExpiring)
 +                header.skipTTL(in);
 +
 +            if (column.isComplex())
 +                column.cellPathSerializer().skip(in);
 +
 +            if (hasValue)
 +                header.getType(column).skipValue(in);
 +
 +            return true;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/Cell.java
index d10cc74,0000000..c69e11f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@@ -1,157 -1,0 +1,166 @@@
 +/*
 + * 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.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Comparator;
 +
++import com.google.common.annotations.VisibleForTesting;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.Attributes;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +
 +/**
 + * A cell is our atomic unit for a single value of a single column.
 + * <p>
 + * A cell always holds at least a timestamp that gives us how the cell reconcile. We then
 + * have 3 main types of cells:
 + *   1) live regular cells: those will also have a value and, if for a complex column, a path.
 + *   2) expiring cells: on top of regular cells, those have a ttl and a local deletion time (when they are expired).
 + *   3) tombstone cells: those won't have value, but they have a local deletion time (when the tombstone was created).
 + */
 +public abstract class Cell extends ColumnData
 +{
 +    public static final int NO_TTL = 0;
 +    public static final int NO_DELETION_TIME = Integer.MAX_VALUE;
++    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
 +
 +    public final static Comparator<Cell> comparator = (c1, c2) ->
 +    {
 +        int cmp = c1.column().compareTo(c2.column());
 +        if (cmp != 0)
 +            return cmp;
 +
 +        Comparator<CellPath> pathComparator = c1.column().cellPathComparator();
 +        return pathComparator == null ? 0 : pathComparator.compare(c1.path(), c2.path());
 +    };
 +
 +    public static final Serializer serializer = new BufferCell.Serializer();
 +
 +    protected Cell(ColumnDefinition column)
 +    {
 +        super(column);
 +    }
 +
 +    /**
 +     * Whether the cell is a counter cell or not.
 +     *
 +     * @return whether the cell is a counter cell or not.
 +     */
 +    public abstract boolean isCounterCell();
 +
 +    /**
 +     * The cell value.
 +     *
 +     * @return the cell value.
 +     */
 +    public abstract ByteBuffer value();
 +
 +    /**
 +     * The cell timestamp.
 +     * <p>
 +     * @return the cell timestamp.
 +     */
 +    public abstract long timestamp();
 +
 +    /**
 +     * The cell ttl.
 +     *
 +     * @return the cell ttl, or {@code NO_TTL} if the cell isn't an expiring one.
 +     */
 +    public abstract int ttl();
 +
 +    /**
 +     * The cell local deletion time.
 +     *
 +     * @return the cell local deletion time, or {@code NO_DELETION_TIME} if the cell is neither
 +     * a tombstone nor an expiring one.
 +     */
 +    public abstract int localDeletionTime();
 +
 +    /**
 +     * Whether the cell is a tombstone or not.
 +     *
 +     * @return whether the cell is a tombstone or not.
 +     */
 +    public abstract boolean isTombstone();
 +
 +    /**
 +     * Whether the cell is an expiring one or not.
 +     * <p>
 +     * Note that this only correspond to whether the cell liveness info
 +     * have a TTL or not, but doesn't tells whether the cell is already expired
 +     * or not. You should use {@link #isLive} for that latter information.
 +     *
 +     * @return whether the cell is an expiring one or not.
 +     */
 +    public abstract boolean isExpiring();
 +
 +    /**
 +     * Whether the cell is live or not given the current time.
 +     *
 +     * @param nowInSec the current time in seconds. This is used to
 +     * decide if an expiring cell is expired or live.
 +     * @return whether the cell is live or not at {@code nowInSec}.
 +     */
 +    public abstract boolean isLive(int nowInSec);
 +
 +    /**
 +     * For cells belonging to complex types (non-frozen collection and UDT), the
 +     * path to the cell.
 +     *
 +     * @return the cell path for cells of complex column, and {@code null} for other cells.
 +     */
 +    public abstract CellPath path();
 +
 +    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
 +
 +    public abstract Cell withUpdatedValue(ByteBuffer newValue);
 +
++    public abstract Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
++
 +    public abstract Cell copy(AbstractAllocator allocator);
 +
 +    @Override
 +    // Overrides super type to provide a more precise return type.
 +    public abstract Cell markCounterLocalToBeCleared();
 +
 +    @Override
 +    // Overrides super type to provide a more precise return type.
 +    public abstract Cell purge(DeletionPurger purger, int nowInSec);
 +
 +    public interface Serializer
 +    {
 +        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException;
 +
 +        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException;
 +
 +        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header);
 +
 +        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
 +        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index e5a50dd,2c9ac4d..cf8e257
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2705,22 -2599,28 +2705,27 @@@ public class StorageService extends Not
          return status.statusCode;
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
 +        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
 +        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, tables);
+     }
+ 
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
 -                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
 -            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
++            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, reinsertOverflowedTTL, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7344ca8,f336bcc..10d47f7
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -262,14 -262,17 +262,17 @@@ public interface StorageServiceMBean ex
       * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
       */
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+ 
      /**
       * Verify (checksums of) the given keyspace.
 -     * If columnFamilies array is empty, all CFs are verified.
 +     * If tableNames array is empty, all CFs are verified.
       *
       * The entire sstable will be read to ensure each cell validates if extendedVerify is true
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 6ad791d,8bdf9dc..2ab0330
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -332,9 -315,9 +332,9 @@@ public class ThriftValidatio
              if (isCommutative)
                  throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
 -            validateColumnData(metadata, key, null, cosc.column);
 +            validateColumnData(metadata, null, cosc.column);
          }
  
          if (cosc.super_column != null)
@@@ -374,11 -357,13 +374,13 @@@
              if (column.ttl <= 0)
                  throw new org.apache.cassandra.exceptions.InvalidRequestException("ttl must be positive");
  
 -            if (column.ttl > ExpiringCell.MAX_TTL)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
 -            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
 +            if (column.ttl > Attributes.MAX_TTL)
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, Attributes.MAX_TTL));
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
          }
          else
          {
 -            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
              // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
              assert column.ttl == 0;
          }
@@@ -450,9 -435,9 +452,9 @@@
      /**
       * Validates the data part of the column (everything in the column object but the name, which is assumed to be valid)
       */
 -    public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
 +    public static void validateColumnData(CFMetaData metadata, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
      {
-         validateTtl(column);
+         validateTtl(metadata, column);
          if (!column.isSetValue())
              throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
          if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 172b505,17bef02..0d3c078
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -235,35 -228,27 +235,35 @@@ public class NodeProbe implements AutoC
  
      public void close() throws IOException
      {
 -        jmxc.close();
 +        try
 +        {
 +            jmxc.close();
 +        }
 +        catch (ConnectException e)
 +        {
 +            // result of 'stopdaemon' command - i.e. if close() call fails, the daemon is shutdown
 +            System.out.println("Cassandra has shutdown.");
 +        }
      }
  
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
 +        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, tables);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables);
 -        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
++        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTL, jobs, keyspaceName, tables);
      }
  
 -    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
      }
  
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
 +        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames);
      }
  
      private void checkJobs(PrintStream out, int jobs)
@@@ -288,19 -267,13 +288,19 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         switch (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables))
 -        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
++        switch (ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTL, jobs, keyspaceName, tables))
          {
 -            failed = true;
 -            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +            case 1:
 +                failed = true;
 +                out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +                break;
 +            case 2:
 +                failed = true;
 +                out.println("Failed marking some sstables compacting in keyspace "+keyspaceName+", check server logs for more information");
 +                break;
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 6076e32,8319014..19af957
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@@ -90,12 -65,13 +90,13 @@@ public class SSTableMetadataViewe
                  {
                      out.printf("Minimum timestamp: %s%n", stats.minTimestamp);
                      out.printf("Maximum timestamp: %s%n", stats.maxTimestamp);
++                    out.printf("SSTable min local deletion time: %s%n", stats.minLocalDeletionTime);
                      out.printf("SSTable max local deletion time: %s%n", stats.maxLocalDeletionTime);
                      out.printf("Compression ratio: %s%n", stats.compressionRatio);
 -                    out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
 +                    out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000) - gcgs));
                      out.printf("SSTable Level: %d%n", stats.sstableLevel);
                      out.printf("Repaired at: %d%n", stats.repairedAt);
 -                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
 -                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
 +                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
                      out.println("Estimated tombstone drop times:");
                      for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 4249430,f5e84c5..4778d72
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -122,7 -129,7 +129,7 @@@ public class StandaloneScrubbe
                      try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
                          txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
 -                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinserOverflowedTTL))
                          {
                              scrubber.scrub();
                          }
@@@ -199,6 -207,7 +206,7 @@@
          public boolean manifestCheckOnly;
          public boolean skipCorrupted;
          public boolean noValidate;
 -        public boolean reinsertOverflowedTTL;
++        public boolean reinserOverflowedTTL;
  
          private Options(String keyspaceName, String cfName)
          {
@@@ -239,6 -248,7 +247,7 @@@
                  opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                  opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                  opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
 -                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
++                opts.reinserOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
  
                  return opts;
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 2345a85,50224a0..ead2fd4
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -48,11 -49,16 +49,16 @@@ public class Scrub extends NodeToolCm
                     description = "Do not validate columns using column validator")
      private boolean noValidation = false;
  
 -    @Option(title = "jobs",
 -            name = {"-j", "--jobs"},
 -            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 -    private int jobs = 2;
 -
+     @Option(title = "reinsert_overflowed_ttl",
+     name = {"r", "--reinsert-overflowed-ttl"},
+     description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+     private boolean reinsertOverflowedTTL = false;
+ 
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
      @Override
      public void execute(NodeProbe probe)
      {
@@@ -63,13 -69,11 +69,13 @@@
          {
              try
              {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, tableNames);
 -                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 -            } catch (IllegalArgumentException e)
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, tableNames);
 +            }
 +            catch (IllegalArgumentException e)
              {
                  throw e;
 -            } catch (Exception e)
 +            }
 +            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during scrubbing", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-CompressionInfo.db
index 0000000,0000000..d759cec
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
index 0000000,0000000..e7a72da
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
index 0000000,0000000..a3c633a
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++203700622

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Index.db
index 0000000,0000000..d742724
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
index 0000000,0000000..faf367b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
index 0000000,0000000..45113dc
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++CompressionInfo.db
++Data.db
++Summary.db
++Filter.db
++Statistics.db
++TOC.txt
++Digest.crc32
++Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-CompressionInfo.db
index 0000000,0000000..1759c09
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
index 0000000,0000000..c1de572
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
index 0000000,0000000..0403b5b
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++82785930

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Index.db
index 0000000,0000000..a0477eb
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
index 0000000,0000000..e9d6577
new file mode 100644
Binary files differ


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


[28/29] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by pa...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/trunk
Commit: 0a6b6f506b012d4d491757d6216c5fa1c53bedc9
Parents: 6eb65e5 c231ed5
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:58:13 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:59:28 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 +++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 ++
 conf/jvm.options                                |  11 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  15 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../db/ExpirationDateOverflowHandling.java      | 121 +++++++
 .../org/apache/cassandra/db/LegacyLayout.java   |   6 +-
 .../org/apache/cassandra/db/LivenessInfo.java   |  13 +-
 .../db/compaction/CompactionManager.java        |  16 +-
 .../cassandra/db/compaction/Scrubber.java       | 225 +++++++++++--
 .../apache/cassandra/db/rows/BufferCell.java    |   8 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |  10 +
 .../apache/cassandra/db/rows/NativeCell.java    |   5 +
 .../cassandra/service/StorageService.java       |   7 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |   6 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   7 +-
 .../table1/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/mc-1-big-Data.db                     | Bin 0 -> 58 bytes
 .../table1/mc-1-big-Digest.crc32                |   1 +
 .../table1/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Statistics.db               | Bin 0 -> 4676 bytes
 .../table1/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table1/mc-1-big-TOC.txt                     |   8 +
 .../table2/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/mc-1-big-Data.db                     | Bin 0 -> 60 bytes
 .../table2/mc-1-big-Digest.crc32                |   1 +
 .../table2/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Statistics.db               | Bin 0 -> 4686 bytes
 .../table2/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table2/mc-1-big-TOC.txt                     |   8 +
 .../table3/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/mc-1-big-Data.db                     | Bin 0 -> 111 bytes
 .../table3/mc-1-big-Digest.crc32                |   1 +
 .../table3/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Statistics.db               | Bin 0 -> 4732 bytes
 .../table3/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table3/mc-1-big-TOC.txt                     |   8 +
 .../table4/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/mc-1-big-Data.db                     | Bin 0 -> 108 bytes
 .../table4/mc-1-big-Digest.crc32                |   1 +
 .../table4/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Statistics.db               | Bin 0 -> 4742 bytes
 .../table4/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table4/mc-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 327 ++++++++++++++++++-
 test/unit/org/apache/cassandra/db/CellTest.java |  22 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 58 files changed, 893 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c38b69b,a492c42..5b49f48
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -48,26 -28,12 +48,27 @@@ Merged from 2.2
   * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
   * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
   * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
 + * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
  Merged from 2.1:
 - * More PEP8 compliance for cqlsh (CASSANDRA-14021)
++ * Protect against overflow of local expiration time (CASSANDRA-14092)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 -
 -
 -3.0.15
 + * More PEP8 compiance for cqlsh (CASSANDRA-14021)
 +
 +
 +3.11.1
 + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808)
 + * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869)
 + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418)
 + * BTree.Builder memory leak (CASSANDRA-13754)
 + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798)
 + * Add a skip read validation flag to cassandra-stress (CASSANDRA-13772)
 + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
   * Improve TRUNCATE performance (CASSANDRA-13909)
   * Implement short read protection on partition boundaries (CASSANDRA-13595)
   * Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries (CASSANDRA-13911)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index f4b15e7,f574c33..fb1dafe
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,22 -38,14 +38,23 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
+    - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 -   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 -     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 -     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 -     options are used. See CASSANDRA-13006 for more details.
 -   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 -     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 -     for more details.
 +    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +      options are used. See CASSANDRA-13006 for more details
 +    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +      for more details.
 +    - Upgrades from 3.0 might have produced unnecessary schema migrations while
 +      there was at least one 3.0 node in the cluster. It is therefore highly
 +      recommended to upgrade from 3.0 to at least 3.11.2. The root cause of
 +      this schema mismatch was a difference in the way how schema digests were computed
 +      in 3.0 and 3.11.2. To mitigate this issue, 3.11.2 and newer announce
 +      3.0 compatible digests as long as there is at least one 3.0 node in the
 +      cluster. Once all nodes have been upgraded, the "real" schema version will be
 +      announced. Note: this fix is only necessary in 3.11.2 and therefore only applies
 +      to 3.11. (CASSANDRA-14109)
  
  Materialized Views
  -------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/conf/jvm.options
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/redhat/cassandra.spec
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index d915560,832d0a7..d4e230f
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -20,8 -20,9 +20,10 @@@ package org.apache.cassandra.cql3
  import java.nio.ByteBuffer;
  import java.util.List;
  
+ import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.functions.Function;
+ import org.apache.cassandra.db.ExpirationDateOverflowHandling;
 +import org.apache.cassandra.db.LivenessInfo;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
@@@ -105,11 -109,11 +110,11 @@@ public class Attribute
  
          ByteBuffer tval = timeToLive.bindAndGet(options);
          if (tval == null)
 -            throw new InvalidRequestException("Invalid null value of TTL");
 -
 -        if (tval == ByteBufferUtil.UNSET_BYTE_BUFFER) // treat as unlimited
              return 0;
  
 +        if (tval == ByteBufferUtil.UNSET_BYTE_BUFFER)
-             return defaultTimeToLive;
++            return metadata.params.defaultTimeToLive;
 +
          try
          {
              Int32Type.instance.validate(tval);
@@@ -126,9 -130,8 +131,11 @@@
          if (ttl > MAX_TTL)
              throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, MAX_TTL));
  
-         if (defaultTimeToLive != LivenessInfo.NO_TTL && ttl == LivenessInfo.NO_TTL)
++        if (metadata.params.defaultTimeToLive != LivenessInfo.NO_TTL && ttl == LivenessInfo.NO_TTL)
 +            return LivenessInfo.NO_TTL;
 +
+         ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+ 
          return ttl;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LivenessInfo.java
index 5d17aea,f6c9b62..c2a2291
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@@ -66,13 -72,13 +68,13 @@@ public class LivenessInf
      public static LivenessInfo expiring(long timestamp, int ttl, int nowInSec)
      {
          assert ttl != EXPIRED_LIVENESS_TTL;
-         return new ExpiringLivenessInfo(timestamp, ttl, nowInSec + ttl);
+         return new ExpiringLivenessInfo(timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl));
      }
  
 -    public static LivenessInfo create(CFMetaData metadata, long timestamp, int ttl, int nowInSec)
 +    public static LivenessInfo create(long timestamp, int ttl, int nowInSec)
      {
          return ttl == NO_TTL
 -             ? create(metadata, timestamp, nowInSec)
 +             ? create(timestamp, nowInSec)
               : expiring(timestamp, ttl, nowInSec);
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index ed107d7,bc11504..f8fa548
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -69,10 -73,10 +73,10 @@@ public class Scrubber implements Closea
  
      private static final Comparator<Partition> partitionComparator = new Comparator<Partition>()
      {
--         public int compare(Partition r1, Partition r2)
--         {
--             return r1.partitionKey().compareTo(r2.partitionKey());
--         }
++        public int compare(Partition r1, Partition r2)
++        {
++            return r1.partitionKey().compareTo(r2.partitionKey());
++        }
      };
      private final SortedSet<Partition> outOfOrder = new TreeSet<>(partitionComparator);
  
@@@ -112,8 -126,8 +124,8 @@@
          }
          this.checkData = checkData && !this.isIndex; //LocalByPartitionerType does not support validation
          this.expectedBloomFilterSize = Math.max(
--            cfs.metadata.params.minIndexInterval,
--            hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub) : 0);
++        cfs.metadata.params.minIndexInterval,
++        hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub) : 0);
  
          // loop through each row, deserializing to check for damage.
          // we'll also loop through the index at the same time, using the position from the index to recover if the
@@@ -124,8 -138,8 +136,8 @@@
                          : sstable.openDataReader(CompactionManager.instance.getRateLimiter());
  
          this.indexFile = hasIndexFile
--                ? RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)))
--                : null;
++                         ? RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)))
++                         : null;
  
          this.scrubInfo = new ScrubInfo(dataFile, sstable);
  
@@@ -203,8 -220,8 +218,8 @@@
                      if (currentIndexKey != null && !key.getKey().equals(currentIndexKey))
                      {
                          throw new IOError(new IOException(String.format("Key from data file (%s) does not match key from index file (%s)",
--                                //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
--                                "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey))));
++                                                                        //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
++                                                                        "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey))));
                      }
  
                      if (indexFile != null && dataSizeFromIndex > dataFile.length())
@@@ -225,7 -242,7 +240,7 @@@
                          && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex))
                      {
                          outputHandler.output(String.format("Retrying from row index; data is %s bytes starting at %s",
--                                                  dataSizeFromIndex, dataStartFromIndex));
++                                                           dataSizeFromIndex, dataStartFromIndex));
                          key = sstable.decorateKey(currentIndexKey);
                          try
                          {
@@@ -333,6 -352,18 +350,18 @@@
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
+     private UnfilteredRowIterator getIterator(DecoratedKey key)
+     {
 -        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(sstable, dataFile, key);
++        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(SSTableIdentityIterator.create(sstable, dataFile, key));
+         return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(rowMergingIterator,
+                                                                                     outputHandler,
+                                                                                     negativeLocalDeletionInfoMetrics) : rowMergingIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;
@@@ -342,8 -373,8 +371,8 @@@
              nextIndexKey = !indexAvailable() ? null : ByteBufferUtil.readWithShortLength(indexFile);
  
              nextRowPositionFromIndex = !indexAvailable()
--                    ? dataFile.length()
-                     : rowIndexEntrySerializer.deserializePositionAndSkip(indexFile);
 -                    : rowIndexEntrySerializer.deserialize(indexFile).position;
++                                       ? dataFile.length()
++                                       : rowIndexEntrySerializer.deserializePositionAndSkip(indexFile);
          }
          catch (Throwable th)
          {
@@@ -620,6 -652,152 +654,153 @@@
              previous = next;
              return next;
          }
+     }
  
+     /**
+      * This iterator converts negative {@link AbstractCell#localDeletionTime()} into {@link AbstractCell#MAX_DELETION_TIME}
+      *
+      * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+      */
+     private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
+     {
+         /**
+          * The decorated iterator.
+          */
+         private final UnfilteredRowIterator iterator;
+ 
+         private final OutputHandler outputHandler;
+         private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+ 
+         public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, OutputHandler outputHandler,
+                                                     NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+         {
+             this.iterator = iterator;
+             this.outputHandler = outputHandler;
+             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+         }
+ 
+         public CFMetaData metadata()
+         {
+             return iterator.metadata();
+         }
+ 
+         public boolean isReverseOrder()
+         {
+             return iterator.isReverseOrder();
+         }
+ 
+         public PartitionColumns columns()
+         {
+             return iterator.columns();
+         }
+ 
+         public DecoratedKey partitionKey()
+         {
+             return iterator.partitionKey();
+         }
+ 
+         public Row staticRow()
+         {
+             return iterator.staticRow();
+         }
+ 
+         @Override
+         public boolean isEmpty()
+         {
+             return iterator.isEmpty();
+         }
+ 
+         public void close()
+         {
+             iterator.close();
+         }
+ 
+         public DeletionTime partitionLevelDeletion()
+         {
+             return iterator.partitionLevelDeletion();
+         }
+ 
+         public EncodingStats stats()
+         {
+             return iterator.stats();
+         }
+ 
+         protected Unfiltered computeNext()
+         {
+             if (!iterator.hasNext())
+                 return endOfData();
+ 
+             Unfiltered next = iterator.next();
+             if (!next.isRow())
+                 return next;
+ 
+             if (hasNegativeLocalExpirationTime((Row) next))
+             {
+                 outputHandler.debug(String.format("Found row with negative local expiration time: %s", next.toString(metadata(), false)));
+                 negativeLocalExpirationTimeMetrics.fixedRows++;
+                 return fixNegativeLocalExpirationTime((Row) next);
+             }
+ 
+             return next;
+         }
+ 
+         private boolean hasNegativeLocalExpirationTime(Row next)
+         {
+             Row row = next;
+             if (row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0)
+             {
+                 return true;
+             }
+ 
+             for (ColumnData cd : row)
+             {
+                 if (cd.column().isSimple())
+                 {
+                     Cell cell = (Cell)cd;
+                     if (cell.isExpiring() && cell.localDeletionTime() < 0)
+                         return true;
+                 }
+                 else
+                 {
+                     ComplexColumnData complexData = (ComplexColumnData)cd;
+                     for (Cell cell : complexData)
+                     {
+                         if (cell.isExpiring() && cell.localDeletionTime() < 0)
+                             return true;
+                     }
+                 }
+             }
+ 
+             return false;
+         }
+ 
+         private Unfiltered fixNegativeLocalExpirationTime(Row row)
+         {
+             Row.Builder builder = HeapAllocator.instance.cloningBTreeRowBuilder();
+             builder.newRow(row.clustering());
+             builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0 ?
+                                               row.primaryKeyLivenessInfo().withUpdatedTimestampAndLocalDeletionTime(row.primaryKeyLivenessInfo().timestamp() + 1, AbstractCell.MAX_DELETION_TIME)
+                                               :row.primaryKeyLivenessInfo());
+             builder.addRowDeletion(row.deletion());
+             for (ColumnData cd : row)
+             {
+                 if (cd.column().isSimple())
+                 {
+                     Cell cell = (Cell)cd;
+                     builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
+                 }
+                 else
+                 {
+                     ComplexColumnData complexData = (ComplexColumnData)cd;
+                     builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
+                     for (Cell cell : complexData)
+                     {
+                         builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
+                     }
+                 }
+             }
+             return builder.build();
+         }
      }
++
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BufferCell.java
index 9b31c16,df2619c..b62d95a
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@@ -17,12 -17,18 +17,13 @@@
   */
  package org.apache.cassandra.db.rows;
  
 -import java.io.IOException;
  import java.nio.ByteBuffer;
  
 -import org.apache.cassandra.config.*;
 -import org.apache.cassandra.db.*;
 -import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
  import org.apache.cassandra.db.marshal.ByteType;
 -import org.apache.cassandra.io.util.DataInputPlus;
 -import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.ObjectSizes;
 -import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.memory.AbstractAllocator;
  
  public class BufferCell extends AbstractCell

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/Cell.java
index 19d1f30,c69e11f..1205b7d
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@@ -21,7 -21,13 +21,14 @@@ import java.io.IOException
  import java.nio.ByteBuffer;
  import java.util.Comparator;
  
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.cql3.Attributes;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.io.util.DataInputPlus;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/rows/NativeCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/NativeCell.java
index 5930332,0000000..31ce0b7
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/NativeCell.java
+++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java
@@@ -1,156 -1,0 +1,161 @@@
 +/*
 + * 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.nio.ByteOrder;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.memory.MemoryUtil;
 +import org.apache.cassandra.utils.memory.NativeAllocator;
 +
 +public class NativeCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new NativeCell());
 +
 +    private static final long HAS_CELLPATH = 0;
 +    private static final long TIMESTAMP = 1;
 +    private static final long TTL = 9;
 +    private static final long DELETION = 13;
 +    private static final long LENGTH = 17;
 +    private static final long VALUE = 21;
 +
 +    private final long peer;
 +
 +    private NativeCell()
 +    {
 +        super(null);
 +        this.peer = 0;
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      Cell cell)
 +    {
 +        this(allocator,
 +             writeOp,
 +             cell.column(),
 +             cell.timestamp(),
 +             cell.ttl(),
 +             cell.localDeletionTime(),
 +             cell.value(),
 +             cell.path());
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      ColumnDefinition column,
 +                      long timestamp,
 +                      int ttl,
 +                      int localDeletionTime,
 +                      ByteBuffer value,
 +                      CellPath path)
 +    {
 +        super(column);
 +        long size = simpleSize(value.remaining());
 +
 +        assert value.order() == ByteOrder.BIG_ENDIAN;
 +        assert column.isComplex() == (path != null);
 +        if (path != null)
 +        {
 +            assert path.size() == 1;
 +            size += 4 + path.get(0).remaining();
 +        }
 +
 +        if (size > Integer.MAX_VALUE)
 +            throw new IllegalStateException();
 +
 +        // cellpath? : timestamp : ttl : localDeletionTime : length : <data> : [cell path length] : [<cell path data>]
 +        peer = allocator.allocate((int) size, writeOp);
 +        MemoryUtil.setByte(peer + HAS_CELLPATH, (byte)(path == null ? 0 : 1));
 +        MemoryUtil.setLong(peer + TIMESTAMP, timestamp);
 +        MemoryUtil.setInt(peer + TTL, ttl);
 +        MemoryUtil.setInt(peer + DELETION, localDeletionTime);
 +        MemoryUtil.setInt(peer + LENGTH, value.remaining());
 +        MemoryUtil.setBytes(peer + VALUE, value);
 +
 +        if (path != null)
 +        {
 +            ByteBuffer pathbuffer = path.get(0);
 +            assert pathbuffer.order() == ByteOrder.BIG_ENDIAN;
 +
 +            long offset = peer + VALUE + value.remaining();
 +            MemoryUtil.setInt(offset, pathbuffer.remaining());
 +            MemoryUtil.setBytes(offset + 4, pathbuffer);
 +        }
 +    }
 +
 +    private static long simpleSize(int length)
 +    {
 +        return VALUE + length;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return MemoryUtil.getLong(peer + TIMESTAMP);
 +    }
 +
 +    public int ttl()
 +    {
 +        return MemoryUtil.getInt(peer + TTL);
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return MemoryUtil.getInt(peer + DELETION);
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        int length = MemoryUtil.getInt(peer + LENGTH);
 +        return MemoryUtil.getByteBuffer(peer + VALUE, length, ByteOrder.BIG_ENDIAN);
 +    }
 +
 +    public CellPath path()
 +    {
 +        if (MemoryUtil.getByte(peer+ HAS_CELLPATH) == 0)
 +            return null;
 +
 +        long offset = peer + VALUE + MemoryUtil.getInt(peer + LENGTH);
 +        int size = MemoryUtil.getInt(offset);
 +        return CellPath.create(MemoryUtil.getByteBuffer(offset + 4, size, ByteOrder.BIG_ENDIAN));
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
++    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return new BufferCell(column, newTimestamp, ttl(), newLocalDeletionTime, value(), path());
++    }
++
 +    public Cell withUpdatedColumn(ColumnDefinition column)
 +    {
 +        return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), value(), path());
 +    }
 +
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE;
 +    }
 +
 +}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 54b340e,4778d72..ead8fc5
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -266,6 -274,7 +269,7 @@@ public class StandaloneScrubbe
              options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
              options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
              options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
 -            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
++            options.addOption("r",  REINSERT_OVERFLOWED_TTL_OPTION,    "Reinsert found rows with overflowed TTL affected by CASSANDRA-14092");
              return options;
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 2345a85,ead2fd4..812202d
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -48,6 -49,11 +48,11 @@@ public class Scrub extends NodeToolCm
                     description = "Do not validate columns using column validator")
      private boolean noValidation = false;
  
+     @Option(title = "reinsert_overflowed_ttl",
+     name = {"r", "--reinsert-overflowed-ttl"},
 -    description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
++    description = "Reinsert found rows with overflowed TTL affected by CASSANDRA-14092")
+     private boolean reinsertOverflowedTTL = false;
+ 
      @Option(title = "jobs",
              name = {"-j", "--jobs"},
              description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
index 0000000,e7a72da..cb96af3
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
index 0000000,a3c633a..44c47fb
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -203700622
++4223695539

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
index 0000000,faf367b..ebcf4c8
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
index 0000000,45113dc..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
++Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
 -Summary.db
 -Filter.db
+ Statistics.db
 -TOC.txt
 -Digest.crc32
 -Index.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
index 0000000,c1de572..8f41a21
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
index 0000000,0403b5b..da919fe
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -82785930
++2886964045

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
index 0000000,e9d6577..549dabe
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
index 0000000,45113dc..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
++Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
 -Summary.db
 -Filter.db
+ Statistics.db
 -TOC.txt
 -Digest.crc32
 -Index.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
index 0000000,e96f772..008d3e8
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
index 0000000,459804b..0bdc0bf
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -3064924389
++3254141434

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
index 0000000,1ee01e6..62bf84e
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
index 0000000,f445537..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
 -Summary.db
 -TOC.txt
 -Filter.db
 -Index.db
+ Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
+ Statistics.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
index 0000000,a22a7a3..128ea47
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
index 0000000,db7a6c7..9d52209
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -1803989939
++3231150985

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
index 0000000,4ee9294..4eee729
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
index 0000000,f445537..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
 -Summary.db
 -TOC.txt
 -Filter.db
 -Index.db
+ Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
+ Statistics.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/CellTest.java
index ea009f6,22f1b78..c68b4ec
--- a/test/unit/org/apache/cassandra/db/CellTest.java
+++ b/test/unit/org/apache/cassandra/db/CellTest.java
@@@ -6,9 -6,9 +6,9 @@@
   * 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
@@@ -148,88 -141,19 +148,88 @@@ public class CellTes
          // But this should be valid even though the underlying value is an empty BB (catches bug #11618)
          assertValid(BufferCell.tombstone(c, 0, 4));
          // And of course, this should be valid with a proper value
 -        assertValid(BufferCell.live(fakeMetadata, c, 0, ByteBufferUtil.bytes((short)4)));
 +        assertValid(BufferCell.live(c, 0, bbs(4)));
  
          // Invalid ttl
 -        assertInvalid(BufferCell.expiring(c, 0, -4, 4, ByteBufferUtil.bytes(4)));
 -        // Invalid local deletion times
 -        assertInvalid(BufferCell.expiring(c, 0, 4, -4, ByteBufferUtil.bytes(4)));
 -        assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, ByteBufferUtil.bytes(4)));
 +        assertInvalid(BufferCell.expiring(c, 0, -4, 4, bbs(4)));
-         // Invalid local deletion times
-         assertInvalid(BufferCell.expiring(c, 0, 4, -5, bbs(4)));
-         assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bbs(4)));
++        // Cells with overflowed localExpirationTime are valid after CASSANDRA-14092
++        assertValid(BufferCell.expiring(c, 0, 4, -5, bbs(4)));
++        assertValid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bbs(4)));
  
          c = fakeColumn("c", MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
          // Valid cell path
 -        assertValid(BufferCell.live(fakeMetadata, c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes(4))));
 +        assertValid(BufferCell.live(c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes(4))));
          // Invalid cell path (int values should be 0 or 4 bytes)
 -        assertInvalid(BufferCell.live(fakeMetadata, c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes((long)4))));
 +        assertInvalid(BufferCell.live(c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes((long)4))));
 +    }
 +
 +    @Test
 +    public void testValidateNonFrozenUDT()
 +    {
 +        FieldIdentifier f1 = field("f1");  // has field position 0
 +        FieldIdentifier f2 = field("f2");  // has field position 1
 +        UserType udt = new UserType("ks",
 +                                    bb("myType"),
 +                                    asList(f1, f2),
 +                                    asList(Int32Type.instance, UTF8Type.instance),
 +                                    true);
 +        ColumnDefinition c;
 +
 +        // Valid cells
 +        c = fakeColumn("c", udt);
 +        assertValid(BufferCell.live(c, 0, bb(1), CellPath.create(bbs(0))));
 +        assertValid(BufferCell.live(c, 0, bb("foo"), CellPath.create(bbs(1))));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, bb(1), CellPath.create(bbs(0))));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, bb("foo"), CellPath.create(bbs(1))));
 +        assertValid(BufferCell.tombstone(c, 0, 4, CellPath.create(bbs(0))));
 +
 +        // Invalid value (text in an int field)
 +        assertInvalid(BufferCell.live(c, 0, bb("foo"), CellPath.create(bbs(0))));
 +
 +        // Invalid ttl
 +        assertInvalid(BufferCell.expiring(c, 0, -4, 4, bb(1), CellPath.create(bbs(0))));
-         // Invalid local deletion times
-         assertInvalid(BufferCell.expiring(c, 0, 4, -5, bb(1), CellPath.create(bbs(0))));
-         assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bb(1), CellPath.create(bbs(0))));
++        // Cells with overflowed localExpirationTime are valid after CASSANDRA-14092
++        assertValid(BufferCell.expiring(c, 0, 4, -5, bb(1), CellPath.create(bbs(0))));
++        assertValid((BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bb(1), CellPath.create(bbs(0)))));
 +
 +        // Invalid cell path (int values should be 0 or 2 bytes)
 +        assertInvalid(BufferCell.live(c, 0, bb(1), CellPath.create(ByteBufferUtil.bytes((long)4))));
 +    }
 +
 +    @Test
 +    public void testValidateFrozenUDT()
 +    {
 +        FieldIdentifier f1 = field("f1");  // has field position 0
 +        FieldIdentifier f2 = field("f2");  // has field position 1
 +        UserType udt = new UserType("ks",
 +                                    bb("myType"),
 +                                    asList(f1, f2),
 +                                    asList(Int32Type.instance, UTF8Type.instance),
 +                                    false);
 +
 +        ColumnDefinition c = fakeColumn("c", udt);
 +        ByteBuffer val = udt(bb(1), bb("foo"));
 +
 +        // Valid cells
 +        assertValid(BufferCell.live(c, 0, val));
 +        assertValid(BufferCell.live(c, 0, val));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, val));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, val));
 +        assertValid(BufferCell.tombstone(c, 0, 4));
 +        // fewer values than types is accepted
 +        assertValid(BufferCell.live(c, 0, udt(bb(1))));
 +
 +        // Invalid values
 +        // invalid types
 +        assertInvalid(BufferCell.live(c, 0, udt(bb("foo"), bb(1))));
 +        // too many types
 +        assertInvalid(BufferCell.live(c, 0, udt(bb(1), bb("foo"), bb("bar"))));
 +
 +        // Invalid ttl
 +        assertInvalid(BufferCell.expiring(c, 0, -4, 4, val));
-         // Invalid local deletion times
-         assertInvalid(BufferCell.expiring(c, 0, 4, -5, val));
-         assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, val));
++        // Cells with overflowed localExpirationTime are valid after CASSANDRA-14092
++        assertValid(BufferCell.expiring(c, 0, 4, -5, val));
++        assertValid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, val));
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------


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


[11/29] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by pa...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/trunk
Commit: 1602e606348959aead18531cb8027afb15f276e7
Parents: aa831c9 b294943
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:53:20 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:54:56 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  79 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  11 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  14 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/lb-1-big-Data.db                     | Bin 0 -> 103 bytes
 .../table1/lb-1-big-Digest.adler32              |   1 +
 .../table1/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table1/lb-1-big-Statistics.db               | Bin 0 -> 4466 bytes
 .../table1/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table1/lb-1-big-TOC.txt                     |   8 +
 .../table2/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/lb-1-big-Data.db                     | Bin 0 -> 98 bytes
 .../table2/lb-1-big-Digest.adler32              |   1 +
 .../table2/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table2/lb-1-big-Statistics.db               | Bin 0 -> 4478 bytes
 .../table2/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table2/lb-1-big-TOC.txt                     |   8 +
 .../table3/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table3/lb-1-big-Digest.adler32              |   1 +
 .../table3/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table3/lb-1-big-Statistics.db               | Bin 0 -> 4482 bytes
 .../table3/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table3/lb-1-big-TOC.txt                     |   8 +
 .../table4/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table4/lb-1-big-Digest.adler32              |   1 +
 .../table4/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table4/lb-1-big-Statistics.db               | Bin 0 -> 4494 bytes
 .../table4/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table4/lb-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 405 +++++++++++++++++++
 .../unit/org/apache/cassandra/db/ScrubTest.java |   2 +-
 58 files changed, 847 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0f6e61c,9332354..82da6ad
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -2.1.20
 +2.2.12
 + * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
 + * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
 + * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
 + * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 +Merged from 2.1:
+  * Protect against overflow of local expiration time (CASSANDRA-14092)
   * More PEP8 compliance for cqlsh (CASSANDRA-14021)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 5747941,fb6b4ee..4fe3508
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,15 -38,9 +38,16 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
 -   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 -
 -2.1.19
++    - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +      options are used. See CASSANDRA-13006 for more details.
 +    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +      for more details.
 +
 +2.2.11
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/debian/rules
----------------------------------------------------------------------
diff --cc debian/rules
index 35f5a51,70db61c..ff1d64d
--- a/debian/rules
+++ b/debian/rules
@@@ -63,7 -64,7 +63,7 @@@ binary-indep: build instal
  	dh_testroot
  	dh_installchangelogs
  	dh_installinit -u'start 50 2 3 4 5 . stop 50 0 1 6 .'
- 	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html
 -	dh_installdocs README.asc CHANGES.txt NEWS.txt
++	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html CASSANDRA-14092.txt
  	dh_installexamples tools/*.yaml
  	dh_bash-completion
  	dh_compress

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/redhat/cassandra.spec
----------------------------------------------------------------------
diff --cc redhat/cassandra.spec
index 0d4b271,a3f09b0..07c3dc5
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@@ -114,10 -113,10 +114,10 @@@ exit 
  
  %files
  %defattr(0644,root,root,0755)
--%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt
 -%attr(755,root,root) %{_bindir}/cassandra-cli
++%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt
  %attr(755,root,root) %{_bindir}/cassandra-stress
  %attr(755,root,root) %{_bindir}/cqlsh
 +%attr(755,root,root) %{_bindir}/cqlsh.py
  %attr(755,root,root) %{_bindir}/debug-cql
  %attr(755,root,root) %{_bindir}/nodetool
  %attr(755,root,root) %{_bindir}/sstablekeys

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 7b38e9f,23571ca..84f423a
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,17 -18,19 +18,23 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
 +import java.util.Collections;
+ import java.util.concurrent.TimeUnit;
  
 +import com.google.common.collect.Iterables;
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.db.ExpiringCell;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
  import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.NoSpamLogger;
  
  /**
   * Utility class for the Parser to gather attributes for modification

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 7e36e11,2989b9d..45908de
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -905,26 -879,13 +905,26 @@@ public class ColumnFamilyStore implemen
                                                  previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
 -            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
 -            flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 -            postFlushExecutor.submit(task);
 +            ListenableFutureTask<Void> flushTask = ListenableFutureTask.create(flush, null);
 +            flushExecutor.execute(flushTask);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
 +            postFlushExecutor.execute(task);
  
              @SuppressWarnings("unchecked")
-             ListenableFuture<ReplayPosition> future = 
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future =
 +                    // If either of the two tasks errors out, resulting future must also error out.
 +                    // Combine the two futures and only return post-flush result after both have completed.
 +                    // Note that flushTask will always yield null, but Futures.allAsList is
 +                    // order preserving, which is why the transform function returns the result
 +                    // from item 1 in it's input list (i.e. what was yielded by task).
 +                    Futures.transform(Futures.allAsList(flushTask, task),
 +                                      new Function<List<Object>, ReplayPosition>()
 +                                      {
 +                                          public ReplayPosition apply(List<Object> input)
 +                                          {
 +                                              return (ReplayPosition) input.get(1);
 +                                          }
 +                                      });
              return future;
          }
      }
@@@ -1610,29 -1516,48 +1610,28 @@@
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        // skip snapshot creation during scrub, SEE JIRA 5891
 -        if(!disableSnapshot)
 -            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 -    }
 -
 -    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
 -    {
 -        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
 -    }
 -
 -    public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
 -    {
 -        assert !sstables.isEmpty();
 -        data.markObsolete(sstables, compactionType);
 -    }
 -
 -    void replaceFlushed(Memtable memtable, SSTableReader sstable)
 -    {
 -        compactionStrategyWrapper.replaceFlushed(memtable, sstable);
 -    }
 -
 -    public boolean isValid()
 -    {
 -        return valid;
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
      }
  
-     @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public long getMemtableColumnsCount()
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
 -        return metric.memtableColumnsCount.value();
 -    }
 +        // skip snapshot creation during scrub, SEE JIRA 5891
 +        if(!disableSnapshot)
 +            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
  
 -    public long getMemtableDataSize()
 -    {
 -        return metric.memtableOnHeapSize.value();
 -    }
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
  
 -    public int getMemtableSwitchCount()
 -    {
 -        return (int) metric.memtableSwitchCount.count();
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 3350b20,6e3634a..d90abe9
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -348,9 -358,16 +348,15 @@@ public class CompactionManager implemen
          }
      }
  
 -    @Deprecated
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
 +    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
 -                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    throws InterruptedException, ExecutionException
+     {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -360,30 -377,11 +366,30 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
+                 scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
              }
 -        }, jobs);
 +        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
 +        }, 0, OperationType.VERIFY);
      }
  
      public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
@@@ -730,14 -717,14 +736,14 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
      {
 -        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 +        CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
 -        metrics.beginCompaction(scrubInfo);
 -        try
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
          {
 +            scrubInfo = scrubber.getScrubInfo();
 +            metrics.beginCompaction(scrubInfo);
              scrubber.scrub();
          }
          finally

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index b6b20fb,6d4537c..affee11
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -22,33 -22,30 +22,36 @@@ import java.io.*
  import java.util.*;
  
  import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.base.Predicates;
  import com.google.common.base.Throwables;
  import com.google.common.collect.AbstractIterator;
 -import com.google.common.collect.Sets;
  
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+ import org.apache.cassandra.db.composites.CellNames;
  import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.OutputHandler;
+ import org.apache.cassandra.utils.memory.HeapAllocator;
 +import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.concurrent.Refs;
  
  public class Scrubber implements Closeable
  {
      private final ColumnFamilyStore cfs;
      private final SSTableReader sstable;
 +    private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
 -    public final boolean validateColumns;
+     private final boolean reinsertOverflowedTTLRows;
  
      private final CompactionController controller;
      private final boolean isCommutative;
@@@ -84,20 -81,27 +88,28 @@@
      };
      private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
 +    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
+     }
+ 
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
 -        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
 +    @SuppressWarnings("resource")
-     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData) throws IOException
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
 -        this.sstable = sstable;
 +        this.transaction = transaction;
 +        this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.isOffline = isOffline;
 -        this.validateColumns = checkData;
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
  
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
@@@ -322,7 -339,7 +339,7 @@@
          // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
          // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
          // to the outOfOrderRows that will be later written to a new SSTable.
-         OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, checkData),
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
++        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
                                                                cfs.metadata.comparator.onDiskAtomComparator());
          if (prevKey != null && prevKey.compareTo(key) > 0)
          {
@@@ -342,6 -359,18 +359,18 @@@
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
++    private OnDiskAtomIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
++        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
+         return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                     outputHandler,
+                                                                                     negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 6896062,d718765..e416c7b
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@@ -31,8 -32,8 +32,9 @@@ import org.apache.cassandra.cql3.Attrib
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.context.CounterContext;
  import org.apache.cassandra.dht.IPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 +import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.CounterId;
  import org.apache.cassandra.utils.Pair;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 1ecedac,a7a8ca7..2c9ac4d
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2611,10 -2415,16 +2611,16 @@@ public class StorageService extends Not
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                      int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 761eed6,90c0fb5..f336bcc
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -265,15 -274,10 +265,18 @@@ public interface StorageServiceMBean ex
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++
 +    /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Rewrite all sstables to the latest version.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d735676,10e7185..8bdf9dc
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -312,9 -313,9 +313,9 @@@ public class ThriftValidatio
          if (cosc.column != null)
          {
              if (isCommutative)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
              validateColumnData(metadata, key, null, cosc.column);
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 24c5874,fcd4110..17bef02
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -236,16 -243,11 +236,16 @@@ public class NodeProbe implements AutoC
          return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
@@@ -267,22 -268,13 +267,22 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+         if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index a486a13,59d13d5..f5e84c5
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -119,10 -115,10 +126,10 @@@ public class StandaloneScrubbe
              {
                  for (SSTableReader sstable : sstables)
                  {
 -                    try
 +                    try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
 -                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
 -                        try
 +                        txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
                          {
                              scrubber.scrub();
                          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index dafe8d1,0000000..50224a0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,76 -1,0 +1,82 @@@
 +/*
 + * 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.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
++import org.apache.cassandra.tools.StandaloneScrubber;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
++    @Option(title = "reinsert_overflowed_ttl",
++    name = {"r", "--reinsert-overflowed-ttl"},
++    description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
++    private boolean reinsertOverflowedTTL = false;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
index 0000000,0000000..d7cc13b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
index 0000000,0000000..51213c2
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
index 0000000,0000000..d5b12df
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2292388625

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
index 0000000,0000000..3ab96ee
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
index 0000000,0000000..e8cc7e0
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
index 0000000,0000000..38373b4
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
index 0000000,0000000..762a229
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
index 0000000,0000000..ae89849
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3829731931

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
index 0000000,0000000..38a6e4c
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
index 0000000,0000000..64dab43
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
index 0000000,0000000..04a7384
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
index 0000000,0000000..33145df
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
index 0000000,0000000..2a542cd
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3574474340

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
index 0000000,0000000..5fb34e8
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
index 0000000,0000000..51203ae
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
index 0000000,0000000..c814fef
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
index 0000000,0000000..f40e71f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
index 0000000,0000000..e6675e4
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2405377913

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
index 0000000,0000000..8291383
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
index 0000000,0000000..2217c2d
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 0000000,ab4ef21..b1eaac1
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -1,0 -1,410 +1,405 @@@
+ package org.apache.cassandra.cql3.validation.operations;
+ 
+ import java.io.File;
+ import java.io.FileInputStream;
+ import java.io.FileOutputStream;
+ import java.io.IOException;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
+ 
+ import org.apache.cassandra.cql3.Attributes;
+ import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
+ import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.ExpiringCell;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
+ import org.junit.Test;
+ 
+ public class TTLTest extends CQLTester
+ {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
+     public static int MAX_TTL = ExpiringCell.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
+ 
+     @Test
+     public void testTTLPerRequestLimit() throws Throwable
+     {
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         // insert with low TTL should not be denied
+         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+         execute("TRUNCATE %s");
+ 
+         // insert with low TTL should not be denied
+         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+     }
+ 
+ 
+     @Test
+     public void testTTLDefaultLimit() throws Throwable
+     {
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live cannot be smaller than 0"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                         + (MAX_TTL + 1));
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
+         }
+ 
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
+     public void testRejectExpirationDateOverflowPolicy() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         checkTTLIsCapped("i");
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyPerRequest() throws Throwable
+     {
+         // Test cap policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+ 
+         // simple column, clustering, flush
+         baseCapExpirationDateOverflowTest(true, true, true);
+         // simple column, clustering, noflush
+         baseCapExpirationDateOverflowTest(true, true, false);
+         // simple column, noclustering, flush
+         baseCapExpirationDateOverflowTest(true, false, true);
+         // simple column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(true, false, false);
+         // complex column, clustering, flush
+         baseCapExpirationDateOverflowTest(false, true, true);
+         // complex column, clustering, noflush
+         baseCapExpirationDateOverflowTest(false, true, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, true);
+         // complex column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(false, false, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, false);
+ 
+         // Return to previous policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        createTable(true, true);
 -        createTable(true, false);
 -        createTable(false, true);
 -        createTable(false, false);
 -
+         baseTestRecoverOverflowedExpiration(false, false);
+         baseTestRecoverOverflowedExpiration(true, false);
+         baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
+     public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
+         if (simple)
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
+     }
+ 
+     /**
+      * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+      * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
+         UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+      * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+         return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
++        createTable(simple, clustering);
++
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getLiveSSTableCount());
++        assertEquals(0, cfs.getSSTables().size());
+ 
 -        copySSTablesToTableDir(simple, clustering);
++        copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
+             cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -            {
 -                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
 -                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
 -
 -            }
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
 -            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
++            assertEmpty(execute("SELECT * from %s"));
+         }
 -        cfs.truncateBlocking(); //cleanup for next tests
+     }
+ 
 -    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
++    private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
 -        File sourceDir = getTableDir(simple, clustering);
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering) throws Throwable
++    private static File getTableDir(String table, boolean simple, boolean clustering)
++    {
++        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
++    }
++
++    private void createTable(boolean simple, boolean clustering)
+     {
+         if (simple)
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+     }
+ 
+     private static File getTableDir(boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 4cca7ff,4efd082..9b1ede4
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -654,129 -565,4 +654,129 @@@ public class ScrubTes
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }


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


[27/29] cassandra git commit: Merge branch 'cassandra-3.0' into cassandra-3.11

Posted by pa...@apache.org.
Merge branch 'cassandra-3.0' into cassandra-3.11


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

Branch: refs/heads/cassandra-3.11
Commit: 0a6b6f506b012d4d491757d6216c5fa1c53bedc9
Parents: 6eb65e5 c231ed5
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:58:13 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:59:28 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 +++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 ++
 conf/jvm.options                                |  11 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  15 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../db/ExpirationDateOverflowHandling.java      | 121 +++++++
 .../org/apache/cassandra/db/LegacyLayout.java   |   6 +-
 .../org/apache/cassandra/db/LivenessInfo.java   |  13 +-
 .../db/compaction/CompactionManager.java        |  16 +-
 .../cassandra/db/compaction/Scrubber.java       | 225 +++++++++++--
 .../apache/cassandra/db/rows/BufferCell.java    |   8 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |  10 +
 .../apache/cassandra/db/rows/NativeCell.java    |   5 +
 .../cassandra/service/StorageService.java       |   7 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |   6 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   7 +-
 .../table1/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/mc-1-big-Data.db                     | Bin 0 -> 58 bytes
 .../table1/mc-1-big-Digest.crc32                |   1 +
 .../table1/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Statistics.db               | Bin 0 -> 4676 bytes
 .../table1/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table1/mc-1-big-TOC.txt                     |   8 +
 .../table2/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/mc-1-big-Data.db                     | Bin 0 -> 60 bytes
 .../table2/mc-1-big-Digest.crc32                |   1 +
 .../table2/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Statistics.db               | Bin 0 -> 4686 bytes
 .../table2/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table2/mc-1-big-TOC.txt                     |   8 +
 .../table3/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/mc-1-big-Data.db                     | Bin 0 -> 111 bytes
 .../table3/mc-1-big-Digest.crc32                |   1 +
 .../table3/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Statistics.db               | Bin 0 -> 4732 bytes
 .../table3/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table3/mc-1-big-TOC.txt                     |   8 +
 .../table4/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/mc-1-big-Data.db                     | Bin 0 -> 108 bytes
 .../table4/mc-1-big-Digest.crc32                |   1 +
 .../table4/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Statistics.db               | Bin 0 -> 4742 bytes
 .../table4/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table4/mc-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 327 ++++++++++++++++++-
 test/unit/org/apache/cassandra/db/CellTest.java |  22 +-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 58 files changed, 893 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index c38b69b,a492c42..5b49f48
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -48,26 -28,12 +48,27 @@@ Merged from 2.2
   * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
   * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
   * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
 + * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
  Merged from 2.1:
 - * More PEP8 compliance for cqlsh (CASSANDRA-14021)
++ * Protect against overflow of local expiration time (CASSANDRA-14092)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 -
 -
 -3.0.15
 + * More PEP8 compiance for cqlsh (CASSANDRA-14021)
 +
 +
 +3.11.1
 + * Fix the computation of cdc_total_space_in_mb for exabyte filesystems (CASSANDRA-13808)
 + * AbstractTokenTreeBuilder#serializedSize returns wrong value when there is a single leaf and overflow collisions (CASSANDRA-13869)
 + * Add a compaction option to TWCS to ignore sstables overlapping checks (CASSANDRA-13418)
 + * BTree.Builder memory leak (CASSANDRA-13754)
 + * Revert CASSANDRA-10368 of supporting non-pk column filtering due to correctness (CASSANDRA-13798)
 + * Add a skip read validation flag to cassandra-stress (CASSANDRA-13772)
 + * Fix cassandra-stress hang issues when an error during cluster connection happens (CASSANDRA-12938)
 + * Better bootstrap failure message when blocked by (potential) range movement (CASSANDRA-13744)
 + * "ignore" option is ignored in sstableloader (CASSANDRA-13721)
 + * Deadlock in AbstractCommitLogSegmentManager (CASSANDRA-13652)
 + * Duplicate the buffer before passing it to analyser in SASI operation (CASSANDRA-13512)
 + * Properly evict pstmts from prepared statements cache (CASSANDRA-13641)
 +Merged from 3.0:
   * Improve TRUNCATE performance (CASSANDRA-13909)
   * Implement short read protection on partition boundaries (CASSANDRA-13595)
   * Fix ISE thrown by UPI.Serializer.hasNext() for some SELECT queries (CASSANDRA-13911)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index f4b15e7,f574c33..fb1dafe
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,22 -38,14 +38,23 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
+    - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 -   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 -     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 -     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 -     options are used. See CASSANDRA-13006 for more details.
 -   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 -     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 -     for more details.
 +    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +      options are used. See CASSANDRA-13006 for more details
 +    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +      for more details.
 +    - Upgrades from 3.0 might have produced unnecessary schema migrations while
 +      there was at least one 3.0 node in the cluster. It is therefore highly
 +      recommended to upgrade from 3.0 to at least 3.11.2. The root cause of
 +      this schema mismatch was a difference in the way how schema digests were computed
 +      in 3.0 and 3.11.2. To mitigate this issue, 3.11.2 and newer announce
 +      3.0 compatible digests as long as there is at least one 3.0 node in the
 +      cluster. Once all nodes have been upgraded, the "real" schema version will be
 +      announced. Note: this fix is only necessary in 3.11.2 and therefore only applies
 +      to 3.11. (CASSANDRA-14109)
  
  Materialized Views
  -------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/conf/jvm.options
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/redhat/cassandra.spec
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index d915560,832d0a7..d4e230f
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -20,8 -20,9 +20,10 @@@ package org.apache.cassandra.cql3
  import java.nio.ByteBuffer;
  import java.util.List;
  
+ import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.functions.Function;
+ import org.apache.cassandra.db.ExpirationDateOverflowHandling;
 +import org.apache.cassandra.db.LivenessInfo;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
@@@ -105,11 -109,11 +110,11 @@@ public class Attribute
  
          ByteBuffer tval = timeToLive.bindAndGet(options);
          if (tval == null)
 -            throw new InvalidRequestException("Invalid null value of TTL");
 -
 -        if (tval == ByteBufferUtil.UNSET_BYTE_BUFFER) // treat as unlimited
              return 0;
  
 +        if (tval == ByteBufferUtil.UNSET_BYTE_BUFFER)
-             return defaultTimeToLive;
++            return metadata.params.defaultTimeToLive;
 +
          try
          {
              Int32Type.instance.validate(tval);
@@@ -126,9 -130,8 +131,11 @@@
          if (ttl > MAX_TTL)
              throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, MAX_TTL));
  
-         if (defaultTimeToLive != LivenessInfo.NO_TTL && ttl == LivenessInfo.NO_TTL)
++        if (metadata.params.defaultTimeToLive != LivenessInfo.NO_TTL && ttl == LivenessInfo.NO_TTL)
 +            return LivenessInfo.NO_TTL;
 +
+         ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+ 
          return ttl;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LivenessInfo.java
index 5d17aea,f6c9b62..c2a2291
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@@ -66,13 -72,13 +68,13 @@@ public class LivenessInf
      public static LivenessInfo expiring(long timestamp, int ttl, int nowInSec)
      {
          assert ttl != EXPIRED_LIVENESS_TTL;
-         return new ExpiringLivenessInfo(timestamp, ttl, nowInSec + ttl);
+         return new ExpiringLivenessInfo(timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl));
      }
  
 -    public static LivenessInfo create(CFMetaData metadata, long timestamp, int ttl, int nowInSec)
 +    public static LivenessInfo create(long timestamp, int ttl, int nowInSec)
      {
          return ttl == NO_TTL
 -             ? create(metadata, timestamp, nowInSec)
 +             ? create(timestamp, nowInSec)
               : expiring(timestamp, ttl, nowInSec);
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index ed107d7,bc11504..f8fa548
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -69,10 -73,10 +73,10 @@@ public class Scrubber implements Closea
  
      private static final Comparator<Partition> partitionComparator = new Comparator<Partition>()
      {
--         public int compare(Partition r1, Partition r2)
--         {
--             return r1.partitionKey().compareTo(r2.partitionKey());
--         }
++        public int compare(Partition r1, Partition r2)
++        {
++            return r1.partitionKey().compareTo(r2.partitionKey());
++        }
      };
      private final SortedSet<Partition> outOfOrder = new TreeSet<>(partitionComparator);
  
@@@ -112,8 -126,8 +124,8 @@@
          }
          this.checkData = checkData && !this.isIndex; //LocalByPartitionerType does not support validation
          this.expectedBloomFilterSize = Math.max(
--            cfs.metadata.params.minIndexInterval,
--            hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub) : 0);
++        cfs.metadata.params.minIndexInterval,
++        hasIndexFile ? SSTableReader.getApproximateKeyCount(toScrub) : 0);
  
          // loop through each row, deserializing to check for damage.
          // we'll also loop through the index at the same time, using the position from the index to recover if the
@@@ -124,8 -138,8 +136,8 @@@
                          : sstable.openDataReader(CompactionManager.instance.getRateLimiter());
  
          this.indexFile = hasIndexFile
--                ? RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)))
--                : null;
++                         ? RandomAccessReader.open(new File(sstable.descriptor.filenameFor(Component.PRIMARY_INDEX)))
++                         : null;
  
          this.scrubInfo = new ScrubInfo(dataFile, sstable);
  
@@@ -203,8 -220,8 +218,8 @@@
                      if (currentIndexKey != null && !key.getKey().equals(currentIndexKey))
                      {
                          throw new IOError(new IOException(String.format("Key from data file (%s) does not match key from index file (%s)",
--                                //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
--                                "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey))));
++                                                                        //ByteBufferUtil.bytesToHex(key.getKey()), ByteBufferUtil.bytesToHex(currentIndexKey))));
++                                                                        "_too big_", ByteBufferUtil.bytesToHex(currentIndexKey))));
                      }
  
                      if (indexFile != null && dataSizeFromIndex > dataFile.length())
@@@ -225,7 -242,7 +240,7 @@@
                          && (key == null || !key.getKey().equals(currentIndexKey) || dataStart != dataStartFromIndex))
                      {
                          outputHandler.output(String.format("Retrying from row index; data is %s bytes starting at %s",
--                                                  dataSizeFromIndex, dataStartFromIndex));
++                                                           dataSizeFromIndex, dataStartFromIndex));
                          key = sstable.decorateKey(currentIndexKey);
                          try
                          {
@@@ -333,6 -352,18 +350,18 @@@
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
+     private UnfilteredRowIterator getIterator(DecoratedKey key)
+     {
 -        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(sstable, dataFile, key);
++        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(SSTableIdentityIterator.create(sstable, dataFile, key));
+         return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(rowMergingIterator,
+                                                                                     outputHandler,
+                                                                                     negativeLocalDeletionInfoMetrics) : rowMergingIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;
@@@ -342,8 -373,8 +371,8 @@@
              nextIndexKey = !indexAvailable() ? null : ByteBufferUtil.readWithShortLength(indexFile);
  
              nextRowPositionFromIndex = !indexAvailable()
--                    ? dataFile.length()
-                     : rowIndexEntrySerializer.deserializePositionAndSkip(indexFile);
 -                    : rowIndexEntrySerializer.deserialize(indexFile).position;
++                                       ? dataFile.length()
++                                       : rowIndexEntrySerializer.deserializePositionAndSkip(indexFile);
          }
          catch (Throwable th)
          {
@@@ -620,6 -652,152 +654,153 @@@
              previous = next;
              return next;
          }
+     }
  
+     /**
+      * This iterator converts negative {@link AbstractCell#localDeletionTime()} into {@link AbstractCell#MAX_DELETION_TIME}
+      *
+      * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+      */
+     private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
+     {
+         /**
+          * The decorated iterator.
+          */
+         private final UnfilteredRowIterator iterator;
+ 
+         private final OutputHandler outputHandler;
+         private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+ 
+         public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, OutputHandler outputHandler,
+                                                     NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+         {
+             this.iterator = iterator;
+             this.outputHandler = outputHandler;
+             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+         }
+ 
+         public CFMetaData metadata()
+         {
+             return iterator.metadata();
+         }
+ 
+         public boolean isReverseOrder()
+         {
+             return iterator.isReverseOrder();
+         }
+ 
+         public PartitionColumns columns()
+         {
+             return iterator.columns();
+         }
+ 
+         public DecoratedKey partitionKey()
+         {
+             return iterator.partitionKey();
+         }
+ 
+         public Row staticRow()
+         {
+             return iterator.staticRow();
+         }
+ 
+         @Override
+         public boolean isEmpty()
+         {
+             return iterator.isEmpty();
+         }
+ 
+         public void close()
+         {
+             iterator.close();
+         }
+ 
+         public DeletionTime partitionLevelDeletion()
+         {
+             return iterator.partitionLevelDeletion();
+         }
+ 
+         public EncodingStats stats()
+         {
+             return iterator.stats();
+         }
+ 
+         protected Unfiltered computeNext()
+         {
+             if (!iterator.hasNext())
+                 return endOfData();
+ 
+             Unfiltered next = iterator.next();
+             if (!next.isRow())
+                 return next;
+ 
+             if (hasNegativeLocalExpirationTime((Row) next))
+             {
+                 outputHandler.debug(String.format("Found row with negative local expiration time: %s", next.toString(metadata(), false)));
+                 negativeLocalExpirationTimeMetrics.fixedRows++;
+                 return fixNegativeLocalExpirationTime((Row) next);
+             }
+ 
+             return next;
+         }
+ 
+         private boolean hasNegativeLocalExpirationTime(Row next)
+         {
+             Row row = next;
+             if (row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0)
+             {
+                 return true;
+             }
+ 
+             for (ColumnData cd : row)
+             {
+                 if (cd.column().isSimple())
+                 {
+                     Cell cell = (Cell)cd;
+                     if (cell.isExpiring() && cell.localDeletionTime() < 0)
+                         return true;
+                 }
+                 else
+                 {
+                     ComplexColumnData complexData = (ComplexColumnData)cd;
+                     for (Cell cell : complexData)
+                     {
+                         if (cell.isExpiring() && cell.localDeletionTime() < 0)
+                             return true;
+                     }
+                 }
+             }
+ 
+             return false;
+         }
+ 
+         private Unfiltered fixNegativeLocalExpirationTime(Row row)
+         {
+             Row.Builder builder = HeapAllocator.instance.cloningBTreeRowBuilder();
+             builder.newRow(row.clustering());
+             builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0 ?
+                                               row.primaryKeyLivenessInfo().withUpdatedTimestampAndLocalDeletionTime(row.primaryKeyLivenessInfo().timestamp() + 1, AbstractCell.MAX_DELETION_TIME)
+                                               :row.primaryKeyLivenessInfo());
+             builder.addRowDeletion(row.deletion());
+             for (ColumnData cd : row)
+             {
+                 if (cd.column().isSimple())
+                 {
+                     Cell cell = (Cell)cd;
+                     builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
+                 }
+                 else
+                 {
+                     ComplexColumnData complexData = (ComplexColumnData)cd;
+                     builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
+                     for (Cell cell : complexData)
+                     {
+                         builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
+                     }
+                 }
+             }
+             return builder.build();
+         }
      }
++
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BufferCell.java
index 9b31c16,df2619c..b62d95a
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@@ -17,12 -17,18 +17,13 @@@
   */
  package org.apache.cassandra.db.rows;
  
 -import java.io.IOException;
  import java.nio.ByteBuffer;
  
 -import org.apache.cassandra.config.*;
 -import org.apache.cassandra.db.*;
 -import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
  import org.apache.cassandra.db.marshal.ByteType;
 -import org.apache.cassandra.io.util.DataInputPlus;
 -import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.ObjectSizes;
 -import org.apache.cassandra.utils.FBUtilities;
  import org.apache.cassandra.utils.memory.AbstractAllocator;
  
  public class BufferCell extends AbstractCell

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/Cell.java
index 19d1f30,c69e11f..1205b7d
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@@ -21,7 -21,13 +21,14 @@@ import java.io.IOException
  import java.nio.ByteBuffer;
  import java.util.Comparator;
  
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
+ 
+ import org.apache.cassandra.config.CFMetaData;
+ import org.apache.cassandra.config.ColumnDefinition;
+ import org.apache.cassandra.cql3.Attributes;
 +import org.apache.cassandra.config.*;
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.io.util.DataOutputPlus;
  import org.apache.cassandra.io.util.DataInputPlus;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/db/rows/NativeCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/NativeCell.java
index 5930332,0000000..31ce0b7
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/NativeCell.java
+++ b/src/java/org/apache/cassandra/db/rows/NativeCell.java
@@@ -1,156 -1,0 +1,161 @@@
 +/*
 + * 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.nio.ByteOrder;
 +
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.concurrent.OpOrder;
 +import org.apache.cassandra.utils.memory.MemoryUtil;
 +import org.apache.cassandra.utils.memory.NativeAllocator;
 +
 +public class NativeCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new NativeCell());
 +
 +    private static final long HAS_CELLPATH = 0;
 +    private static final long TIMESTAMP = 1;
 +    private static final long TTL = 9;
 +    private static final long DELETION = 13;
 +    private static final long LENGTH = 17;
 +    private static final long VALUE = 21;
 +
 +    private final long peer;
 +
 +    private NativeCell()
 +    {
 +        super(null);
 +        this.peer = 0;
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      Cell cell)
 +    {
 +        this(allocator,
 +             writeOp,
 +             cell.column(),
 +             cell.timestamp(),
 +             cell.ttl(),
 +             cell.localDeletionTime(),
 +             cell.value(),
 +             cell.path());
 +    }
 +
 +    public NativeCell(NativeAllocator allocator,
 +                      OpOrder.Group writeOp,
 +                      ColumnDefinition column,
 +                      long timestamp,
 +                      int ttl,
 +                      int localDeletionTime,
 +                      ByteBuffer value,
 +                      CellPath path)
 +    {
 +        super(column);
 +        long size = simpleSize(value.remaining());
 +
 +        assert value.order() == ByteOrder.BIG_ENDIAN;
 +        assert column.isComplex() == (path != null);
 +        if (path != null)
 +        {
 +            assert path.size() == 1;
 +            size += 4 + path.get(0).remaining();
 +        }
 +
 +        if (size > Integer.MAX_VALUE)
 +            throw new IllegalStateException();
 +
 +        // cellpath? : timestamp : ttl : localDeletionTime : length : <data> : [cell path length] : [<cell path data>]
 +        peer = allocator.allocate((int) size, writeOp);
 +        MemoryUtil.setByte(peer + HAS_CELLPATH, (byte)(path == null ? 0 : 1));
 +        MemoryUtil.setLong(peer + TIMESTAMP, timestamp);
 +        MemoryUtil.setInt(peer + TTL, ttl);
 +        MemoryUtil.setInt(peer + DELETION, localDeletionTime);
 +        MemoryUtil.setInt(peer + LENGTH, value.remaining());
 +        MemoryUtil.setBytes(peer + VALUE, value);
 +
 +        if (path != null)
 +        {
 +            ByteBuffer pathbuffer = path.get(0);
 +            assert pathbuffer.order() == ByteOrder.BIG_ENDIAN;
 +
 +            long offset = peer + VALUE + value.remaining();
 +            MemoryUtil.setInt(offset, pathbuffer.remaining());
 +            MemoryUtil.setBytes(offset + 4, pathbuffer);
 +        }
 +    }
 +
 +    private static long simpleSize(int length)
 +    {
 +        return VALUE + length;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return MemoryUtil.getLong(peer + TIMESTAMP);
 +    }
 +
 +    public int ttl()
 +    {
 +        return MemoryUtil.getInt(peer + TTL);
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return MemoryUtil.getInt(peer + DELETION);
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        int length = MemoryUtil.getInt(peer + LENGTH);
 +        return MemoryUtil.getByteBuffer(peer + VALUE, length, ByteOrder.BIG_ENDIAN);
 +    }
 +
 +    public CellPath path()
 +    {
 +        if (MemoryUtil.getByte(peer+ HAS_CELLPATH) == 0)
 +            return null;
 +
 +        long offset = peer + VALUE + MemoryUtil.getInt(peer + LENGTH);
 +        int size = MemoryUtil.getInt(offset);
 +        return CellPath.create(MemoryUtil.getByteBuffer(offset + 4, size, ByteOrder.BIG_ENDIAN));
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        throw new UnsupportedOperationException();
 +    }
 +
++    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return new BufferCell(column, newTimestamp, ttl(), newLocalDeletionTime, value(), path());
++    }
++
 +    public Cell withUpdatedColumn(ColumnDefinition column)
 +    {
 +        return new BufferCell(column, timestamp(), ttl(), localDeletionTime(), value(), path());
 +    }
 +
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE;
 +    }
 +
 +}

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

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 54b340e,4778d72..ead8fc5
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -266,6 -274,7 +269,7 @@@ public class StandaloneScrubbe
              options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
              options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
              options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
 -            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
++            options.addOption("r",  REINSERT_OVERFLOWED_TTL_OPTION,    "Reinsert found rows with overflowed TTL affected by CASSANDRA-14092");
              return options;
          }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 2345a85,ead2fd4..812202d
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -48,6 -49,11 +48,11 @@@ public class Scrub extends NodeToolCm
                     description = "Do not validate columns using column validator")
      private boolean noValidation = false;
  
+     @Option(title = "reinsert_overflowed_ttl",
+     name = {"r", "--reinsert-overflowed-ttl"},
 -    description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
++    description = "Reinsert found rows with overflowed TTL affected by CASSANDRA-14092")
+     private boolean reinsertOverflowedTTL = false;
+ 
      @Option(title = "jobs",
              name = {"-j", "--jobs"},
              description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
index 0000000,e7a72da..cb96af3
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
index 0000000,a3c633a..44c47fb
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -203700622
++4223695539

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
index 0000000,faf367b..ebcf4c8
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
index 0000000,45113dc..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
++Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
 -Summary.db
 -Filter.db
+ Statistics.db
 -TOC.txt
 -Digest.crc32
 -Index.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
index 0000000,c1de572..8f41a21
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
index 0000000,0403b5b..da919fe
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -82785930
++2886964045

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
index 0000000,e9d6577..549dabe
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
index 0000000,45113dc..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
++Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
 -Summary.db
 -Filter.db
+ Statistics.db
 -TOC.txt
 -Digest.crc32
 -Index.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
index 0000000,e96f772..008d3e8
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
index 0000000,459804b..0bdc0bf
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -3064924389
++3254141434

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
index 0000000,1ee01e6..62bf84e
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
index 0000000,f445537..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
 -Summary.db
 -TOC.txt
 -Filter.db
 -Index.db
+ Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
+ Statistics.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
index 0000000,a22a7a3..128ea47
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
index 0000000,db7a6c7..9d52209
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
@@@ -1,0 -1,1 +1,1 @@@
 -1803989939
++3231150985

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
index 0000000,4ee9294..4eee729
mode 000000,100644..100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
index 0000000,f445537..831e376
mode 000000,100644..100644
--- a/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
@@@ -1,0 -1,8 +1,8 @@@
 -Summary.db
 -TOC.txt
 -Filter.db
 -Index.db
+ Digest.crc32
+ CompressionInfo.db
++Index.db
++TOC.txt
+ Data.db
+ Statistics.db
++Filter.db
++Summary.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/CellTest.java
index ea009f6,22f1b78..c68b4ec
--- a/test/unit/org/apache/cassandra/db/CellTest.java
+++ b/test/unit/org/apache/cassandra/db/CellTest.java
@@@ -6,9 -6,9 +6,9 @@@
   * 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
@@@ -148,88 -141,19 +148,88 @@@ public class CellTes
          // But this should be valid even though the underlying value is an empty BB (catches bug #11618)
          assertValid(BufferCell.tombstone(c, 0, 4));
          // And of course, this should be valid with a proper value
 -        assertValid(BufferCell.live(fakeMetadata, c, 0, ByteBufferUtil.bytes((short)4)));
 +        assertValid(BufferCell.live(c, 0, bbs(4)));
  
          // Invalid ttl
 -        assertInvalid(BufferCell.expiring(c, 0, -4, 4, ByteBufferUtil.bytes(4)));
 -        // Invalid local deletion times
 -        assertInvalid(BufferCell.expiring(c, 0, 4, -4, ByteBufferUtil.bytes(4)));
 -        assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, ByteBufferUtil.bytes(4)));
 +        assertInvalid(BufferCell.expiring(c, 0, -4, 4, bbs(4)));
-         // Invalid local deletion times
-         assertInvalid(BufferCell.expiring(c, 0, 4, -5, bbs(4)));
-         assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bbs(4)));
++        // Cells with overflowed localExpirationTime are valid after CASSANDRA-14092
++        assertValid(BufferCell.expiring(c, 0, 4, -5, bbs(4)));
++        assertValid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bbs(4)));
  
          c = fakeColumn("c", MapType.getInstance(Int32Type.instance, Int32Type.instance, true));
          // Valid cell path
 -        assertValid(BufferCell.live(fakeMetadata, c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes(4))));
 +        assertValid(BufferCell.live(c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes(4))));
          // Invalid cell path (int values should be 0 or 4 bytes)
 -        assertInvalid(BufferCell.live(fakeMetadata, c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes((long)4))));
 +        assertInvalid(BufferCell.live(c, 0, ByteBufferUtil.bytes(4), CellPath.create(ByteBufferUtil.bytes((long)4))));
 +    }
 +
 +    @Test
 +    public void testValidateNonFrozenUDT()
 +    {
 +        FieldIdentifier f1 = field("f1");  // has field position 0
 +        FieldIdentifier f2 = field("f2");  // has field position 1
 +        UserType udt = new UserType("ks",
 +                                    bb("myType"),
 +                                    asList(f1, f2),
 +                                    asList(Int32Type.instance, UTF8Type.instance),
 +                                    true);
 +        ColumnDefinition c;
 +
 +        // Valid cells
 +        c = fakeColumn("c", udt);
 +        assertValid(BufferCell.live(c, 0, bb(1), CellPath.create(bbs(0))));
 +        assertValid(BufferCell.live(c, 0, bb("foo"), CellPath.create(bbs(1))));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, bb(1), CellPath.create(bbs(0))));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, bb("foo"), CellPath.create(bbs(1))));
 +        assertValid(BufferCell.tombstone(c, 0, 4, CellPath.create(bbs(0))));
 +
 +        // Invalid value (text in an int field)
 +        assertInvalid(BufferCell.live(c, 0, bb("foo"), CellPath.create(bbs(0))));
 +
 +        // Invalid ttl
 +        assertInvalid(BufferCell.expiring(c, 0, -4, 4, bb(1), CellPath.create(bbs(0))));
-         // Invalid local deletion times
-         assertInvalid(BufferCell.expiring(c, 0, 4, -5, bb(1), CellPath.create(bbs(0))));
-         assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bb(1), CellPath.create(bbs(0))));
++        // Cells with overflowed localExpirationTime are valid after CASSANDRA-14092
++        assertValid(BufferCell.expiring(c, 0, 4, -5, bb(1), CellPath.create(bbs(0))));
++        assertValid((BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, bb(1), CellPath.create(bbs(0)))));
 +
 +        // Invalid cell path (int values should be 0 or 2 bytes)
 +        assertInvalid(BufferCell.live(c, 0, bb(1), CellPath.create(ByteBufferUtil.bytes((long)4))));
 +    }
 +
 +    @Test
 +    public void testValidateFrozenUDT()
 +    {
 +        FieldIdentifier f1 = field("f1");  // has field position 0
 +        FieldIdentifier f2 = field("f2");  // has field position 1
 +        UserType udt = new UserType("ks",
 +                                    bb("myType"),
 +                                    asList(f1, f2),
 +                                    asList(Int32Type.instance, UTF8Type.instance),
 +                                    false);
 +
 +        ColumnDefinition c = fakeColumn("c", udt);
 +        ByteBuffer val = udt(bb(1), bb("foo"));
 +
 +        // Valid cells
 +        assertValid(BufferCell.live(c, 0, val));
 +        assertValid(BufferCell.live(c, 0, val));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, val));
 +        assertValid(BufferCell.expiring(c, 0, 4, 4, val));
 +        assertValid(BufferCell.tombstone(c, 0, 4));
 +        // fewer values than types is accepted
 +        assertValid(BufferCell.live(c, 0, udt(bb(1))));
 +
 +        // Invalid values
 +        // invalid types
 +        assertInvalid(BufferCell.live(c, 0, udt(bb("foo"), bb(1))));
 +        // too many types
 +        assertInvalid(BufferCell.live(c, 0, udt(bb(1), bb("foo"), bb("bar"))));
 +
 +        // Invalid ttl
 +        assertInvalid(BufferCell.expiring(c, 0, -4, 4, val));
-         // Invalid local deletion times
-         assertInvalid(BufferCell.expiring(c, 0, 4, -5, val));
-         assertInvalid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, val));
++        // Cells with overflowed localExpirationTime are valid after CASSANDRA-14092
++        assertValid(BufferCell.expiring(c, 0, 4, -5, val));
++        assertValid(BufferCell.expiring(c, 0, 4, Cell.NO_DELETION_TIME, val));
      }
  
      @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/0a6b6f50/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------


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


[26/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.0
Commit: c231ed5be0f47b030181185f544132523a2cf908
Parents: 21978bf 1602e60
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:55:27 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:57:53 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 +++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 ++
 conf/jvm.options                                |  11 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  14 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../db/ExpirationDateOverflowHandling.java      | 121 +++++++
 .../org/apache/cassandra/db/LegacyLayout.java   |   7 +-
 .../org/apache/cassandra/db/LivenessInfo.java   |  12 +-
 .../db/compaction/CompactionManager.java        |  16 +-
 .../cassandra/db/compaction/Scrubber.java       | 198 ++++++++++-
 .../apache/cassandra/db/rows/BufferCell.java    |   7 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |   9 +
 .../cassandra/service/StorageService.java       |   7 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/SSTableMetadataViewer.java  |   1 +
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/mc-1-big-Data.db                     | Bin 0 -> 58 bytes
 .../table1/mc-1-big-Digest.crc32                |   1 +
 .../table1/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Statistics.db               | Bin 0 -> 4676 bytes
 .../table1/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table1/mc-1-big-TOC.txt                     |   8 +
 .../table2/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/mc-1-big-Data.db                     | Bin 0 -> 60 bytes
 .../table2/mc-1-big-Digest.crc32                |   1 +
 .../table2/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Statistics.db               | Bin 0 -> 4686 bytes
 .../table2/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table2/mc-1-big-TOC.txt                     |   8 +
 .../table3/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/mc-1-big-Data.db                     | Bin 0 -> 111 bytes
 .../table3/mc-1-big-Digest.crc32                |   1 +
 .../table3/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Statistics.db               | Bin 0 -> 4732 bytes
 .../table3/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table3/mc-1-big-TOC.txt                     |   8 +
 .../table4/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/mc-1-big-Data.db                     | Bin 0 -> 108 bytes
 .../table4/mc-1-big-Digest.crc32                |   1 +
 .../table4/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Statistics.db               | Bin 0 -> 4742 bytes
 .../table4/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table4/mc-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 327 ++++++++++++++++++-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 57 files changed, 867 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --cc CASSANDRA-14092.txt
index 0000000,5ac872c..f95380b
mode 000000,100644..100644
--- a/CASSANDRA-14092.txt
+++ b/CASSANDRA-14092.txt
@@@ -1,0 -1,81 +1,81 @@@
+ CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+ ---------------------------------------------
+ 
+ The maximum expiration timestamp that can be represented by the storage engine is
+ 2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+ after this date are not currently supported.
+ 
+ # Expiration Date Overflow Policy
+ 
+ We plan to lift this limitation in newer versions, but while the fix is not available,
+ operators can decide which policy to apply when dealing with inserts with TTL exceeding
+ the maximum supported expiration date:
+   -     REJECT: this is the default policy and will reject any requests with expiration
+                 date timestamp after 2038-01-19T03:14:06+00:00.
+   -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                 2038-01-19T03:14:06+00:00 and the client will receive a warning.
+   - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+ 
+ These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
 -startup option.
++startup option in the jvm.options configuration file.
+ 
+ # Potential data loss on earlier versions
+ 
+ Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+ INSERTS with TTL expiring after the maximum supported date, causing the expiration
+ time field to overflow and the records to expire immediately. Expired records due
+ to overflow will not be queryable and will be permanently removed after a compaction.
+ 
+ 2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+ since an AssertionError is thrown during INSERT when the expiration time field overflows
+ on these versions. When assertions are disabled then it is possible to INSERT entries
+ with overflowed local expiration time and even the earlier versions are subject to data
+ loss due to this bug.
+ 
+ This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+ of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+ the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+ For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+ will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+ where this issue is addressed as soon as possible.
+ 
+ # Data Recovery
+ 
+ SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+ with overflowed expiration time that were backed up or did not go through compaction can
+ be recovered by reinserting overflowed entries with a valid expiration time and a higher
+ timestamp, since tombstones may have been generated with the original timestamp.
+ 
+ To find out if an SSTable has an entry with overflowed expiration, inspect it with the
 -sstable2json tool and look for a negative "local deletion time" field. SSTables in this
 -condition should be backed up immediately, as they are subject to data loss during
++'sstablemetadata' tool and look for a negative "Minimum local deletion time" field. SSTables
++in this condition should be backed up immediately, as they are subject to data loss during
+ compaction.
+ 
+ A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+ rows with overflowed expiration time with the maximum expiration date of
+ 2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+ for recovery of SSTables via scrub:
+ 
+ - Offline scrub:
+    - Clone the data directory tree to another location, keeping only the folders and the
+      contents of the system tables.
+    - Clone the configuration directory to another location, setting the data_file_directories
+      property to the cloned data directory in the cloned cassandra.yaml.
+    - Copy the affected SSTables to the cloned data location of the affected table.
+    - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+    - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+          WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+          compaction, so the data with overflowed will be removed - make sure to back up
+          your SSTables before running scrub.
+    - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+    - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+ 
+ - Online scrub:
+    - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+      as otherwise, the data may be removed permanently during compaction.
+    - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+    - run "nodetool refresh" to load the SSTables.
+    - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+    - Re-enable compactions after verifying that scrub recovered the missing entries.
+ 
 -See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
++See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index f42f3f4,82da6ad..a492c42
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,29 -1,4 +1,30 @@@
 -2.2.12
 +3.0.16
++ * Protect against overflow of local expiration time (CASSANDRA-14092)
 + * Use the correct digest file and reload sstable metadata in nodetool verify (CASSANDRA-14217)
 + * Handle failure when mutating repaired status in Verifier (CASSANDRA-13933)
 + * Close socket on error during connect on OutboundTcpConnection (CASSANDRA-9630)
 + * Set encoding for javadoc generation (CASSANDRA-14154)
 + * Fix index target computation for dense composite tables with dropped compact storage (CASSANDRA-14104)
 + * Improve commit log chain marker updating (CASSANDRA-14108)
 + * Extra range tombstone bound creates double rows (CASSANDRA-14008)
 + * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
 + * Accept role names containing forward-slash (CASSANDRA-14088)
 + * Optimize CRC check chance probability calculations (CASSANDRA-14094)
 + * Fix cleanup on keyspace with no replicas (CASSANDRA-13526)
 + * Fix updating base table rows with TTL not removing materialized view entries (CASSANDRA-14071)
 + * Reduce garbage created by DynamicSnitch (CASSANDRA-14091)
 + * More frequent commitlog chained markers (CASSANDRA-13987)
 + * Fix serialized size of DataLimits (CASSANDRA-14057)
 + * Add flag to allow dropping oversized read repair mutations (CASSANDRA-13975)
 + * Fix SSTableLoader logger message (CASSANDRA-14003)
 + * Fix repair race that caused gossip to block (CASSANDRA-13849)
 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964)
 + * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959)
 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813)
 + * Provide a JMX call to sync schema with local storage (CASSANDRA-13954)
 + * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939)
 + * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922)
 +Merged from 2.2:
   * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
   * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
   * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 1bbe1aa,4fe3508..f574c33
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,71 -33,7 +33,72 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.12
 +3.0.16
 +=====
 +
 +Upgrading
 +---------
++   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +     options are used. See CASSANDRA-13006 for more details.
 +   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +     for more details.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
 +3.0.15
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous upgrading sections,
 +     especially if you are upgrading from 2.2.
 +
 +Compact Storage
 +---------------
 +    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
 +      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
 +      exposing internal structure of Thrift/Compact Tables. You can find more details
 +      on exposed internal structure under: 
 +      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
 +
 +      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
 +      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
 +      current connection.
 +
 +      In order to upgrade, clients supporting a non-compact schema view can be rolled out
 +      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
 +      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
 +      after that.
 +
 +Materialized Views
 +-------------------
 +    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
 +    - A change was made in the way the Materialized View timestamp is computed, which
 +      may cause an old deletion to a base column which is view primary key (PK) column
 +      to not be reflected in the view when repairing the base table post-upgrade. This
 +      condition is only possible when a column deletion to an MV primary key (PK) column
 +      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
 +      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
 +      If such column deletions are done on a view PK column which is not a base PK, it's advisable
 +      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
 +      to fix potential inconsistencies by running repair on the views after upgrade or drop and
 +      re-create the views. See CASSANDRA-11500 for more details.
 +    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
 +      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
 +      situations so we advise against doing deletions on base columns not selected in views
 +      until this is fixed on CASSANDRA-13826.
 +
 +3.0.14
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/conf/jvm.options
----------------------------------------------------------------------
diff --cc conf/jvm.options
index a7b3bd8,0000000..eb2ad19
mode 100644,000000..100644
--- a/conf/jvm.options
+++ b/conf/jvm.options
@@@ -1,108 -1,0 +1,119 @@@
 +###########################################################################
 +#                             jvm.options                                 #
 +#                                                                         #
 +# - all flags defined here will be used by cassandra to startup the JVM   #
 +# - one flag should be specified per line                                 #
 +# - lines that do not start with '-' will be ignored                      #
 +# - only static flags are accepted (no variables or parameters)           #
 +# - dynamic flags will be appended to these on cassandra-env              #
 +###########################################################################
 +
 +#################
 +# HEAP SETTINGS #
 +#################
 +
 +# Heap size is automatically calculated by cassandra-env based on this
 +# formula: max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
 +# That is:
 +# - calculate 1/2 ram and cap to 1024MB
 +# - calculate 1/4 ram and cap to 8192MB
 +# - pick the max
 +#
 +# For production use you may wish to adjust this for your environment.
 +# If that's the case, uncomment the -Xmx and Xms options below to override the
 +# automatic calculation of JVM heap memory.
 +#
 +# It is recommended to set min (-Xms) and max (-Xmx) heap sizes to
 +# the same value to avoid stop-the-world GC pauses during resize, and
 +# so that we can lock the heap in memory on startup to prevent any
 +# of it from being swapped out.
 +#-Xms4G
 +#-Xmx4G
 +
 +# Young generation size is automatically calculated by cassandra-env
 +# based on this formula: min(100 * num_cores, 1/4 * heap size)
 +#
 +# The main trade-off for the young generation is that the larger it
 +# is, the longer GC pause times will be. The shorter it is, the more
 +# expensive GC will be (usually).
 +#
 +# It is not recommended to set the young generation size if using the
 +# G1 GC, since that will override the target pause-time goal.
 +# More info: http://www.oracle.com/technetwork/articles/java/g1gc-1984535.html
 +#
 +# The example below assumes a modern 8-core+ machine for decent
 +# times. If in doubt, and if you do not particularly want to tweak, go
 +# 100 MB per physical CPU core.
 +#-Xmn800M
 +
++###################################
++# EXPIRATION DATE OVERFLOW POLICY #
++###################################
++
++# Defines how to handle INSERT requests with TTL exceeding the maximum supported expiration date:
++# * REJECT: this is the default policy and will reject any requests with expiration date timestamp after 2038-01-19T03:14:06+00:00.
++# * CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on 2038-01-19T03:14:06+00:00 and the client will receive a warning.
++# * CAP_NOWARN: same as previous, except that the client warning will not be emitted.
++#
++#-Dcassandra.expiration_date_overflow_policy=REJECT
++
 +#################
 +#  GC SETTINGS  #
 +#################
 +
 +### CMS Settings
 +
 +-XX:+UseParNewGC
 +-XX:+UseConcMarkSweepGC
 +-XX:+CMSParallelRemarkEnabled
 +-XX:SurvivorRatio=8
 +-XX:MaxTenuringThreshold=1
 +-XX:CMSInitiatingOccupancyFraction=75
 +-XX:+UseCMSInitiatingOccupancyOnly
 +-XX:CMSWaitDuration=10000
 +-XX:+CMSParallelInitialMarkEnabled
 +-XX:+CMSEdenChunksRecordAlways
 +# some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541
 +-XX:+CMSClassUnloadingEnabled
 +
 +### G1 Settings (experimental, comment previous section and uncomment section below to enable)
 +
 +## Use the Hotspot garbage-first collector.
 +#-XX:+UseG1GC
 +#
 +## Have the JVM do less remembered set work during STW, instead
 +## preferring concurrent GC. Reduces p99.9 latency.
 +#-XX:G1RSetUpdatingPauseTimePercent=5
 +#
 +## Main G1GC tunable: lowering the pause target will lower throughput and vise versa.
 +## 200ms is the JVM default and lowest viable setting
 +## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml.
 +#-XX:MaxGCPauseMillis=500
 +
 +## Optional G1 Settings
 +
 +# Save CPU time on large (>= 16GB) heaps by delaying region scanning
 +# until the heap is 70% full. The default in Hotspot 8u40 is 40%.
 +#-XX:InitiatingHeapOccupancyPercent=70
 +
 +# For systems with > 8 cores, the default ParallelGCThreads is 5/8 the number of logical cores.
 +# Otherwise equal to the number of cores when 8 or less.
 +# Machines with > 10 cores should try setting these to <= full cores.
 +#-XX:ParallelGCThreads=16
 +# By default, ConcGCThreads is 1/4 of ParallelGCThreads.
 +# Setting both to the same value can reduce STW durations.
 +#-XX:ConcGCThreads=16
 +
 +### GC logging options -- uncomment to enable
 +
 +-XX:+PrintGCDetails
 +-XX:+PrintGCDateStamps
 +-XX:+PrintHeapAtGC
 +-XX:+PrintTenuringDistribution
 +-XX:+PrintGCApplicationStoppedTime
 +-XX:+PrintPromotionFailure
 +#-XX:PrintFLSStatistics=1
 +#-Xloggc:/var/log/cassandra/gc.log
 +-XX:+UseGCLogFileRotation
 +-XX:NumberOfGCLogFiles=10
 +-XX:GCLogFileSize=10M

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/redhat/cassandra.spec
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 4ed0f83,84f423a..832d0a7
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,12 -18,17 +18,11 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
--import java.util.Collections;
 -import java.util.concurrent.TimeUnit;
 -
 -import com.google.common.collect.Iterables;
 -import com.google.common.annotations.VisibleForTesting;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 +import java.util.List;
  
- import com.google.common.collect.Iterables;
- 
+ import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.functions.Function;
 -import org.apache.cassandra.db.ExpiringCell;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
@@@ -100,10 -137,13 +99,13 @@@ public class Attribute
          return LongType.instance.compose(tval);
      }
  
-     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+     public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
      {
          if (timeToLive == null)
-             return 0;
+         {
 -            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
 -            return metadata.getDefaultTimeToLive();
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
++            return metadata.params.defaultTimeToLive;
+         }
  
          ByteBuffer tval = timeToLive.bindAndGet(options);
          if (tval == null)
@@@ -125,9 -165,11 +127,11 @@@
          if (ttl < 0)
              throw new InvalidRequestException("A TTL must be greater or equal to 0, but was " + ttl);
  
 -        if (ttl > ExpiringCell.MAX_TTL)
 -            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 +        if (ttl > MAX_TTL)
 +            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, MAX_TTL));
  
 -        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
++        ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+ 
          return ttl;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 536681f,45908de..b5946bb
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1379,13 -1610,12 +1379,13 @@@ public class ColumnFamilyStore implemen
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
++        return scrub(disableSnapshot, skipCorrupted, reinsertOverflowedTTL, false, checkData, jobs);
      }
  
 -    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean reinsertOverflowedTTL, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
@@@ -1393,7 -1623,7 +1393,7 @@@
  
          try
          {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 -            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTL, jobs);
          }
          catch(Throwable t)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
index 0000000,0000000..852dcb1
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
@@@ -1,0 -1,0 +1,121 @@@
++/*
++ * 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;
++
++import java.util.concurrent.TimeUnit;
++
++import com.google.common.annotations.VisibleForTesting;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++import org.slf4j.helpers.MessageFormatter;
++
++import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.cql3.Attributes;
++import org.apache.cassandra.db.rows.BufferCell;
++import org.apache.cassandra.db.rows.Cell;
++import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.service.ClientWarn;
++import org.apache.cassandra.utils.NoSpamLogger;
++
++public class ExpirationDateOverflowHandling
++{
++    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
++
++    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
++
++    public enum ExpirationDateOverflowPolicy
++    {
++        REJECT, CAP_NOWARN, CAP
++    }
++
++    @VisibleForTesting
++    public static ExpirationDateOverflowPolicy policy;
++
++    static {
++        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
++        try
++        {
++            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
++        }
++        catch (RuntimeException e)
++        {
++            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
++            policy = ExpirationDateOverflowPolicy.REJECT;
++        }
++    }
++
++    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
++                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
++                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
++                                                                          "is fixed. See CASSANDRA-14092 for more details.";
++
++    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
++                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
++                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
++                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
++
++    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
++    {
++        if (ttl == BufferCell.NO_TTL)
++            return;
++
++        // Check for localExpirationTime overflow (CASSANDRA-14092)
++        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
++        if (ttl + nowInSecs < 0)
++        {
++            switch (policy)
++            {
++                case CAP:
++                    ClientWarn.instance.warn(MessageFormatter.arrayFormat(MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING, new Object[] { metadata.ksName,
++                                                                                                                                   metadata.cfName,
++                                                                                                                                   isDefaultTTL? "default " : "", ttl })
++                                                             .getMessage());
++                case CAP_NOWARN:
++                    /**
++                     * Capping at this stage is basically not rejecting the request. The actual capping is done
++                     * by {@link #computeLocalExpirationTime(int, int)}, which converts the negative TTL
++                     * to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
++                     */
++                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES, TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
++                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
++                    return;
++
++                default:
++                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
++                                                                    isDefaultTTL? "default " : "", ttl));
++            }
++        }
++    }
++
++    /**
++     * This method computes the {@link Cell#localDeletionTime()}, maybe capping to the maximum representable value
++     * which is {@link Cell#MAX_DELETION_TIME}.
++     *
++     * Please note that the {@link ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy} is applied
++     * during {@link ExpirationDateOverflowHandling#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
++     * so if the request was not denied it means its expiration date should be capped.
++     *
++     * See CASSANDRA-14092
++     */
++    public static int computeLocalExpirationTime(int nowInSec, int timeToLive)
++    {
++        int localExpirationTime = nowInSec + timeToLive;
++        return localExpirationTime >= 0? localExpirationTime : Cell.MAX_DELETION_TIME;
++    }
++}


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


[12/29] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by pa...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-2.2
Commit: 1602e606348959aead18531cb8027afb15f276e7
Parents: aa831c9 b294943
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:53:20 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:54:56 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  79 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  11 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  14 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/lb-1-big-Data.db                     | Bin 0 -> 103 bytes
 .../table1/lb-1-big-Digest.adler32              |   1 +
 .../table1/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table1/lb-1-big-Statistics.db               | Bin 0 -> 4466 bytes
 .../table1/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table1/lb-1-big-TOC.txt                     |   8 +
 .../table2/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/lb-1-big-Data.db                     | Bin 0 -> 98 bytes
 .../table2/lb-1-big-Digest.adler32              |   1 +
 .../table2/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table2/lb-1-big-Statistics.db               | Bin 0 -> 4478 bytes
 .../table2/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table2/lb-1-big-TOC.txt                     |   8 +
 .../table3/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table3/lb-1-big-Digest.adler32              |   1 +
 .../table3/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table3/lb-1-big-Statistics.db               | Bin 0 -> 4482 bytes
 .../table3/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table3/lb-1-big-TOC.txt                     |   8 +
 .../table4/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table4/lb-1-big-Digest.adler32              |   1 +
 .../table4/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table4/lb-1-big-Statistics.db               | Bin 0 -> 4494 bytes
 .../table4/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table4/lb-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 405 +++++++++++++++++++
 .../unit/org/apache/cassandra/db/ScrubTest.java |   2 +-
 58 files changed, 847 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0f6e61c,9332354..82da6ad
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -2.1.20
 +2.2.12
 + * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
 + * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
 + * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
 + * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 +Merged from 2.1:
+  * Protect against overflow of local expiration time (CASSANDRA-14092)
   * More PEP8 compliance for cqlsh (CASSANDRA-14021)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 5747941,fb6b4ee..4fe3508
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,15 -38,9 +38,16 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
 -   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 -
 -2.1.19
++    - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +      options are used. See CASSANDRA-13006 for more details.
 +    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +      for more details.
 +
 +2.2.11
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/debian/rules
----------------------------------------------------------------------
diff --cc debian/rules
index 35f5a51,70db61c..ff1d64d
--- a/debian/rules
+++ b/debian/rules
@@@ -63,7 -64,7 +63,7 @@@ binary-indep: build instal
  	dh_testroot
  	dh_installchangelogs
  	dh_installinit -u'start 50 2 3 4 5 . stop 50 0 1 6 .'
- 	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html
 -	dh_installdocs README.asc CHANGES.txt NEWS.txt
++	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html CASSANDRA-14092.txt
  	dh_installexamples tools/*.yaml
  	dh_bash-completion
  	dh_compress

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/redhat/cassandra.spec
----------------------------------------------------------------------
diff --cc redhat/cassandra.spec
index 0d4b271,a3f09b0..07c3dc5
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@@ -114,10 -113,10 +114,10 @@@ exit 
  
  %files
  %defattr(0644,root,root,0755)
--%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt
 -%attr(755,root,root) %{_bindir}/cassandra-cli
++%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt
  %attr(755,root,root) %{_bindir}/cassandra-stress
  %attr(755,root,root) %{_bindir}/cqlsh
 +%attr(755,root,root) %{_bindir}/cqlsh.py
  %attr(755,root,root) %{_bindir}/debug-cql
  %attr(755,root,root) %{_bindir}/nodetool
  %attr(755,root,root) %{_bindir}/sstablekeys

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 7b38e9f,23571ca..84f423a
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,17 -18,19 +18,23 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
 +import java.util.Collections;
+ import java.util.concurrent.TimeUnit;
  
 +import com.google.common.collect.Iterables;
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.db.ExpiringCell;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
  import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.NoSpamLogger;
  
  /**
   * Utility class for the Parser to gather attributes for modification

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 7e36e11,2989b9d..45908de
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -905,26 -879,13 +905,26 @@@ public class ColumnFamilyStore implemen
                                                  previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
 -            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
 -            flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 -            postFlushExecutor.submit(task);
 +            ListenableFutureTask<Void> flushTask = ListenableFutureTask.create(flush, null);
 +            flushExecutor.execute(flushTask);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
 +            postFlushExecutor.execute(task);
  
              @SuppressWarnings("unchecked")
-             ListenableFuture<ReplayPosition> future = 
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future =
 +                    // If either of the two tasks errors out, resulting future must also error out.
 +                    // Combine the two futures and only return post-flush result after both have completed.
 +                    // Note that flushTask will always yield null, but Futures.allAsList is
 +                    // order preserving, which is why the transform function returns the result
 +                    // from item 1 in it's input list (i.e. what was yielded by task).
 +                    Futures.transform(Futures.allAsList(flushTask, task),
 +                                      new Function<List<Object>, ReplayPosition>()
 +                                      {
 +                                          public ReplayPosition apply(List<Object> input)
 +                                          {
 +                                              return (ReplayPosition) input.get(1);
 +                                          }
 +                                      });
              return future;
          }
      }
@@@ -1610,29 -1516,48 +1610,28 @@@
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        // skip snapshot creation during scrub, SEE JIRA 5891
 -        if(!disableSnapshot)
 -            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 -    }
 -
 -    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
 -    {
 -        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
 -    }
 -
 -    public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
 -    {
 -        assert !sstables.isEmpty();
 -        data.markObsolete(sstables, compactionType);
 -    }
 -
 -    void replaceFlushed(Memtable memtable, SSTableReader sstable)
 -    {
 -        compactionStrategyWrapper.replaceFlushed(memtable, sstable);
 -    }
 -
 -    public boolean isValid()
 -    {
 -        return valid;
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
      }
  
-     @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public long getMemtableColumnsCount()
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
 -        return metric.memtableColumnsCount.value();
 -    }
 +        // skip snapshot creation during scrub, SEE JIRA 5891
 +        if(!disableSnapshot)
 +            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
  
 -    public long getMemtableDataSize()
 -    {
 -        return metric.memtableOnHeapSize.value();
 -    }
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
  
 -    public int getMemtableSwitchCount()
 -    {
 -        return (int) metric.memtableSwitchCount.count();
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 3350b20,6e3634a..d90abe9
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -348,9 -358,16 +348,15 @@@ public class CompactionManager implemen
          }
      }
  
 -    @Deprecated
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
 +    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
 -                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    throws InterruptedException, ExecutionException
+     {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -360,30 -377,11 +366,30 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
+                 scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
              }
 -        }, jobs);
 +        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
 +        }, 0, OperationType.VERIFY);
      }
  
      public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
@@@ -730,14 -717,14 +736,14 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
      {
 -        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 +        CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
 -        metrics.beginCompaction(scrubInfo);
 -        try
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
          {
 +            scrubInfo = scrubber.getScrubInfo();
 +            metrics.beginCompaction(scrubInfo);
              scrubber.scrub();
          }
          finally

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index b6b20fb,6d4537c..affee11
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -22,33 -22,30 +22,36 @@@ import java.io.*
  import java.util.*;
  
  import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.base.Predicates;
  import com.google.common.base.Throwables;
  import com.google.common.collect.AbstractIterator;
 -import com.google.common.collect.Sets;
  
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+ import org.apache.cassandra.db.composites.CellNames;
  import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.OutputHandler;
+ import org.apache.cassandra.utils.memory.HeapAllocator;
 +import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.concurrent.Refs;
  
  public class Scrubber implements Closeable
  {
      private final ColumnFamilyStore cfs;
      private final SSTableReader sstable;
 +    private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
 -    public final boolean validateColumns;
+     private final boolean reinsertOverflowedTTLRows;
  
      private final CompactionController controller;
      private final boolean isCommutative;
@@@ -84,20 -81,27 +88,28 @@@
      };
      private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
 +    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
+     }
+ 
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
 -        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
 +    @SuppressWarnings("resource")
-     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData) throws IOException
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
 -        this.sstable = sstable;
 +        this.transaction = transaction;
 +        this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.isOffline = isOffline;
 -        this.validateColumns = checkData;
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
  
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
@@@ -322,7 -339,7 +339,7 @@@
          // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
          // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
          // to the outOfOrderRows that will be later written to a new SSTable.
-         OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, checkData),
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
++        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
                                                                cfs.metadata.comparator.onDiskAtomComparator());
          if (prevKey != null && prevKey.compareTo(key) > 0)
          {
@@@ -342,6 -359,18 +359,18 @@@
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
++    private OnDiskAtomIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
++        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
+         return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                     outputHandler,
+                                                                                     negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 6896062,d718765..e416c7b
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@@ -31,8 -32,8 +32,9 @@@ import org.apache.cassandra.cql3.Attrib
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.context.CounterContext;
  import org.apache.cassandra.dht.IPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 +import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.CounterId;
  import org.apache.cassandra.utils.Pair;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 1ecedac,a7a8ca7..2c9ac4d
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2611,10 -2415,16 +2611,16 @@@ public class StorageService extends Not
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                      int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 761eed6,90c0fb5..f336bcc
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -265,15 -274,10 +265,18 @@@ public interface StorageServiceMBean ex
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++
 +    /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Rewrite all sstables to the latest version.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d735676,10e7185..8bdf9dc
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -312,9 -313,9 +313,9 @@@ public class ThriftValidatio
          if (cosc.column != null)
          {
              if (isCommutative)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
              validateColumnData(metadata, key, null, cosc.column);
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 24c5874,fcd4110..17bef02
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -236,16 -243,11 +236,16 @@@ public class NodeProbe implements AutoC
          return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
@@@ -267,22 -268,13 +267,22 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+         if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index a486a13,59d13d5..f5e84c5
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -119,10 -115,10 +126,10 @@@ public class StandaloneScrubbe
              {
                  for (SSTableReader sstable : sstables)
                  {
 -                    try
 +                    try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
 -                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
 -                        try
 +                        txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
                          {
                              scrubber.scrub();
                          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index dafe8d1,0000000..50224a0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,76 -1,0 +1,82 @@@
 +/*
 + * 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.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
++import org.apache.cassandra.tools.StandaloneScrubber;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
++    @Option(title = "reinsert_overflowed_ttl",
++    name = {"r", "--reinsert-overflowed-ttl"},
++    description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
++    private boolean reinsertOverflowedTTL = false;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
index 0000000,0000000..d7cc13b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
index 0000000,0000000..51213c2
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
index 0000000,0000000..d5b12df
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2292388625

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
index 0000000,0000000..3ab96ee
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
index 0000000,0000000..e8cc7e0
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
index 0000000,0000000..38373b4
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
index 0000000,0000000..762a229
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
index 0000000,0000000..ae89849
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3829731931

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
index 0000000,0000000..38a6e4c
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
index 0000000,0000000..64dab43
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
index 0000000,0000000..04a7384
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
index 0000000,0000000..33145df
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
index 0000000,0000000..2a542cd
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3574474340

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
index 0000000,0000000..5fb34e8
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
index 0000000,0000000..51203ae
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
index 0000000,0000000..c814fef
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
index 0000000,0000000..f40e71f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
index 0000000,0000000..e6675e4
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2405377913

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
index 0000000,0000000..8291383
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
index 0000000,0000000..2217c2d
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 0000000,ab4ef21..b1eaac1
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -1,0 -1,410 +1,405 @@@
+ package org.apache.cassandra.cql3.validation.operations;
+ 
+ import java.io.File;
+ import java.io.FileInputStream;
+ import java.io.FileOutputStream;
+ import java.io.IOException;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
+ 
+ import org.apache.cassandra.cql3.Attributes;
+ import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
+ import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.ExpiringCell;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
+ import org.junit.Test;
+ 
+ public class TTLTest extends CQLTester
+ {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
+     public static int MAX_TTL = ExpiringCell.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
+ 
+     @Test
+     public void testTTLPerRequestLimit() throws Throwable
+     {
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         // insert with low TTL should not be denied
+         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+         execute("TRUNCATE %s");
+ 
+         // insert with low TTL should not be denied
+         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+     }
+ 
+ 
+     @Test
+     public void testTTLDefaultLimit() throws Throwable
+     {
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live cannot be smaller than 0"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                         + (MAX_TTL + 1));
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
+         }
+ 
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
+     public void testRejectExpirationDateOverflowPolicy() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         checkTTLIsCapped("i");
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyPerRequest() throws Throwable
+     {
+         // Test cap policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+ 
+         // simple column, clustering, flush
+         baseCapExpirationDateOverflowTest(true, true, true);
+         // simple column, clustering, noflush
+         baseCapExpirationDateOverflowTest(true, true, false);
+         // simple column, noclustering, flush
+         baseCapExpirationDateOverflowTest(true, false, true);
+         // simple column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(true, false, false);
+         // complex column, clustering, flush
+         baseCapExpirationDateOverflowTest(false, true, true);
+         // complex column, clustering, noflush
+         baseCapExpirationDateOverflowTest(false, true, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, true);
+         // complex column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(false, false, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, false);
+ 
+         // Return to previous policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        createTable(true, true);
 -        createTable(true, false);
 -        createTable(false, true);
 -        createTable(false, false);
 -
+         baseTestRecoverOverflowedExpiration(false, false);
+         baseTestRecoverOverflowedExpiration(true, false);
+         baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
+     public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
+         if (simple)
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
+     }
+ 
+     /**
+      * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+      * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
+         UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+      * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+         return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
++        createTable(simple, clustering);
++
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getLiveSSTableCount());
++        assertEquals(0, cfs.getSSTables().size());
+ 
 -        copySSTablesToTableDir(simple, clustering);
++        copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
+             cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -            {
 -                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
 -                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
 -
 -            }
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
 -            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
++            assertEmpty(execute("SELECT * from %s"));
+         }
 -        cfs.truncateBlocking(); //cleanup for next tests
+     }
+ 
 -    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
++    private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
 -        File sourceDir = getTableDir(simple, clustering);
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering) throws Throwable
++    private static File getTableDir(String table, boolean simple, boolean clustering)
++    {
++        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
++    }
++
++    private void createTable(boolean simple, boolean clustering)
+     {
+         if (simple)
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+     }
+ 
+     private static File getTableDir(boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 4cca7ff,4efd082..9b1ede4
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -654,129 -565,4 +654,129 @@@ public class ScrubTes
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }


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


[08/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
Protect against overflow of local expiration time

Patch by Paulo Motta; Reviewed by Sam Tunnicliffe for CASSANDRA-14092


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

Branch: refs/heads/cassandra-3.0
Commit: b2949439ec62077128103540e42570238520f4ee
Parents: 5ba9e6d
Author: Paulo Motta <pa...@gmail.com>
Authored: Thu Feb 1 04:01:28 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:33:50 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  23 +-
 .../cassandra/cql/AbstractModification.java     |   4 +
 .../org/apache/cassandra/cql/Attributes.java    |  19 +
 .../apache/cassandra/cql/BatchStatement.java    |   4 +
 .../org/apache/cassandra/cql/CFPropDefs.java    |   7 +
 .../org/apache/cassandra/cql3/Attributes.java   |  81 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   4 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  15 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |  17 +-
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  12 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  23 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 ...test_keyspace-table1-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table1-ka-1-Data.db       | Bin 0 -> 103 bytes
 .../cql_test_keyspace-table1-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table1-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table1-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table1-ka-1-Statistics.db | Bin 0 -> 4454 bytes
 .../cql_test_keyspace-table1-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table1-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table2-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table2-ka-1-Data.db       | Bin 0 -> 96 bytes
 .../cql_test_keyspace-table2-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table2-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table2-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table2-ka-1-Statistics.db | Bin 0 -> 4466 bytes
 .../cql_test_keyspace-table2-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table2-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table3-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table3-ka-1-Data.db       | Bin 0 -> 182 bytes
 .../cql_test_keyspace-table3-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table3-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table3-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table3-ka-1-Statistics.db | Bin 0 -> 4470 bytes
 .../cql_test_keyspace-table3-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table3-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table4-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table4-ka-1-Data.db       | Bin 0 -> 181 bytes
 .../cql_test_keyspace-table4-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table4-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table4-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table4-ka-1-Statistics.db | Bin 0 -> 4482 bytes
 .../cql_test_keyspace-table4-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table4-ka-1-TOC.txt       |   8 +
 .../cql3/validation/operations/TTLTest.java     | 410 +++++++++++++++++++
 59 files changed, 898 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --git a/CASSANDRA-14092.txt b/CASSANDRA-14092.txt
new file mode 100644
index 0000000..5ac872c
--- /dev/null
+++ b/CASSANDRA-14092.txt
@@ -0,0 +1,81 @@
+CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+---------------------------------------------
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+after this date are not currently supported.
+
+# Expiration Date Overflow Policy
+
+We plan to lift this limitation in newer versions, but while the fix is not available,
+operators can decide which policy to apply when dealing with inserts with TTL exceeding
+the maximum supported expiration date:
+  -     REJECT: this is the default policy and will reject any requests with expiration
+                date timestamp after 2038-01-19T03:14:06+00:00.
+  -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                2038-01-19T03:14:06+00:00 and the client will receive a warning.
+  - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+
+These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
+startup option.
+
+# Potential data loss on earlier versions
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+INSERTS with TTL expiring after the maximum supported date, causing the expiration
+time field to overflow and the records to expire immediately. Expired records due
+to overflow will not be queryable and will be permanently removed after a compaction.
+
+2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+since an AssertionError is thrown during INSERT when the expiration time field overflows
+on these versions. When assertions are disabled then it is possible to INSERT entries
+with overflowed local expiration time and even the earlier versions are subject to data
+loss due to this bug.
+
+This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+where this issue is addressed as soon as possible.
+
+# Data Recovery
+
+SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+with overflowed expiration time that were backed up or did not go through compaction can
+be recovered by reinserting overflowed entries with a valid expiration time and a higher
+timestamp, since tombstones may have been generated with the original timestamp.
+
+To find out if an SSTable has an entry with overflowed expiration, inspect it with the
+sstable2json tool and look for a negative "local deletion time" field. SSTables in this
+condition should be backed up immediately, as they are subject to data loss during
+compaction.
+
+A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+rows with overflowed expiration time with the maximum expiration date of
+2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+for recovery of SSTables via scrub:
+
+- Offline scrub:
+   - Clone the data directory tree to another location, keeping only the folders and the
+     contents of the system tables.
+   - Clone the configuration directory to another location, setting the data_file_directories
+     property to the cloned data directory in the cloned cassandra.yaml.
+   - Copy the affected SSTables to the cloned data location of the affected table.
+   - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+   - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+         WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+         compaction, so the data with overflowed will be removed - make sure to back up
+         your SSTables before running scrub.
+   - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+   - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+
+- Online scrub:
+   - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+     as otherwise, the data may be removed permanently during compaction.
+   - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+   - run "nodetool refresh" to load the SSTables.
+   - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+   - Re-enable compactions after verifying that scrub recovered the missing entries.
+
+See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e17093d..9332354 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.20
+ * Protect against overflow of local expiration time (CASSANDRA-14092)
  * More PEP8 compliance for cqlsh (CASSANDRA-14021)
  * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 796a424..fb6b4ee 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -1,3 +1,23 @@
+PLEASE READ: MAXIMUM TTL EXPIRATION DATE NOTICE (CASSANDRA-14092)
+------------------------------------------------------------------
+(General upgrading instructions are available in the next section)
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that inserts with TTL thatl expire after
+this date are not currently supported. By default, INSERTS with TTL exceeding the
+maximum supported date are rejected, but it's possible to choose a different
+ expiration overflow policy. See CASSANDRA-14092.txt for more details.
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x) there was no protection against INSERTS
+with TTL expiring after the maximum supported date, causing the expiration time
+field to overflow and the records to expire immediately. Clusters in the 2.X and
+lower series are not subject to this when assertions are enabled. Backed up SSTables
+can be potentially recovered and recovery instructions can be found on the
+CASSANDRA-14092.txt file.
+
+If you use or plan to use very large TTLS (10 to 20 years), read CASSANDRA-14092.txt
+for more information.
+
 GENERAL UPGRADING ADVICE FOR ANY VERSION
 ========================================
 
@@ -18,8 +38,7 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
+   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 
 2.1.19
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/AbstractModification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
index 8da2611..e98764b 100644
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ b/src/java/org/apache/cassandra/cql/AbstractModification.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.cql;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.thrift.ThriftClientState;
+import org.w3c.dom.Attr;
 
 public abstract class AbstractModification
 {
@@ -89,6 +92,7 @@ public abstract class AbstractModification
 
     public int getTimeToLive()
     {
+        Attributes.maybeApplyExpirationDateOverflowPolicy(keyspace, columnFamily, timeToLive);
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
index faee3b8..c1c37ef 100644
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ b/src/java/org/apache/cassandra/cql/Attributes.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.cql;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
  * Class to contain attributes for statements
@@ -73,4 +76,20 @@ public class Attributes
         return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
     }
 
+    public static void maybeApplyExpirationDateOverflowPolicy(String keyspace, String columnFamily, Integer timeToLive)
+    {
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
+        if (metadata != null)
+        {
+            try
+            {
+                org.apache.cassandra.cql3.Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, timeToLive, false);
+            }
+            catch (InvalidRequestException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
index b141bcc..e5a95b8 100644
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql/BatchStatement.java
@@ -72,6 +72,10 @@ public class BatchStatement
 
     public int getTimeToLive()
     {
+        for (AbstractModification statement : statements)
+        {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(statement.keyspace, statement.columnFamily, timeToLive);
+        }
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
index f65cb94..a0c8d0d 100644
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql/CFPropDefs.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -191,6 +192,12 @@ public class CFPropDefs {
                         KW_DEFAULT_TIME_TO_LIVE,
                         0,
                         CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
+
+            if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+                throw new InvalidRequestException(String.format("%s must be less than or equal to %d (got %s)",
+                                                                KW_DEFAULT_TIME_TO_LIVE,
+                                                                ExpiringCell.MAX_TTL,
+                                                                defaultTimeToLive));
         }
 
         CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index 435757b..23571ca 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.NoSpamLogger;
 
 /**
  * Utility class for the Parser to gather attributes for modification
@@ -32,6 +38,41 @@ import org.apache.cassandra.serializers.MarshalException;
  */
 public class Attributes
 {
+    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
+
+    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
+
+    public enum ExpirationDateOverflowPolicy
+    {
+        REJECT, CAP
+    }
+
+    @VisibleForTesting
+    public static ExpirationDateOverflowPolicy policy;
+
+    static {
+        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
+        try
+        {
+            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
+        }
+        catch (RuntimeException e)
+        {
+            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
+            policy = ExpirationDateOverflowPolicy.REJECT;
+        }
+    }
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
+                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
+                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
+                                                                          "is fixed. See CASSANDRA-14092 for more details.";
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
+                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
+                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
+                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
+
     private final Term timestamp;
     private final Term timeToLive;
 
@@ -77,10 +118,13 @@ public class Attributes
         return LongType.instance.compose(tval);
     }
 
-    public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+    public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
     {
         if (timeToLive == null)
-            return 0;
+        {
+            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
+            return metadata.getDefaultTimeToLive();
+        }
 
         ByteBuffer tval = timeToLive.bindAndGet(options);
         if (tval == null)
@@ -102,6 +146,8 @@ public class Attributes
         if (ttl > ExpiringCell.MAX_TTL)
             throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 
+        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+
         return ttl;
     }
 
@@ -135,4 +181,33 @@ public class Attributes
             return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[ttl]", true), Int32Type.instance);
         }
     }
+
+    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
+    {
+        if (ttl == 0)
+            return;
+
+        // Check for localExpirationTime overflow (CASSANDRA-14092)
+        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
+        if (ttl + nowInSecs < 0)
+        {
+            switch (policy)
+            {
+                case CAP:
+                    /**
+                     * Capping at this stage is basically not rejecting the request. The actual capping is done
+                     * by {@link org.apache.cassandra.db.BufferExpiringCell#computeLocalExpirationTime(int)},
+                     * which converts the negative TTL to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
+                     */
+                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES,
+                                     TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
+                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
+                    return;
+
+                default: //REJECT
+                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
+                                                                    isDefaultTTL? "default " : "", ttl));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 17edd6d..27dd57f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -22,6 +22,7 @@ import java.util.*;
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.CFMetaData.SpeculativeRetry;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -127,6 +128,12 @@ public class CFPropDefs extends PropertyDefinitions
         }
 
         validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
+        Integer defaultTimeToLive = getInt(KW_DEFAULT_TIME_TO_LIVE, 0);
+        if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+            throw new ConfigurationException(String.format("%s must be less than or equal to %d (got %s)",
+                                                           KW_DEFAULT_TIME_TO_LIVE,
+                                                           ExpiringCell.MAX_TTL,
+                                                           defaultTimeToLive));
 
         Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null);
         Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f84188a..8038c6c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -124,7 +124,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options);
+        return attrs.getTimeToLive(options, cfm);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
index e01d860..1b2c384 100644
--- a/src/java/org/apache/cassandra/db/AbstractNativeCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -575,6 +575,12 @@ public abstract class AbstractNativeCell extends AbstractCell implements CellNam
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     protected long internalSize()
     {
         return MemoryUtil.getInt(peer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCell.java b/src/java/org/apache/cassandra/db/BufferCell.java
index a7d632d..ee5fe41 100644
--- a/src/java/org/apache/cassandra/db/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCell.java
@@ -69,6 +69,12 @@ public class BufferCell extends AbstractCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public CellName name() {
         return name;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDeletedCell.java b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
index a38f322..3762e1f 100644
--- a/src/java/org/apache/cassandra/db/BufferDeletedCell.java
+++ b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
@@ -54,6 +54,12 @@ public class BufferDeletedCell extends BufferCell implements DeletedCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public boolean isLive()
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
index 25172c8..ea40676 100644
--- a/src/java/org/apache/cassandra/db/BufferExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -31,19 +31,23 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 public class BufferExpiringCell extends BufferCell implements ExpiringCell
 {
+    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
+
     private final int localExpirationTime;
     private final int timeToLive;
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
-        this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+        super(name, value, timestamp);
+        assert timeToLive > 0 : timeToLive;
+        this.timeToLive = timeToLive;
+        this.localExpirationTime = computeLocalExpirationTime(timeToLive);
     }
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
     {
         super(name, value, timestamp);
         assert timeToLive > 0 : timeToLive;
-        assert localExpirationTime > 0 : localExpirationTime;
         this.timeToLive = timeToLive;
         this.localExpirationTime = localExpirationTime;
     }
@@ -66,6 +70,12 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        return new BufferExpiringCell(name(), value(), newTimestamp, timeToLive, newLocalDeletionTime);
+    }
+
+    @Override
     public int cellDataSize()
     {
         return super.cellDataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
@@ -176,7 +186,9 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     /** @return Either a DeletedCell, or an ExpiringCell. */
     public static Cell create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
     {
-        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
+        // CASSANDRA-14092 may have written rows with negative localExpirationTime, so we don't turn them into tombstones yet
+        // to be able to recover them with scrub.
+        if (localExpirationTime < 0 || localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
             return new BufferExpiringCell(name, value, timestamp, timeToLive, localExpirationTime);
         // The column is now expired, we can safely return a simple tombstone. Note that
         // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
@@ -184,4 +196,20 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
         // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
         return new BufferDeletedCell(name, localExpirationTime - timeToLive, timestamp);
     }
+
+    /**
+     * This method computes the {@link #localExpirationTime}, maybe capping to the maximum representable value
+     * which is {@link #MAX_DELETION_TIME}.
+     *
+     * Please note that the {@link org.apache.cassandra.cql3.Attributes.ExpirationDateOverflowPolicy} is applied
+     * during {@link org.apache.cassandra.cql3.Attributes#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
+     * so if the request was not denied it means it's expiration date should be capped.
+     *
+     * See CASSANDRA-14092
+     */
+    private int computeLocalExpirationTime(int timeToLive)
+    {
+        int localExpirationTime =  (int) (System.currentTimeMillis() / 1000) + timeToLive;
+        return localExpirationTime >= 0? localExpirationTime : MAX_DELETION_TIME;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Cell.java b/src/java/org/apache/cassandra/db/Cell.java
index 7c3926a..274f369 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -38,6 +38,8 @@ public interface Cell extends OnDiskAtom
 
     public Cell withUpdatedTimestamp(long newTimestamp);
 
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
+
     @Override
     public CellName name();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6e82745..2989b9d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1516,12 +1516,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
     }
 
     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index 99cfe35..c10a15f 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -60,6 +60,7 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
     @VisibleForTesting
     public DeletionTime(long markedForDeleteAt, int localDeletionTime)
     {
+        assert localDeletionTime >= 0 : localDeletionTime;
         this.markedForDeleteAt = markedForDeleteAt;
         this.localDeletionTime = localDeletionTime;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 87819ba..6e3634a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -358,8 +358,15 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
+    @Deprecated
     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
+        return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+    }
+
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
+                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
+    {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -372,7 +379,7 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void execute(SSTableReader input) throws IOException
             {
-                scrubOne(cfs, input, skipCorrupted, checkData);
+                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
             }
         }, jobs);
     }
@@ -710,9 +717,9 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 
         CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
         metrics.beginCompaction(scrubInfo);
@@ -1352,7 +1359,7 @@ public class CompactionManager implements CompactionManagerMBean
         public void afterExecute(Runnable r, Throwable t)
         {
             DebuggableThreadPoolExecutor.maybeResetTraceSessionWrapper(r);
-    
+
             if (t == null)
                 t = DebuggableThreadPoolExecutor.extractThrowable(r);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 2df3665..6d4537c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
@@ -35,6 +36,7 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class Scrubber implements Closeable
 {
@@ -43,6 +45,7 @@ public class Scrubber implements Closeable
     private final File destination;
     private final boolean skipCorrupted;
     public final boolean validateColumns;
+    private final boolean reinsertOverflowedTTLRows;
 
     private final CompactionController controller;
     private final boolean isCommutative;
@@ -67,6 +70,7 @@ public class Scrubber implements Closeable
     long nextRowPositionFromIndex;
 
     private final OutputHandler outputHandler;
+    private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
 
     private static final Comparator<Row> rowComparator = new Comparator<Row>()
     {
@@ -79,10 +83,17 @@ public class Scrubber implements Closeable
 
     public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
+    {
+        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
+    }
+
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
     {
         this.cfs = cfs;
         this.sstable = sstable;
@@ -90,6 +101,7 @@ public class Scrubber implements Closeable
         this.skipCorrupted = skipCorrupted;
         this.isOffline = isOffline;
         this.validateColumns = checkData;
+        this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 
         List<SSTableReader> toScrub = Collections.singletonList(sstable);
 
@@ -131,6 +143,9 @@ public class Scrubber implements Closeable
 
         this.currentRowPositionFromIndex = 0;
         this.nextRowPositionFromIndex = 0;
+
+        if (reinsertOverflowedTTLRows)
+            outputHandler.output("Starting scrub with reinsert overflowed TTL option");
     }
 
     public void scrub()
@@ -311,6 +326,8 @@ public class Scrubber implements Closeable
         else
         {
             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+            if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
             if (badRows > 0)
                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
         }
@@ -322,7 +339,7 @@ public class Scrubber implements Closeable
         // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
         // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
         // to the outOfOrderRows that will be later written to a new SSTable.
-        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
+        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
                                                               cfs.metadata.comparator.onDiskAtomComparator());
         if (prevKey != null && prevKey.compareTo(key) > 0)
         {
@@ -342,6 +359,18 @@ public class Scrubber implements Closeable
         return true;
     }
 
+    /**
+     * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+     * is specified
+     */
+    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
+    {
+        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
+        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                    outputHandler,
+                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+    }
+
     private void updateIndexKey()
     {
         currentIndexKey = nextIndexKey;
@@ -516,6 +545,11 @@ public class Scrubber implements Closeable
         }
     }
 
+    public class NegativeLocalDeletionInfoMetrics
+    {
+        public volatile int fixedRows = 0;
+    }
+
     /**
      * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
      * cells order and collect the out of order cells to correct the problem.
@@ -601,4 +635,61 @@ public class Scrubber implements Closeable
             return cf;
         }
     }
+
+    /**
+     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
+     *
+     * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+     */
+    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+    {
+        /**
+         * The decorated iterator.
+         */
+        private final OnDiskAtomIterator iterator;
+
+        private final OutputHandler outputHandler;
+        private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+
+        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
+                                                    NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+        {
+            this.iterator = iterator;
+            this.outputHandler = outputHandler;
+            this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+        }
+
+        public ColumnFamily getColumnFamily()
+        {
+            return iterator.getColumnFamily();
+        }
+
+        public DecoratedKey getKey()
+        {
+            return iterator.getKey();
+        }
+
+        public void close() throws IOException
+        {
+            iterator.close();
+        }
+
+        @Override
+        protected OnDiskAtom computeNext()
+        {
+            if (!iterator.hasNext())
+                return endOfData();
+
+            OnDiskAtom next = iterator.next();
+
+            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
+            {
+                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
+                negativeLocalExpirationTimeMetrics.fixedRows++;
+                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
+            }
+
+            return next;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 557c3de..d718765 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -28,9 +28,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CounterId;
@@ -155,7 +157,20 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS) throws IOException
     {
-        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        int localExpirationTime = (int) (expirationTimestampMS / 1000);
+        try
+        {
+            // This will throw exception if policy is REJECT and now() + ttl is higher than MAX_DELETION_TIME
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+            // If exception was not thrown, this means the policy was CAP, so we check for overflow and cap if that's the case
+            if (localExpirationTime < 0)
+                localExpirationTime = BufferExpiringCell.MAX_DELETION_TIME;
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e);
+        }
+        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, localExpirationTime));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 27939f9..a7a8ca7 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2415,10 +2415,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d3a1725..90c0fb5 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -219,16 +219,16 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
-     * 
+     *
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
      * @param columnFamilyList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
-    
-    
-    
+
+
+
     /**
      * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
@@ -274,8 +274,11 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
@@ -381,11 +384,11 @@ public interface StorageServiceMBean extends NotificationEmitter
      * If level cannot be parsed, then the level will be defaulted to DEBUG<br>
      * <br>
      * The logback configuration should have < jmxConfigurator /> set
-     * 
+     *
      * @param classQualifier The logger's classQualifer
      * @param level The log level
-     * @throws Exception 
-     * 
+     * @throws Exception
+     *
      *  @see ch.qos.logback.classic.Level#toLevel(String)
      */
     public void setLoggingLevel(String classQualifier, String level) throws Exception;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d5d9f73..10e7185 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -24,6 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -314,7 +315,7 @@ public class ThriftValidation
             if (isCommutative)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 
-            validateTtl(cosc.column);
+            validateTtl(metadata, cosc.column);
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
             validateColumnData(metadata, key, null, cosc.column);
         }
@@ -349,7 +350,7 @@ public class ThriftValidation
         }
     }
 
-    private static void validateTtl(Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
+    private static void validateTtl(CFMetaData metadata, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
         if (column.isSetTtl())
         {
@@ -358,9 +359,11 @@ public class ThriftValidation
 
             if (column.ttl > ExpiringCell.MAX_TTL)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
         }
         else
         {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
             // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
             assert column.ttl == 0;
         }
@@ -434,7 +437,7 @@ public class ThriftValidation
      */
     public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        validateTtl(column);
+        validateTtl(metadata, column);
         if (!column.isSetValue())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
         if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 13c7acf..fcd4110 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -243,9 +243,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
     }
 
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
@@ -268,10 +268,10 @@ public class NodeProbe implements AutoCloseable
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         checkJobs(out, jobs);
-        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
@@ -562,7 +562,7 @@ public class NodeProbe implements AutoCloseable
 
     /**
      * Take a snapshot of all column family from different keyspaces.
-     * 
+     *
      * @param snapshotName
      *            the name of the snapshot.
      * @param columnfamilylist
@@ -1302,7 +1302,7 @@ public class NodeProbe implements AutoCloseable
         }
         catch (Exception e)
         {
-          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e); 
+          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d1afb6f..54d7fb7 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -551,20 +551,20 @@ public class NodeTool
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } 
+            }
             catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
                 errors.append("Note: " + ex.getMessage() + "%n");
                 showEffectiveOwnership = false;
-            } 
+            }
             catch (IllegalArgumentException ex)
             {
                 System.out.printf("%nError: " + ex.getMessage() + "%n");
                 return;
             }
 
-            
+
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
                 printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
@@ -1282,6 +1282,11 @@ public class NodeTool
                 description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
         private int jobs = 2;
 
+        @Option(title = "reinsert_overflowed_ttl",
+        name = {"r", "--reinsert-overflowed-ttl"},
+        description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+        private boolean reinsertOverflowedTTL = false;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1292,7 +1297,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, reinsertOverflowedTTL   , !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -2197,7 +2202,7 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
-            
+
             StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships = null;
@@ -2249,9 +2254,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-            
+
             System.out.printf("%n" + errors.toString());
-            
+
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -2726,7 +2731,7 @@ public class NodeTool
                 probe.truncateHints(endpoint);
         }
     }
-    
+
     @Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
     public static class SetLoggingLevel extends NodeToolCmd
     {
@@ -2741,7 +2746,7 @@ public class NodeTool
             probe.setLoggingLevel(classQualifier, level);
         }
     }
-    
+
     @Command(name = "getlogginglevels", description = "Get the runtime logging levels")
     public static class GetLoggingLevels extends NodeToolCmd
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index fdf6c8d..59d13d5 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -47,6 +47,12 @@ import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 public class StandaloneScrubber
 {
+    public static final String REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION = "Rewrites rows with overflowed expiration date affected by CASSANDRA-14092 with " +
+                                                                            "the maximum supported expiration date of 2038-01-19T03:14:06+00:00. " +
+                                                                            "The rows are rewritten with the original timestamp incremented by one millisecond " +
+                                                                            "to override/supersede any potential tombstone that may have been generated " +
+                                                                            "during compaction of the affected rows.";
+
     private static final String TOOL_NAME = "sstablescrub";
     private static final String VERBOSE_OPTION  = "verbose";
     private static final String DEBUG_OPTION  = "debug";
@@ -54,6 +60,7 @@ public class StandaloneScrubber
     private static final String MANIFEST_CHECK_OPTION  = "manifest-check";
     private static final String SKIP_CORRUPTED_OPTION = "skip-corrupted";
     private static final String NO_VALIDATE_OPTION = "no-validate";
+    private static final String REINSERT_OVERFLOWED_TTL_OPTION = "reinsert-overflowed-ttl";
 
     public static void main(String args[])
     {
@@ -110,7 +117,7 @@ public class StandaloneScrubber
                 {
                     try
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
                         try
                         {
                             scrubber.scrub();
@@ -192,6 +199,7 @@ public class StandaloneScrubber
         public boolean manifestCheckOnly;
         public boolean skipCorrupted;
         public boolean noValidate;
+        public boolean reinsertOverflowedTTL;
 
         private Options(String keyspaceName, String cfName)
         {
@@ -232,6 +240,7 @@ public class StandaloneScrubber
                 opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                 opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                 opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
+                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
 
                 return opts;
             }
@@ -258,6 +267,7 @@ public class StandaloneScrubber
             options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
             options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
             options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
+            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
             return options;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d7cc13b
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
new file mode 100644
index 0000000..0e3da66
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
new file mode 100644
index 0000000..8a6dcba
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
@@ -0,0 +1 @@
+4012184764
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
new file mode 100644
index 0000000..3ab96ee
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
new file mode 100644
index 0000000..9bde77e
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..38373b4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
new file mode 100644
index 0000000..bdd4549
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
new file mode 100644
index 0000000..f58914a
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
@@ -0,0 +1 @@
+3463582096
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
new file mode 100644
index 0000000..38a6e4c
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
new file mode 100644
index 0000000..8ee9116
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..04a7384
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
new file mode 100644
index 0000000..1fc8ba4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
new file mode 100644
index 0000000..cd091ad
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1524836732
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
new file mode 100644
index 0000000..5fb34e8
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
new file mode 100644
index 0000000..4d961fb
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..c814fef
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
new file mode 100644
index 0000000..92032a7
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
new file mode 100644
index 0000000..a45d821
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2189764235
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
new file mode 100644
index 0000000..8291383
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
new file mode 100644
index 0000000..68f76ae
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db


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


[10/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
Protect against overflow of local expiration time

Patch by Paulo Motta; Reviewed by Sam Tunnicliffe for CASSANDRA-14092


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

Branch: refs/heads/cassandra-3.11
Commit: b2949439ec62077128103540e42570238520f4ee
Parents: 5ba9e6d
Author: Paulo Motta <pa...@gmail.com>
Authored: Thu Feb 1 04:01:28 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:33:50 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  23 +-
 .../cassandra/cql/AbstractModification.java     |   4 +
 .../org/apache/cassandra/cql/Attributes.java    |  19 +
 .../apache/cassandra/cql/BatchStatement.java    |   4 +
 .../org/apache/cassandra/cql/CFPropDefs.java    |   7 +
 .../org/apache/cassandra/cql3/Attributes.java   |  81 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   4 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  15 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |  17 +-
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  12 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  23 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 ...test_keyspace-table1-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table1-ka-1-Data.db       | Bin 0 -> 103 bytes
 .../cql_test_keyspace-table1-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table1-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table1-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table1-ka-1-Statistics.db | Bin 0 -> 4454 bytes
 .../cql_test_keyspace-table1-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table1-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table2-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table2-ka-1-Data.db       | Bin 0 -> 96 bytes
 .../cql_test_keyspace-table2-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table2-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table2-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table2-ka-1-Statistics.db | Bin 0 -> 4466 bytes
 .../cql_test_keyspace-table2-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table2-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table3-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table3-ka-1-Data.db       | Bin 0 -> 182 bytes
 .../cql_test_keyspace-table3-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table3-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table3-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table3-ka-1-Statistics.db | Bin 0 -> 4470 bytes
 .../cql_test_keyspace-table3-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table3-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table4-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table4-ka-1-Data.db       | Bin 0 -> 181 bytes
 .../cql_test_keyspace-table4-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table4-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table4-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table4-ka-1-Statistics.db | Bin 0 -> 4482 bytes
 .../cql_test_keyspace-table4-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table4-ka-1-TOC.txt       |   8 +
 .../cql3/validation/operations/TTLTest.java     | 410 +++++++++++++++++++
 59 files changed, 898 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --git a/CASSANDRA-14092.txt b/CASSANDRA-14092.txt
new file mode 100644
index 0000000..5ac872c
--- /dev/null
+++ b/CASSANDRA-14092.txt
@@ -0,0 +1,81 @@
+CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+---------------------------------------------
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+after this date are not currently supported.
+
+# Expiration Date Overflow Policy
+
+We plan to lift this limitation in newer versions, but while the fix is not available,
+operators can decide which policy to apply when dealing with inserts with TTL exceeding
+the maximum supported expiration date:
+  -     REJECT: this is the default policy and will reject any requests with expiration
+                date timestamp after 2038-01-19T03:14:06+00:00.
+  -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                2038-01-19T03:14:06+00:00 and the client will receive a warning.
+  - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+
+These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
+startup option.
+
+# Potential data loss on earlier versions
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+INSERTS with TTL expiring after the maximum supported date, causing the expiration
+time field to overflow and the records to expire immediately. Expired records due
+to overflow will not be queryable and will be permanently removed after a compaction.
+
+2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+since an AssertionError is thrown during INSERT when the expiration time field overflows
+on these versions. When assertions are disabled then it is possible to INSERT entries
+with overflowed local expiration time and even the earlier versions are subject to data
+loss due to this bug.
+
+This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+where this issue is addressed as soon as possible.
+
+# Data Recovery
+
+SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+with overflowed expiration time that were backed up or did not go through compaction can
+be recovered by reinserting overflowed entries with a valid expiration time and a higher
+timestamp, since tombstones may have been generated with the original timestamp.
+
+To find out if an SSTable has an entry with overflowed expiration, inspect it with the
+sstable2json tool and look for a negative "local deletion time" field. SSTables in this
+condition should be backed up immediately, as they are subject to data loss during
+compaction.
+
+A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+rows with overflowed expiration time with the maximum expiration date of
+2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+for recovery of SSTables via scrub:
+
+- Offline scrub:
+   - Clone the data directory tree to another location, keeping only the folders and the
+     contents of the system tables.
+   - Clone the configuration directory to another location, setting the data_file_directories
+     property to the cloned data directory in the cloned cassandra.yaml.
+   - Copy the affected SSTables to the cloned data location of the affected table.
+   - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+   - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+         WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+         compaction, so the data with overflowed will be removed - make sure to back up
+         your SSTables before running scrub.
+   - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+   - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+
+- Online scrub:
+   - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+     as otherwise, the data may be removed permanently during compaction.
+   - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+   - run "nodetool refresh" to load the SSTables.
+   - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+   - Re-enable compactions after verifying that scrub recovered the missing entries.
+
+See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e17093d..9332354 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.20
+ * Protect against overflow of local expiration time (CASSANDRA-14092)
  * More PEP8 compliance for cqlsh (CASSANDRA-14021)
  * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 796a424..fb6b4ee 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -1,3 +1,23 @@
+PLEASE READ: MAXIMUM TTL EXPIRATION DATE NOTICE (CASSANDRA-14092)
+------------------------------------------------------------------
+(General upgrading instructions are available in the next section)
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that inserts with TTL thatl expire after
+this date are not currently supported. By default, INSERTS with TTL exceeding the
+maximum supported date are rejected, but it's possible to choose a different
+ expiration overflow policy. See CASSANDRA-14092.txt for more details.
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x) there was no protection against INSERTS
+with TTL expiring after the maximum supported date, causing the expiration time
+field to overflow and the records to expire immediately. Clusters in the 2.X and
+lower series are not subject to this when assertions are enabled. Backed up SSTables
+can be potentially recovered and recovery instructions can be found on the
+CASSANDRA-14092.txt file.
+
+If you use or plan to use very large TTLS (10 to 20 years), read CASSANDRA-14092.txt
+for more information.
+
 GENERAL UPGRADING ADVICE FOR ANY VERSION
 ========================================
 
@@ -18,8 +38,7 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
+   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 
 2.1.19
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/AbstractModification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
index 8da2611..e98764b 100644
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ b/src/java/org/apache/cassandra/cql/AbstractModification.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.cql;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.thrift.ThriftClientState;
+import org.w3c.dom.Attr;
 
 public abstract class AbstractModification
 {
@@ -89,6 +92,7 @@ public abstract class AbstractModification
 
     public int getTimeToLive()
     {
+        Attributes.maybeApplyExpirationDateOverflowPolicy(keyspace, columnFamily, timeToLive);
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
index faee3b8..c1c37ef 100644
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ b/src/java/org/apache/cassandra/cql/Attributes.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.cql;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
  * Class to contain attributes for statements
@@ -73,4 +76,20 @@ public class Attributes
         return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
     }
 
+    public static void maybeApplyExpirationDateOverflowPolicy(String keyspace, String columnFamily, Integer timeToLive)
+    {
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
+        if (metadata != null)
+        {
+            try
+            {
+                org.apache.cassandra.cql3.Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, timeToLive, false);
+            }
+            catch (InvalidRequestException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
index b141bcc..e5a95b8 100644
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql/BatchStatement.java
@@ -72,6 +72,10 @@ public class BatchStatement
 
     public int getTimeToLive()
     {
+        for (AbstractModification statement : statements)
+        {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(statement.keyspace, statement.columnFamily, timeToLive);
+        }
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
index f65cb94..a0c8d0d 100644
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql/CFPropDefs.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -191,6 +192,12 @@ public class CFPropDefs {
                         KW_DEFAULT_TIME_TO_LIVE,
                         0,
                         CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
+
+            if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+                throw new InvalidRequestException(String.format("%s must be less than or equal to %d (got %s)",
+                                                                KW_DEFAULT_TIME_TO_LIVE,
+                                                                ExpiringCell.MAX_TTL,
+                                                                defaultTimeToLive));
         }
 
         CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index 435757b..23571ca 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.NoSpamLogger;
 
 /**
  * Utility class for the Parser to gather attributes for modification
@@ -32,6 +38,41 @@ import org.apache.cassandra.serializers.MarshalException;
  */
 public class Attributes
 {
+    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
+
+    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
+
+    public enum ExpirationDateOverflowPolicy
+    {
+        REJECT, CAP
+    }
+
+    @VisibleForTesting
+    public static ExpirationDateOverflowPolicy policy;
+
+    static {
+        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
+        try
+        {
+            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
+        }
+        catch (RuntimeException e)
+        {
+            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
+            policy = ExpirationDateOverflowPolicy.REJECT;
+        }
+    }
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
+                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
+                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
+                                                                          "is fixed. See CASSANDRA-14092 for more details.";
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
+                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
+                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
+                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
+
     private final Term timestamp;
     private final Term timeToLive;
 
@@ -77,10 +118,13 @@ public class Attributes
         return LongType.instance.compose(tval);
     }
 
-    public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+    public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
     {
         if (timeToLive == null)
-            return 0;
+        {
+            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
+            return metadata.getDefaultTimeToLive();
+        }
 
         ByteBuffer tval = timeToLive.bindAndGet(options);
         if (tval == null)
@@ -102,6 +146,8 @@ public class Attributes
         if (ttl > ExpiringCell.MAX_TTL)
             throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 
+        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+
         return ttl;
     }
 
@@ -135,4 +181,33 @@ public class Attributes
             return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[ttl]", true), Int32Type.instance);
         }
     }
+
+    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
+    {
+        if (ttl == 0)
+            return;
+
+        // Check for localExpirationTime overflow (CASSANDRA-14092)
+        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
+        if (ttl + nowInSecs < 0)
+        {
+            switch (policy)
+            {
+                case CAP:
+                    /**
+                     * Capping at this stage is basically not rejecting the request. The actual capping is done
+                     * by {@link org.apache.cassandra.db.BufferExpiringCell#computeLocalExpirationTime(int)},
+                     * which converts the negative TTL to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
+                     */
+                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES,
+                                     TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
+                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
+                    return;
+
+                default: //REJECT
+                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
+                                                                    isDefaultTTL? "default " : "", ttl));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 17edd6d..27dd57f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -22,6 +22,7 @@ import java.util.*;
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.CFMetaData.SpeculativeRetry;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -127,6 +128,12 @@ public class CFPropDefs extends PropertyDefinitions
         }
 
         validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
+        Integer defaultTimeToLive = getInt(KW_DEFAULT_TIME_TO_LIVE, 0);
+        if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+            throw new ConfigurationException(String.format("%s must be less than or equal to %d (got %s)",
+                                                           KW_DEFAULT_TIME_TO_LIVE,
+                                                           ExpiringCell.MAX_TTL,
+                                                           defaultTimeToLive));
 
         Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null);
         Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f84188a..8038c6c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -124,7 +124,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options);
+        return attrs.getTimeToLive(options, cfm);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
index e01d860..1b2c384 100644
--- a/src/java/org/apache/cassandra/db/AbstractNativeCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -575,6 +575,12 @@ public abstract class AbstractNativeCell extends AbstractCell implements CellNam
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     protected long internalSize()
     {
         return MemoryUtil.getInt(peer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCell.java b/src/java/org/apache/cassandra/db/BufferCell.java
index a7d632d..ee5fe41 100644
--- a/src/java/org/apache/cassandra/db/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCell.java
@@ -69,6 +69,12 @@ public class BufferCell extends AbstractCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public CellName name() {
         return name;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDeletedCell.java b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
index a38f322..3762e1f 100644
--- a/src/java/org/apache/cassandra/db/BufferDeletedCell.java
+++ b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
@@ -54,6 +54,12 @@ public class BufferDeletedCell extends BufferCell implements DeletedCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public boolean isLive()
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
index 25172c8..ea40676 100644
--- a/src/java/org/apache/cassandra/db/BufferExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -31,19 +31,23 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 public class BufferExpiringCell extends BufferCell implements ExpiringCell
 {
+    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
+
     private final int localExpirationTime;
     private final int timeToLive;
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
-        this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+        super(name, value, timestamp);
+        assert timeToLive > 0 : timeToLive;
+        this.timeToLive = timeToLive;
+        this.localExpirationTime = computeLocalExpirationTime(timeToLive);
     }
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
     {
         super(name, value, timestamp);
         assert timeToLive > 0 : timeToLive;
-        assert localExpirationTime > 0 : localExpirationTime;
         this.timeToLive = timeToLive;
         this.localExpirationTime = localExpirationTime;
     }
@@ -66,6 +70,12 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        return new BufferExpiringCell(name(), value(), newTimestamp, timeToLive, newLocalDeletionTime);
+    }
+
+    @Override
     public int cellDataSize()
     {
         return super.cellDataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
@@ -176,7 +186,9 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     /** @return Either a DeletedCell, or an ExpiringCell. */
     public static Cell create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
     {
-        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
+        // CASSANDRA-14092 may have written rows with negative localExpirationTime, so we don't turn them into tombstones yet
+        // to be able to recover them with scrub.
+        if (localExpirationTime < 0 || localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
             return new BufferExpiringCell(name, value, timestamp, timeToLive, localExpirationTime);
         // The column is now expired, we can safely return a simple tombstone. Note that
         // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
@@ -184,4 +196,20 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
         // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
         return new BufferDeletedCell(name, localExpirationTime - timeToLive, timestamp);
     }
+
+    /**
+     * This method computes the {@link #localExpirationTime}, maybe capping to the maximum representable value
+     * which is {@link #MAX_DELETION_TIME}.
+     *
+     * Please note that the {@link org.apache.cassandra.cql3.Attributes.ExpirationDateOverflowPolicy} is applied
+     * during {@link org.apache.cassandra.cql3.Attributes#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
+     * so if the request was not denied it means it's expiration date should be capped.
+     *
+     * See CASSANDRA-14092
+     */
+    private int computeLocalExpirationTime(int timeToLive)
+    {
+        int localExpirationTime =  (int) (System.currentTimeMillis() / 1000) + timeToLive;
+        return localExpirationTime >= 0? localExpirationTime : MAX_DELETION_TIME;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Cell.java b/src/java/org/apache/cassandra/db/Cell.java
index 7c3926a..274f369 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -38,6 +38,8 @@ public interface Cell extends OnDiskAtom
 
     public Cell withUpdatedTimestamp(long newTimestamp);
 
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
+
     @Override
     public CellName name();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6e82745..2989b9d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1516,12 +1516,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
     }
 
     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index 99cfe35..c10a15f 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -60,6 +60,7 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
     @VisibleForTesting
     public DeletionTime(long markedForDeleteAt, int localDeletionTime)
     {
+        assert localDeletionTime >= 0 : localDeletionTime;
         this.markedForDeleteAt = markedForDeleteAt;
         this.localDeletionTime = localDeletionTime;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 87819ba..6e3634a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -358,8 +358,15 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
+    @Deprecated
     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
+        return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+    }
+
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
+                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
+    {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -372,7 +379,7 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void execute(SSTableReader input) throws IOException
             {
-                scrubOne(cfs, input, skipCorrupted, checkData);
+                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
             }
         }, jobs);
     }
@@ -710,9 +717,9 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 
         CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
         metrics.beginCompaction(scrubInfo);
@@ -1352,7 +1359,7 @@ public class CompactionManager implements CompactionManagerMBean
         public void afterExecute(Runnable r, Throwable t)
         {
             DebuggableThreadPoolExecutor.maybeResetTraceSessionWrapper(r);
-    
+
             if (t == null)
                 t = DebuggableThreadPoolExecutor.extractThrowable(r);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 2df3665..6d4537c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
@@ -35,6 +36,7 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class Scrubber implements Closeable
 {
@@ -43,6 +45,7 @@ public class Scrubber implements Closeable
     private final File destination;
     private final boolean skipCorrupted;
     public final boolean validateColumns;
+    private final boolean reinsertOverflowedTTLRows;
 
     private final CompactionController controller;
     private final boolean isCommutative;
@@ -67,6 +70,7 @@ public class Scrubber implements Closeable
     long nextRowPositionFromIndex;
 
     private final OutputHandler outputHandler;
+    private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
 
     private static final Comparator<Row> rowComparator = new Comparator<Row>()
     {
@@ -79,10 +83,17 @@ public class Scrubber implements Closeable
 
     public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
+    {
+        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
+    }
+
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
     {
         this.cfs = cfs;
         this.sstable = sstable;
@@ -90,6 +101,7 @@ public class Scrubber implements Closeable
         this.skipCorrupted = skipCorrupted;
         this.isOffline = isOffline;
         this.validateColumns = checkData;
+        this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 
         List<SSTableReader> toScrub = Collections.singletonList(sstable);
 
@@ -131,6 +143,9 @@ public class Scrubber implements Closeable
 
         this.currentRowPositionFromIndex = 0;
         this.nextRowPositionFromIndex = 0;
+
+        if (reinsertOverflowedTTLRows)
+            outputHandler.output("Starting scrub with reinsert overflowed TTL option");
     }
 
     public void scrub()
@@ -311,6 +326,8 @@ public class Scrubber implements Closeable
         else
         {
             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+            if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
             if (badRows > 0)
                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
         }
@@ -322,7 +339,7 @@ public class Scrubber implements Closeable
         // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
         // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
         // to the outOfOrderRows that will be later written to a new SSTable.
-        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
+        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
                                                               cfs.metadata.comparator.onDiskAtomComparator());
         if (prevKey != null && prevKey.compareTo(key) > 0)
         {
@@ -342,6 +359,18 @@ public class Scrubber implements Closeable
         return true;
     }
 
+    /**
+     * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+     * is specified
+     */
+    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
+    {
+        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
+        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                    outputHandler,
+                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+    }
+
     private void updateIndexKey()
     {
         currentIndexKey = nextIndexKey;
@@ -516,6 +545,11 @@ public class Scrubber implements Closeable
         }
     }
 
+    public class NegativeLocalDeletionInfoMetrics
+    {
+        public volatile int fixedRows = 0;
+    }
+
     /**
      * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
      * cells order and collect the out of order cells to correct the problem.
@@ -601,4 +635,61 @@ public class Scrubber implements Closeable
             return cf;
         }
     }
+
+    /**
+     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
+     *
+     * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+     */
+    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+    {
+        /**
+         * The decorated iterator.
+         */
+        private final OnDiskAtomIterator iterator;
+
+        private final OutputHandler outputHandler;
+        private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+
+        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
+                                                    NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+        {
+            this.iterator = iterator;
+            this.outputHandler = outputHandler;
+            this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+        }
+
+        public ColumnFamily getColumnFamily()
+        {
+            return iterator.getColumnFamily();
+        }
+
+        public DecoratedKey getKey()
+        {
+            return iterator.getKey();
+        }
+
+        public void close() throws IOException
+        {
+            iterator.close();
+        }
+
+        @Override
+        protected OnDiskAtom computeNext()
+        {
+            if (!iterator.hasNext())
+                return endOfData();
+
+            OnDiskAtom next = iterator.next();
+
+            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
+            {
+                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
+                negativeLocalExpirationTimeMetrics.fixedRows++;
+                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
+            }
+
+            return next;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 557c3de..d718765 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -28,9 +28,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CounterId;
@@ -155,7 +157,20 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS) throws IOException
     {
-        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        int localExpirationTime = (int) (expirationTimestampMS / 1000);
+        try
+        {
+            // This will throw exception if policy is REJECT and now() + ttl is higher than MAX_DELETION_TIME
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+            // If exception was not thrown, this means the policy was CAP, so we check for overflow and cap if that's the case
+            if (localExpirationTime < 0)
+                localExpirationTime = BufferExpiringCell.MAX_DELETION_TIME;
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e);
+        }
+        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, localExpirationTime));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 27939f9..a7a8ca7 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2415,10 +2415,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d3a1725..90c0fb5 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -219,16 +219,16 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
-     * 
+     *
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
      * @param columnFamilyList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
-    
-    
-    
+
+
+
     /**
      * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
@@ -274,8 +274,11 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
@@ -381,11 +384,11 @@ public interface StorageServiceMBean extends NotificationEmitter
      * If level cannot be parsed, then the level will be defaulted to DEBUG<br>
      * <br>
      * The logback configuration should have < jmxConfigurator /> set
-     * 
+     *
      * @param classQualifier The logger's classQualifer
      * @param level The log level
-     * @throws Exception 
-     * 
+     * @throws Exception
+     *
      *  @see ch.qos.logback.classic.Level#toLevel(String)
      */
     public void setLoggingLevel(String classQualifier, String level) throws Exception;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d5d9f73..10e7185 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -24,6 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -314,7 +315,7 @@ public class ThriftValidation
             if (isCommutative)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 
-            validateTtl(cosc.column);
+            validateTtl(metadata, cosc.column);
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
             validateColumnData(metadata, key, null, cosc.column);
         }
@@ -349,7 +350,7 @@ public class ThriftValidation
         }
     }
 
-    private static void validateTtl(Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
+    private static void validateTtl(CFMetaData metadata, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
         if (column.isSetTtl())
         {
@@ -358,9 +359,11 @@ public class ThriftValidation
 
             if (column.ttl > ExpiringCell.MAX_TTL)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
         }
         else
         {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
             // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
             assert column.ttl == 0;
         }
@@ -434,7 +437,7 @@ public class ThriftValidation
      */
     public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        validateTtl(column);
+        validateTtl(metadata, column);
         if (!column.isSetValue())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
         if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 13c7acf..fcd4110 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -243,9 +243,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
     }
 
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
@@ -268,10 +268,10 @@ public class NodeProbe implements AutoCloseable
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         checkJobs(out, jobs);
-        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
@@ -562,7 +562,7 @@ public class NodeProbe implements AutoCloseable
 
     /**
      * Take a snapshot of all column family from different keyspaces.
-     * 
+     *
      * @param snapshotName
      *            the name of the snapshot.
      * @param columnfamilylist
@@ -1302,7 +1302,7 @@ public class NodeProbe implements AutoCloseable
         }
         catch (Exception e)
         {
-          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e); 
+          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d1afb6f..54d7fb7 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -551,20 +551,20 @@ public class NodeTool
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } 
+            }
             catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
                 errors.append("Note: " + ex.getMessage() + "%n");
                 showEffectiveOwnership = false;
-            } 
+            }
             catch (IllegalArgumentException ex)
             {
                 System.out.printf("%nError: " + ex.getMessage() + "%n");
                 return;
             }
 
-            
+
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
                 printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
@@ -1282,6 +1282,11 @@ public class NodeTool
                 description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
         private int jobs = 2;
 
+        @Option(title = "reinsert_overflowed_ttl",
+        name = {"r", "--reinsert-overflowed-ttl"},
+        description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+        private boolean reinsertOverflowedTTL = false;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1292,7 +1297,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, reinsertOverflowedTTL   , !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -2197,7 +2202,7 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
-            
+
             StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships = null;
@@ -2249,9 +2254,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-            
+
             System.out.printf("%n" + errors.toString());
-            
+
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -2726,7 +2731,7 @@ public class NodeTool
                 probe.truncateHints(endpoint);
         }
     }
-    
+
     @Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
     public static class SetLoggingLevel extends NodeToolCmd
     {
@@ -2741,7 +2746,7 @@ public class NodeTool
             probe.setLoggingLevel(classQualifier, level);
         }
     }
-    
+
     @Command(name = "getlogginglevels", description = "Get the runtime logging levels")
     public static class GetLoggingLevels extends NodeToolCmd
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index fdf6c8d..59d13d5 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -47,6 +47,12 @@ import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 public class StandaloneScrubber
 {
+    public static final String REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION = "Rewrites rows with overflowed expiration date affected by CASSANDRA-14092 with " +
+                                                                            "the maximum supported expiration date of 2038-01-19T03:14:06+00:00. " +
+                                                                            "The rows are rewritten with the original timestamp incremented by one millisecond " +
+                                                                            "to override/supersede any potential tombstone that may have been generated " +
+                                                                            "during compaction of the affected rows.";
+
     private static final String TOOL_NAME = "sstablescrub";
     private static final String VERBOSE_OPTION  = "verbose";
     private static final String DEBUG_OPTION  = "debug";
@@ -54,6 +60,7 @@ public class StandaloneScrubber
     private static final String MANIFEST_CHECK_OPTION  = "manifest-check";
     private static final String SKIP_CORRUPTED_OPTION = "skip-corrupted";
     private static final String NO_VALIDATE_OPTION = "no-validate";
+    private static final String REINSERT_OVERFLOWED_TTL_OPTION = "reinsert-overflowed-ttl";
 
     public static void main(String args[])
     {
@@ -110,7 +117,7 @@ public class StandaloneScrubber
                 {
                     try
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
                         try
                         {
                             scrubber.scrub();
@@ -192,6 +199,7 @@ public class StandaloneScrubber
         public boolean manifestCheckOnly;
         public boolean skipCorrupted;
         public boolean noValidate;
+        public boolean reinsertOverflowedTTL;
 
         private Options(String keyspaceName, String cfName)
         {
@@ -232,6 +240,7 @@ public class StandaloneScrubber
                 opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                 opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                 opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
+                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
 
                 return opts;
             }
@@ -258,6 +267,7 @@ public class StandaloneScrubber
             options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
             options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
             options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
+            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
             return options;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d7cc13b
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
new file mode 100644
index 0000000..0e3da66
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
new file mode 100644
index 0000000..8a6dcba
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
@@ -0,0 +1 @@
+4012184764
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
new file mode 100644
index 0000000..3ab96ee
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
new file mode 100644
index 0000000..9bde77e
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..38373b4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
new file mode 100644
index 0000000..bdd4549
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
new file mode 100644
index 0000000..f58914a
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
@@ -0,0 +1 @@
+3463582096
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
new file mode 100644
index 0000000..38a6e4c
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
new file mode 100644
index 0000000..8ee9116
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..04a7384
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
new file mode 100644
index 0000000..1fc8ba4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
new file mode 100644
index 0000000..cd091ad
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1524836732
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
new file mode 100644
index 0000000..5fb34e8
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
new file mode 100644
index 0000000..4d961fb
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..c814fef
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
new file mode 100644
index 0000000..92032a7
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
new file mode 100644
index 0000000..a45d821
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2189764235
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
new file mode 100644
index 0000000..8291383
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
new file mode 100644
index 0000000..68f76ae
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db


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


[04/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
Protect against overflow of local expiration time

Patch by Paulo Motta; Reviewed by Sam Tunnicliffe for CASSANDRA-14092


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

Branch: refs/heads/cassandra-2.2
Commit: b2949439ec62077128103540e42570238520f4ee
Parents: 5ba9e6d
Author: Paulo Motta <pa...@gmail.com>
Authored: Thu Feb 1 04:01:28 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:33:50 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  23 +-
 .../cassandra/cql/AbstractModification.java     |   4 +
 .../org/apache/cassandra/cql/Attributes.java    |  19 +
 .../apache/cassandra/cql/BatchStatement.java    |   4 +
 .../org/apache/cassandra/cql/CFPropDefs.java    |   7 +
 .../org/apache/cassandra/cql3/Attributes.java   |  81 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   4 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  15 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |  17 +-
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  12 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  23 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 ...test_keyspace-table1-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table1-ka-1-Data.db       | Bin 0 -> 103 bytes
 .../cql_test_keyspace-table1-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table1-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table1-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table1-ka-1-Statistics.db | Bin 0 -> 4454 bytes
 .../cql_test_keyspace-table1-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table1-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table2-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table2-ka-1-Data.db       | Bin 0 -> 96 bytes
 .../cql_test_keyspace-table2-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table2-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table2-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table2-ka-1-Statistics.db | Bin 0 -> 4466 bytes
 .../cql_test_keyspace-table2-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table2-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table3-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table3-ka-1-Data.db       | Bin 0 -> 182 bytes
 .../cql_test_keyspace-table3-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table3-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table3-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table3-ka-1-Statistics.db | Bin 0 -> 4470 bytes
 .../cql_test_keyspace-table3-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table3-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table4-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table4-ka-1-Data.db       | Bin 0 -> 181 bytes
 .../cql_test_keyspace-table4-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table4-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table4-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table4-ka-1-Statistics.db | Bin 0 -> 4482 bytes
 .../cql_test_keyspace-table4-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table4-ka-1-TOC.txt       |   8 +
 .../cql3/validation/operations/TTLTest.java     | 410 +++++++++++++++++++
 59 files changed, 898 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --git a/CASSANDRA-14092.txt b/CASSANDRA-14092.txt
new file mode 100644
index 0000000..5ac872c
--- /dev/null
+++ b/CASSANDRA-14092.txt
@@ -0,0 +1,81 @@
+CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+---------------------------------------------
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+after this date are not currently supported.
+
+# Expiration Date Overflow Policy
+
+We plan to lift this limitation in newer versions, but while the fix is not available,
+operators can decide which policy to apply when dealing with inserts with TTL exceeding
+the maximum supported expiration date:
+  -     REJECT: this is the default policy and will reject any requests with expiration
+                date timestamp after 2038-01-19T03:14:06+00:00.
+  -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                2038-01-19T03:14:06+00:00 and the client will receive a warning.
+  - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+
+These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
+startup option.
+
+# Potential data loss on earlier versions
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+INSERTS with TTL expiring after the maximum supported date, causing the expiration
+time field to overflow and the records to expire immediately. Expired records due
+to overflow will not be queryable and will be permanently removed after a compaction.
+
+2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+since an AssertionError is thrown during INSERT when the expiration time field overflows
+on these versions. When assertions are disabled then it is possible to INSERT entries
+with overflowed local expiration time and even the earlier versions are subject to data
+loss due to this bug.
+
+This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+where this issue is addressed as soon as possible.
+
+# Data Recovery
+
+SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+with overflowed expiration time that were backed up or did not go through compaction can
+be recovered by reinserting overflowed entries with a valid expiration time and a higher
+timestamp, since tombstones may have been generated with the original timestamp.
+
+To find out if an SSTable has an entry with overflowed expiration, inspect it with the
+sstable2json tool and look for a negative "local deletion time" field. SSTables in this
+condition should be backed up immediately, as they are subject to data loss during
+compaction.
+
+A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+rows with overflowed expiration time with the maximum expiration date of
+2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+for recovery of SSTables via scrub:
+
+- Offline scrub:
+   - Clone the data directory tree to another location, keeping only the folders and the
+     contents of the system tables.
+   - Clone the configuration directory to another location, setting the data_file_directories
+     property to the cloned data directory in the cloned cassandra.yaml.
+   - Copy the affected SSTables to the cloned data location of the affected table.
+   - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+   - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+         WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+         compaction, so the data with overflowed will be removed - make sure to back up
+         your SSTables before running scrub.
+   - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+   - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+
+- Online scrub:
+   - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+     as otherwise, the data may be removed permanently during compaction.
+   - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+   - run "nodetool refresh" to load the SSTables.
+   - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+   - Re-enable compactions after verifying that scrub recovered the missing entries.
+
+See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e17093d..9332354 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.20
+ * Protect against overflow of local expiration time (CASSANDRA-14092)
  * More PEP8 compliance for cqlsh (CASSANDRA-14021)
  * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 796a424..fb6b4ee 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -1,3 +1,23 @@
+PLEASE READ: MAXIMUM TTL EXPIRATION DATE NOTICE (CASSANDRA-14092)
+------------------------------------------------------------------
+(General upgrading instructions are available in the next section)
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that inserts with TTL thatl expire after
+this date are not currently supported. By default, INSERTS with TTL exceeding the
+maximum supported date are rejected, but it's possible to choose a different
+ expiration overflow policy. See CASSANDRA-14092.txt for more details.
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x) there was no protection against INSERTS
+with TTL expiring after the maximum supported date, causing the expiration time
+field to overflow and the records to expire immediately. Clusters in the 2.X and
+lower series are not subject to this when assertions are enabled. Backed up SSTables
+can be potentially recovered and recovery instructions can be found on the
+CASSANDRA-14092.txt file.
+
+If you use or plan to use very large TTLS (10 to 20 years), read CASSANDRA-14092.txt
+for more information.
+
 GENERAL UPGRADING ADVICE FOR ANY VERSION
 ========================================
 
@@ -18,8 +38,7 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
+   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 
 2.1.19
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/AbstractModification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
index 8da2611..e98764b 100644
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ b/src/java/org/apache/cassandra/cql/AbstractModification.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.cql;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.thrift.ThriftClientState;
+import org.w3c.dom.Attr;
 
 public abstract class AbstractModification
 {
@@ -89,6 +92,7 @@ public abstract class AbstractModification
 
     public int getTimeToLive()
     {
+        Attributes.maybeApplyExpirationDateOverflowPolicy(keyspace, columnFamily, timeToLive);
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
index faee3b8..c1c37ef 100644
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ b/src/java/org/apache/cassandra/cql/Attributes.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.cql;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
  * Class to contain attributes for statements
@@ -73,4 +76,20 @@ public class Attributes
         return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
     }
 
+    public static void maybeApplyExpirationDateOverflowPolicy(String keyspace, String columnFamily, Integer timeToLive)
+    {
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
+        if (metadata != null)
+        {
+            try
+            {
+                org.apache.cassandra.cql3.Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, timeToLive, false);
+            }
+            catch (InvalidRequestException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
index b141bcc..e5a95b8 100644
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql/BatchStatement.java
@@ -72,6 +72,10 @@ public class BatchStatement
 
     public int getTimeToLive()
     {
+        for (AbstractModification statement : statements)
+        {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(statement.keyspace, statement.columnFamily, timeToLive);
+        }
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
index f65cb94..a0c8d0d 100644
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql/CFPropDefs.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -191,6 +192,12 @@ public class CFPropDefs {
                         KW_DEFAULT_TIME_TO_LIVE,
                         0,
                         CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
+
+            if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+                throw new InvalidRequestException(String.format("%s must be less than or equal to %d (got %s)",
+                                                                KW_DEFAULT_TIME_TO_LIVE,
+                                                                ExpiringCell.MAX_TTL,
+                                                                defaultTimeToLive));
         }
 
         CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index 435757b..23571ca 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.NoSpamLogger;
 
 /**
  * Utility class for the Parser to gather attributes for modification
@@ -32,6 +38,41 @@ import org.apache.cassandra.serializers.MarshalException;
  */
 public class Attributes
 {
+    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
+
+    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
+
+    public enum ExpirationDateOverflowPolicy
+    {
+        REJECT, CAP
+    }
+
+    @VisibleForTesting
+    public static ExpirationDateOverflowPolicy policy;
+
+    static {
+        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
+        try
+        {
+            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
+        }
+        catch (RuntimeException e)
+        {
+            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
+            policy = ExpirationDateOverflowPolicy.REJECT;
+        }
+    }
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
+                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
+                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
+                                                                          "is fixed. See CASSANDRA-14092 for more details.";
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
+                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
+                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
+                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
+
     private final Term timestamp;
     private final Term timeToLive;
 
@@ -77,10 +118,13 @@ public class Attributes
         return LongType.instance.compose(tval);
     }
 
-    public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+    public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
     {
         if (timeToLive == null)
-            return 0;
+        {
+            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
+            return metadata.getDefaultTimeToLive();
+        }
 
         ByteBuffer tval = timeToLive.bindAndGet(options);
         if (tval == null)
@@ -102,6 +146,8 @@ public class Attributes
         if (ttl > ExpiringCell.MAX_TTL)
             throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 
+        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+
         return ttl;
     }
 
@@ -135,4 +181,33 @@ public class Attributes
             return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[ttl]", true), Int32Type.instance);
         }
     }
+
+    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
+    {
+        if (ttl == 0)
+            return;
+
+        // Check for localExpirationTime overflow (CASSANDRA-14092)
+        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
+        if (ttl + nowInSecs < 0)
+        {
+            switch (policy)
+            {
+                case CAP:
+                    /**
+                     * Capping at this stage is basically not rejecting the request. The actual capping is done
+                     * by {@link org.apache.cassandra.db.BufferExpiringCell#computeLocalExpirationTime(int)},
+                     * which converts the negative TTL to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
+                     */
+                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES,
+                                     TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
+                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
+                    return;
+
+                default: //REJECT
+                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
+                                                                    isDefaultTTL? "default " : "", ttl));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 17edd6d..27dd57f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -22,6 +22,7 @@ import java.util.*;
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.CFMetaData.SpeculativeRetry;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -127,6 +128,12 @@ public class CFPropDefs extends PropertyDefinitions
         }
 
         validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
+        Integer defaultTimeToLive = getInt(KW_DEFAULT_TIME_TO_LIVE, 0);
+        if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+            throw new ConfigurationException(String.format("%s must be less than or equal to %d (got %s)",
+                                                           KW_DEFAULT_TIME_TO_LIVE,
+                                                           ExpiringCell.MAX_TTL,
+                                                           defaultTimeToLive));
 
         Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null);
         Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f84188a..8038c6c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -124,7 +124,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options);
+        return attrs.getTimeToLive(options, cfm);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
index e01d860..1b2c384 100644
--- a/src/java/org/apache/cassandra/db/AbstractNativeCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -575,6 +575,12 @@ public abstract class AbstractNativeCell extends AbstractCell implements CellNam
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     protected long internalSize()
     {
         return MemoryUtil.getInt(peer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCell.java b/src/java/org/apache/cassandra/db/BufferCell.java
index a7d632d..ee5fe41 100644
--- a/src/java/org/apache/cassandra/db/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCell.java
@@ -69,6 +69,12 @@ public class BufferCell extends AbstractCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public CellName name() {
         return name;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDeletedCell.java b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
index a38f322..3762e1f 100644
--- a/src/java/org/apache/cassandra/db/BufferDeletedCell.java
+++ b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
@@ -54,6 +54,12 @@ public class BufferDeletedCell extends BufferCell implements DeletedCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public boolean isLive()
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
index 25172c8..ea40676 100644
--- a/src/java/org/apache/cassandra/db/BufferExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -31,19 +31,23 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 public class BufferExpiringCell extends BufferCell implements ExpiringCell
 {
+    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
+
     private final int localExpirationTime;
     private final int timeToLive;
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
-        this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+        super(name, value, timestamp);
+        assert timeToLive > 0 : timeToLive;
+        this.timeToLive = timeToLive;
+        this.localExpirationTime = computeLocalExpirationTime(timeToLive);
     }
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
     {
         super(name, value, timestamp);
         assert timeToLive > 0 : timeToLive;
-        assert localExpirationTime > 0 : localExpirationTime;
         this.timeToLive = timeToLive;
         this.localExpirationTime = localExpirationTime;
     }
@@ -66,6 +70,12 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        return new BufferExpiringCell(name(), value(), newTimestamp, timeToLive, newLocalDeletionTime);
+    }
+
+    @Override
     public int cellDataSize()
     {
         return super.cellDataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
@@ -176,7 +186,9 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     /** @return Either a DeletedCell, or an ExpiringCell. */
     public static Cell create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
     {
-        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
+        // CASSANDRA-14092 may have written rows with negative localExpirationTime, so we don't turn them into tombstones yet
+        // to be able to recover them with scrub.
+        if (localExpirationTime < 0 || localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
             return new BufferExpiringCell(name, value, timestamp, timeToLive, localExpirationTime);
         // The column is now expired, we can safely return a simple tombstone. Note that
         // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
@@ -184,4 +196,20 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
         // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
         return new BufferDeletedCell(name, localExpirationTime - timeToLive, timestamp);
     }
+
+    /**
+     * This method computes the {@link #localExpirationTime}, maybe capping to the maximum representable value
+     * which is {@link #MAX_DELETION_TIME}.
+     *
+     * Please note that the {@link org.apache.cassandra.cql3.Attributes.ExpirationDateOverflowPolicy} is applied
+     * during {@link org.apache.cassandra.cql3.Attributes#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
+     * so if the request was not denied it means it's expiration date should be capped.
+     *
+     * See CASSANDRA-14092
+     */
+    private int computeLocalExpirationTime(int timeToLive)
+    {
+        int localExpirationTime =  (int) (System.currentTimeMillis() / 1000) + timeToLive;
+        return localExpirationTime >= 0? localExpirationTime : MAX_DELETION_TIME;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Cell.java b/src/java/org/apache/cassandra/db/Cell.java
index 7c3926a..274f369 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -38,6 +38,8 @@ public interface Cell extends OnDiskAtom
 
     public Cell withUpdatedTimestamp(long newTimestamp);
 
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
+
     @Override
     public CellName name();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6e82745..2989b9d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1516,12 +1516,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
     }
 
     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index 99cfe35..c10a15f 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -60,6 +60,7 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
     @VisibleForTesting
     public DeletionTime(long markedForDeleteAt, int localDeletionTime)
     {
+        assert localDeletionTime >= 0 : localDeletionTime;
         this.markedForDeleteAt = markedForDeleteAt;
         this.localDeletionTime = localDeletionTime;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 87819ba..6e3634a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -358,8 +358,15 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
+    @Deprecated
     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
+        return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+    }
+
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
+                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
+    {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -372,7 +379,7 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void execute(SSTableReader input) throws IOException
             {
-                scrubOne(cfs, input, skipCorrupted, checkData);
+                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
             }
         }, jobs);
     }
@@ -710,9 +717,9 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 
         CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
         metrics.beginCompaction(scrubInfo);
@@ -1352,7 +1359,7 @@ public class CompactionManager implements CompactionManagerMBean
         public void afterExecute(Runnable r, Throwable t)
         {
             DebuggableThreadPoolExecutor.maybeResetTraceSessionWrapper(r);
-    
+
             if (t == null)
                 t = DebuggableThreadPoolExecutor.extractThrowable(r);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 2df3665..6d4537c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
@@ -35,6 +36,7 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class Scrubber implements Closeable
 {
@@ -43,6 +45,7 @@ public class Scrubber implements Closeable
     private final File destination;
     private final boolean skipCorrupted;
     public final boolean validateColumns;
+    private final boolean reinsertOverflowedTTLRows;
 
     private final CompactionController controller;
     private final boolean isCommutative;
@@ -67,6 +70,7 @@ public class Scrubber implements Closeable
     long nextRowPositionFromIndex;
 
     private final OutputHandler outputHandler;
+    private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
 
     private static final Comparator<Row> rowComparator = new Comparator<Row>()
     {
@@ -79,10 +83,17 @@ public class Scrubber implements Closeable
 
     public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
+    {
+        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
+    }
+
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
     {
         this.cfs = cfs;
         this.sstable = sstable;
@@ -90,6 +101,7 @@ public class Scrubber implements Closeable
         this.skipCorrupted = skipCorrupted;
         this.isOffline = isOffline;
         this.validateColumns = checkData;
+        this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 
         List<SSTableReader> toScrub = Collections.singletonList(sstable);
 
@@ -131,6 +143,9 @@ public class Scrubber implements Closeable
 
         this.currentRowPositionFromIndex = 0;
         this.nextRowPositionFromIndex = 0;
+
+        if (reinsertOverflowedTTLRows)
+            outputHandler.output("Starting scrub with reinsert overflowed TTL option");
     }
 
     public void scrub()
@@ -311,6 +326,8 @@ public class Scrubber implements Closeable
         else
         {
             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+            if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
             if (badRows > 0)
                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
         }
@@ -322,7 +339,7 @@ public class Scrubber implements Closeable
         // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
         // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
         // to the outOfOrderRows that will be later written to a new SSTable.
-        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
+        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
                                                               cfs.metadata.comparator.onDiskAtomComparator());
         if (prevKey != null && prevKey.compareTo(key) > 0)
         {
@@ -342,6 +359,18 @@ public class Scrubber implements Closeable
         return true;
     }
 
+    /**
+     * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+     * is specified
+     */
+    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
+    {
+        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
+        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                    outputHandler,
+                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+    }
+
     private void updateIndexKey()
     {
         currentIndexKey = nextIndexKey;
@@ -516,6 +545,11 @@ public class Scrubber implements Closeable
         }
     }
 
+    public class NegativeLocalDeletionInfoMetrics
+    {
+        public volatile int fixedRows = 0;
+    }
+
     /**
      * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
      * cells order and collect the out of order cells to correct the problem.
@@ -601,4 +635,61 @@ public class Scrubber implements Closeable
             return cf;
         }
     }
+
+    /**
+     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
+     *
+     * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+     */
+    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+    {
+        /**
+         * The decorated iterator.
+         */
+        private final OnDiskAtomIterator iterator;
+
+        private final OutputHandler outputHandler;
+        private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+
+        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
+                                                    NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+        {
+            this.iterator = iterator;
+            this.outputHandler = outputHandler;
+            this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+        }
+
+        public ColumnFamily getColumnFamily()
+        {
+            return iterator.getColumnFamily();
+        }
+
+        public DecoratedKey getKey()
+        {
+            return iterator.getKey();
+        }
+
+        public void close() throws IOException
+        {
+            iterator.close();
+        }
+
+        @Override
+        protected OnDiskAtom computeNext()
+        {
+            if (!iterator.hasNext())
+                return endOfData();
+
+            OnDiskAtom next = iterator.next();
+
+            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
+            {
+                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
+                negativeLocalExpirationTimeMetrics.fixedRows++;
+                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
+            }
+
+            return next;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 557c3de..d718765 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -28,9 +28,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CounterId;
@@ -155,7 +157,20 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS) throws IOException
     {
-        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        int localExpirationTime = (int) (expirationTimestampMS / 1000);
+        try
+        {
+            // This will throw exception if policy is REJECT and now() + ttl is higher than MAX_DELETION_TIME
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+            // If exception was not thrown, this means the policy was CAP, so we check for overflow and cap if that's the case
+            if (localExpirationTime < 0)
+                localExpirationTime = BufferExpiringCell.MAX_DELETION_TIME;
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e);
+        }
+        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, localExpirationTime));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 27939f9..a7a8ca7 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2415,10 +2415,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d3a1725..90c0fb5 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -219,16 +219,16 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
-     * 
+     *
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
      * @param columnFamilyList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
-    
-    
-    
+
+
+
     /**
      * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
@@ -274,8 +274,11 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
@@ -381,11 +384,11 @@ public interface StorageServiceMBean extends NotificationEmitter
      * If level cannot be parsed, then the level will be defaulted to DEBUG<br>
      * <br>
      * The logback configuration should have < jmxConfigurator /> set
-     * 
+     *
      * @param classQualifier The logger's classQualifer
      * @param level The log level
-     * @throws Exception 
-     * 
+     * @throws Exception
+     *
      *  @see ch.qos.logback.classic.Level#toLevel(String)
      */
     public void setLoggingLevel(String classQualifier, String level) throws Exception;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d5d9f73..10e7185 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -24,6 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -314,7 +315,7 @@ public class ThriftValidation
             if (isCommutative)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 
-            validateTtl(cosc.column);
+            validateTtl(metadata, cosc.column);
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
             validateColumnData(metadata, key, null, cosc.column);
         }
@@ -349,7 +350,7 @@ public class ThriftValidation
         }
     }
 
-    private static void validateTtl(Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
+    private static void validateTtl(CFMetaData metadata, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
         if (column.isSetTtl())
         {
@@ -358,9 +359,11 @@ public class ThriftValidation
 
             if (column.ttl > ExpiringCell.MAX_TTL)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
         }
         else
         {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
             // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
             assert column.ttl == 0;
         }
@@ -434,7 +437,7 @@ public class ThriftValidation
      */
     public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        validateTtl(column);
+        validateTtl(metadata, column);
         if (!column.isSetValue())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
         if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 13c7acf..fcd4110 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -243,9 +243,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
     }
 
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
@@ -268,10 +268,10 @@ public class NodeProbe implements AutoCloseable
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         checkJobs(out, jobs);
-        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
@@ -562,7 +562,7 @@ public class NodeProbe implements AutoCloseable
 
     /**
      * Take a snapshot of all column family from different keyspaces.
-     * 
+     *
      * @param snapshotName
      *            the name of the snapshot.
      * @param columnfamilylist
@@ -1302,7 +1302,7 @@ public class NodeProbe implements AutoCloseable
         }
         catch (Exception e)
         {
-          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e); 
+          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d1afb6f..54d7fb7 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -551,20 +551,20 @@ public class NodeTool
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } 
+            }
             catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
                 errors.append("Note: " + ex.getMessage() + "%n");
                 showEffectiveOwnership = false;
-            } 
+            }
             catch (IllegalArgumentException ex)
             {
                 System.out.printf("%nError: " + ex.getMessage() + "%n");
                 return;
             }
 
-            
+
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
                 printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
@@ -1282,6 +1282,11 @@ public class NodeTool
                 description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
         private int jobs = 2;
 
+        @Option(title = "reinsert_overflowed_ttl",
+        name = {"r", "--reinsert-overflowed-ttl"},
+        description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+        private boolean reinsertOverflowedTTL = false;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1292,7 +1297,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, reinsertOverflowedTTL   , !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -2197,7 +2202,7 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
-            
+
             StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships = null;
@@ -2249,9 +2254,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-            
+
             System.out.printf("%n" + errors.toString());
-            
+
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -2726,7 +2731,7 @@ public class NodeTool
                 probe.truncateHints(endpoint);
         }
     }
-    
+
     @Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
     public static class SetLoggingLevel extends NodeToolCmd
     {
@@ -2741,7 +2746,7 @@ public class NodeTool
             probe.setLoggingLevel(classQualifier, level);
         }
     }
-    
+
     @Command(name = "getlogginglevels", description = "Get the runtime logging levels")
     public static class GetLoggingLevels extends NodeToolCmd
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index fdf6c8d..59d13d5 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -47,6 +47,12 @@ import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 public class StandaloneScrubber
 {
+    public static final String REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION = "Rewrites rows with overflowed expiration date affected by CASSANDRA-14092 with " +
+                                                                            "the maximum supported expiration date of 2038-01-19T03:14:06+00:00. " +
+                                                                            "The rows are rewritten with the original timestamp incremented by one millisecond " +
+                                                                            "to override/supersede any potential tombstone that may have been generated " +
+                                                                            "during compaction of the affected rows.";
+
     private static final String TOOL_NAME = "sstablescrub";
     private static final String VERBOSE_OPTION  = "verbose";
     private static final String DEBUG_OPTION  = "debug";
@@ -54,6 +60,7 @@ public class StandaloneScrubber
     private static final String MANIFEST_CHECK_OPTION  = "manifest-check";
     private static final String SKIP_CORRUPTED_OPTION = "skip-corrupted";
     private static final String NO_VALIDATE_OPTION = "no-validate";
+    private static final String REINSERT_OVERFLOWED_TTL_OPTION = "reinsert-overflowed-ttl";
 
     public static void main(String args[])
     {
@@ -110,7 +117,7 @@ public class StandaloneScrubber
                 {
                     try
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
                         try
                         {
                             scrubber.scrub();
@@ -192,6 +199,7 @@ public class StandaloneScrubber
         public boolean manifestCheckOnly;
         public boolean skipCorrupted;
         public boolean noValidate;
+        public boolean reinsertOverflowedTTL;
 
         private Options(String keyspaceName, String cfName)
         {
@@ -232,6 +240,7 @@ public class StandaloneScrubber
                 opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                 opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                 opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
+                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
 
                 return opts;
             }
@@ -258,6 +267,7 @@ public class StandaloneScrubber
             options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
             options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
             options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
+            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
             return options;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d7cc13b
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
new file mode 100644
index 0000000..0e3da66
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
new file mode 100644
index 0000000..8a6dcba
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
@@ -0,0 +1 @@
+4012184764
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
new file mode 100644
index 0000000..3ab96ee
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
new file mode 100644
index 0000000..9bde77e
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..38373b4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
new file mode 100644
index 0000000..bdd4549
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
new file mode 100644
index 0000000..f58914a
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
@@ -0,0 +1 @@
+3463582096
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
new file mode 100644
index 0000000..38a6e4c
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
new file mode 100644
index 0000000..8ee9116
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..04a7384
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
new file mode 100644
index 0000000..1fc8ba4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
new file mode 100644
index 0000000..cd091ad
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1524836732
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
new file mode 100644
index 0000000..5fb34e8
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
new file mode 100644
index 0000000..4d961fb
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..c814fef
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
new file mode 100644
index 0000000..92032a7
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
new file mode 100644
index 0000000..a45d821
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2189764235
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
new file mode 100644
index 0000000..8291383
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
new file mode 100644
index 0000000..68f76ae
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db


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


[22/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
Merge branch 'cassandra-2.2' into cassandra-3.0


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

Branch: refs/heads/cassandra-3.11
Commit: c231ed5be0f47b030181185f544132523a2cf908
Parents: 21978bf 1602e60
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:55:27 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:57:53 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 +++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 ++
 conf/jvm.options                                |  11 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  14 +-
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   8 +-
 .../db/ExpirationDateOverflowHandling.java      | 121 +++++++
 .../org/apache/cassandra/db/LegacyLayout.java   |   7 +-
 .../org/apache/cassandra/db/LivenessInfo.java   |  12 +-
 .../db/compaction/CompactionManager.java        |  16 +-
 .../cassandra/db/compaction/Scrubber.java       | 198 ++++++++++-
 .../apache/cassandra/db/rows/BufferCell.java    |   7 +-
 src/java/org/apache/cassandra/db/rows/Cell.java |   9 +
 .../cassandra/service/StorageService.java       |   7 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   8 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/SSTableMetadataViewer.java  |   1 +
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/mc-1-big-Data.db                     | Bin 0 -> 58 bytes
 .../table1/mc-1-big-Digest.crc32                |   1 +
 .../table1/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table1/mc-1-big-Statistics.db               | Bin 0 -> 4676 bytes
 .../table1/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table1/mc-1-big-TOC.txt                     |   8 +
 .../table2/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/mc-1-big-Data.db                     | Bin 0 -> 60 bytes
 .../table2/mc-1-big-Digest.crc32                |   1 +
 .../table2/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table2/mc-1-big-Statistics.db               | Bin 0 -> 4686 bytes
 .../table2/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table2/mc-1-big-TOC.txt                     |   8 +
 .../table3/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/mc-1-big-Data.db                     | Bin 0 -> 111 bytes
 .../table3/mc-1-big-Digest.crc32                |   1 +
 .../table3/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table3/mc-1-big-Statistics.db               | Bin 0 -> 4732 bytes
 .../table3/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table3/mc-1-big-TOC.txt                     |   8 +
 .../table4/mc-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/mc-1-big-Data.db                     | Bin 0 -> 108 bytes
 .../table4/mc-1-big-Digest.crc32                |   1 +
 .../table4/mc-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Index.db                    | Bin 0 -> 16 bytes
 .../table4/mc-1-big-Statistics.db               | Bin 0 -> 4742 bytes
 .../table4/mc-1-big-Summary.db                  | Bin 0 -> 56 bytes
 .../table4/mc-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 327 ++++++++++++++++++-
 .../unit/org/apache/cassandra/db/ScrubTest.java |   6 +-
 57 files changed, 867 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --cc CASSANDRA-14092.txt
index 0000000,5ac872c..f95380b
mode 000000,100644..100644
--- a/CASSANDRA-14092.txt
+++ b/CASSANDRA-14092.txt
@@@ -1,0 -1,81 +1,81 @@@
+ CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+ ---------------------------------------------
+ 
+ The maximum expiration timestamp that can be represented by the storage engine is
+ 2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+ after this date are not currently supported.
+ 
+ # Expiration Date Overflow Policy
+ 
+ We plan to lift this limitation in newer versions, but while the fix is not available,
+ operators can decide which policy to apply when dealing with inserts with TTL exceeding
+ the maximum supported expiration date:
+   -     REJECT: this is the default policy and will reject any requests with expiration
+                 date timestamp after 2038-01-19T03:14:06+00:00.
+   -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                 2038-01-19T03:14:06+00:00 and the client will receive a warning.
+   - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+ 
+ These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
 -startup option.
++startup option in the jvm.options configuration file.
+ 
+ # Potential data loss on earlier versions
+ 
+ Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+ INSERTS with TTL expiring after the maximum supported date, causing the expiration
+ time field to overflow and the records to expire immediately. Expired records due
+ to overflow will not be queryable and will be permanently removed after a compaction.
+ 
+ 2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+ since an AssertionError is thrown during INSERT when the expiration time field overflows
+ on these versions. When assertions are disabled then it is possible to INSERT entries
+ with overflowed local expiration time and even the earlier versions are subject to data
+ loss due to this bug.
+ 
+ This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+ of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+ the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+ For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+ will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+ where this issue is addressed as soon as possible.
+ 
+ # Data Recovery
+ 
+ SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+ with overflowed expiration time that were backed up or did not go through compaction can
+ be recovered by reinserting overflowed entries with a valid expiration time and a higher
+ timestamp, since tombstones may have been generated with the original timestamp.
+ 
+ To find out if an SSTable has an entry with overflowed expiration, inspect it with the
 -sstable2json tool and look for a negative "local deletion time" field. SSTables in this
 -condition should be backed up immediately, as they are subject to data loss during
++'sstablemetadata' tool and look for a negative "Minimum local deletion time" field. SSTables
++in this condition should be backed up immediately, as they are subject to data loss during
+ compaction.
+ 
+ A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+ rows with overflowed expiration time with the maximum expiration date of
+ 2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+ for recovery of SSTables via scrub:
+ 
+ - Offline scrub:
+    - Clone the data directory tree to another location, keeping only the folders and the
+      contents of the system tables.
+    - Clone the configuration directory to another location, setting the data_file_directories
+      property to the cloned data directory in the cloned cassandra.yaml.
+    - Copy the affected SSTables to the cloned data location of the affected table.
+    - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+    - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+          WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+          compaction, so the data with overflowed will be removed - make sure to back up
+          your SSTables before running scrub.
+    - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+    - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+ 
+ - Online scrub:
+    - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+      as otherwise, the data may be removed permanently during compaction.
+    - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+    - run "nodetool refresh" to load the SSTables.
+    - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+    - Re-enable compactions after verifying that scrub recovered the missing entries.
+ 
 -See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
++See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index f42f3f4,82da6ad..a492c42
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,29 -1,4 +1,30 @@@
 -2.2.12
 +3.0.16
++ * Protect against overflow of local expiration time (CASSANDRA-14092)
 + * Use the correct digest file and reload sstable metadata in nodetool verify (CASSANDRA-14217)
 + * Handle failure when mutating repaired status in Verifier (CASSANDRA-13933)
 + * Close socket on error during connect on OutboundTcpConnection (CASSANDRA-9630)
 + * Set encoding for javadoc generation (CASSANDRA-14154)
 + * Fix index target computation for dense composite tables with dropped compact storage (CASSANDRA-14104)
 + * Improve commit log chain marker updating (CASSANDRA-14108)
 + * Extra range tombstone bound creates double rows (CASSANDRA-14008)
 + * Fix SStable ordering by max timestamp in SinglePartitionReadCommand (CASSANDRA-14010)
 + * Accept role names containing forward-slash (CASSANDRA-14088)
 + * Optimize CRC check chance probability calculations (CASSANDRA-14094)
 + * Fix cleanup on keyspace with no replicas (CASSANDRA-13526)
 + * Fix updating base table rows with TTL not removing materialized view entries (CASSANDRA-14071)
 + * Reduce garbage created by DynamicSnitch (CASSANDRA-14091)
 + * More frequent commitlog chained markers (CASSANDRA-13987)
 + * Fix serialized size of DataLimits (CASSANDRA-14057)
 + * Add flag to allow dropping oversized read repair mutations (CASSANDRA-13975)
 + * Fix SSTableLoader logger message (CASSANDRA-14003)
 + * Fix repair race that caused gossip to block (CASSANDRA-13849)
 + * Tracing interferes with digest requests when using RandomPartitioner (CASSANDRA-13964)
 + * Add flag to disable materialized views, and warnings on creation (CASSANDRA-13959)
 + * Don't let user drop or generally break tables in system_distributed (CASSANDRA-13813)
 + * Provide a JMX call to sync schema with local storage (CASSANDRA-13954)
 + * Mishandling of cells for removed/dropped columns when reading legacy files (CASSANDRA-13939)
 + * Deserialise sstable metadata in nodetool verify (CASSANDRA-13922)
 +Merged from 2.2:
   * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
   * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
   * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 1bbe1aa,4fe3508..f574c33
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -13,71 -33,7 +33,72 @@@ restore snapshots created with the prev
  'sstableloader' tool. You can upgrade the file format of your snapshots
  using the provided 'sstableupgrade' tool.
  
 -2.2.12
 +3.0.16
 +=====
 +
 +Upgrading
 +---------
++   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +   - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +     rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +     are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +     options are used. See CASSANDRA-13006 for more details.
 +   - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +     set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +     for more details.
 +
 +Materialized Views
 +-------------------
 +   - Following a discussion regarding concerns about the design and safety of Materialized Views, the C* development
 +     community no longer recommends them for production use, and considers them experimental. Warnings messages will
 +     now be logged when they are created. (See https://www.mail-archive.com/dev@cassandra.apache.org/msg11511.html)
 +   - An 'enable_materialized_views' flag has been added to cassandra.yaml to allow operators to prevent creation of
 +     views
 +
 +3.0.15
 +=====
 +
 +Upgrading
 +---------
 +   - Nothing specific to this release, but please see previous upgrading sections,
 +     especially if you are upgrading from 2.2.
 +
 +Compact Storage
 +---------------
 +    - Starting version 4.0, Thrift and COMPACT STORAGE is no longer supported.
 +      'ALTER ... DROP COMPACT STORAGE' statement makes Compact Tables CQL-compatible,
 +      exposing internal structure of Thrift/Compact Tables. You can find more details
 +      on exposed internal structure under: 
 +      http://cassandra.apache.org/doc/latest/cql/appendices.html#appendix-c-dropping-compact-storage
 +
 +      For uninterrupted cluster upgrades, drivers now support 'NO_COMPACT' startup option.
 +      Supplying this flag will have same effect as 'DROP COMPACT STORAGE', but only for the
 +      current connection.
 +
 +      In order to upgrade, clients supporting a non-compact schema view can be rolled out
 +      gradually. When all the clients are updated 'ALTER ... DROP COMPACT STORAGE' can be
 +      executed. After dropping compact storage, ’NO_COMPACT' option will have no effect
 +      after that.
 +
 +Materialized Views
 +-------------------
 +    - Cassandra will no longer allow dropping columns on tables with Materialized Views.
 +    - A change was made in the way the Materialized View timestamp is computed, which
 +      may cause an old deletion to a base column which is view primary key (PK) column
 +      to not be reflected in the view when repairing the base table post-upgrade. This
 +      condition is only possible when a column deletion to an MV primary key (PK) column
 +      not present in the base table PK (via UPDATE base SET view_pk_col = null or DELETE
 +      view_pk_col FROM base) is missed before the upgrade and received by repair after the upgrade.
 +      If such column deletions are done on a view PK column which is not a base PK, it's advisable
 +      to run repair on the base table of all nodes prior to the upgrade. Alternatively it's possible
 +      to fix potential inconsistencies by running repair on the views after upgrade or drop and
 +      re-create the views. See CASSANDRA-11500 for more details.
 +    - Removal of columns not selected in the Materialized View (via UPDATE base SET unselected_column
 +      = null or DELETE unselected_column FROM base) may not be properly reflected in the view in some
 +      situations so we advise against doing deletions on base columns not selected in views
 +      until this is fixed on CASSANDRA-13826.
 +
 +3.0.14
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/conf/jvm.options
----------------------------------------------------------------------
diff --cc conf/jvm.options
index a7b3bd8,0000000..eb2ad19
mode 100644,000000..100644
--- a/conf/jvm.options
+++ b/conf/jvm.options
@@@ -1,108 -1,0 +1,119 @@@
 +###########################################################################
 +#                             jvm.options                                 #
 +#                                                                         #
 +# - all flags defined here will be used by cassandra to startup the JVM   #
 +# - one flag should be specified per line                                 #
 +# - lines that do not start with '-' will be ignored                      #
 +# - only static flags are accepted (no variables or parameters)           #
 +# - dynamic flags will be appended to these on cassandra-env              #
 +###########################################################################
 +
 +#################
 +# HEAP SETTINGS #
 +#################
 +
 +# Heap size is automatically calculated by cassandra-env based on this
 +# formula: max(min(1/2 ram, 1024MB), min(1/4 ram, 8GB))
 +# That is:
 +# - calculate 1/2 ram and cap to 1024MB
 +# - calculate 1/4 ram and cap to 8192MB
 +# - pick the max
 +#
 +# For production use you may wish to adjust this for your environment.
 +# If that's the case, uncomment the -Xmx and Xms options below to override the
 +# automatic calculation of JVM heap memory.
 +#
 +# It is recommended to set min (-Xms) and max (-Xmx) heap sizes to
 +# the same value to avoid stop-the-world GC pauses during resize, and
 +# so that we can lock the heap in memory on startup to prevent any
 +# of it from being swapped out.
 +#-Xms4G
 +#-Xmx4G
 +
 +# Young generation size is automatically calculated by cassandra-env
 +# based on this formula: min(100 * num_cores, 1/4 * heap size)
 +#
 +# The main trade-off for the young generation is that the larger it
 +# is, the longer GC pause times will be. The shorter it is, the more
 +# expensive GC will be (usually).
 +#
 +# It is not recommended to set the young generation size if using the
 +# G1 GC, since that will override the target pause-time goal.
 +# More info: http://www.oracle.com/technetwork/articles/java/g1gc-1984535.html
 +#
 +# The example below assumes a modern 8-core+ machine for decent
 +# times. If in doubt, and if you do not particularly want to tweak, go
 +# 100 MB per physical CPU core.
 +#-Xmn800M
 +
++###################################
++# EXPIRATION DATE OVERFLOW POLICY #
++###################################
++
++# Defines how to handle INSERT requests with TTL exceeding the maximum supported expiration date:
++# * REJECT: this is the default policy and will reject any requests with expiration date timestamp after 2038-01-19T03:14:06+00:00.
++# * CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on 2038-01-19T03:14:06+00:00 and the client will receive a warning.
++# * CAP_NOWARN: same as previous, except that the client warning will not be emitted.
++#
++#-Dcassandra.expiration_date_overflow_policy=REJECT
++
 +#################
 +#  GC SETTINGS  #
 +#################
 +
 +### CMS Settings
 +
 +-XX:+UseParNewGC
 +-XX:+UseConcMarkSweepGC
 +-XX:+CMSParallelRemarkEnabled
 +-XX:SurvivorRatio=8
 +-XX:MaxTenuringThreshold=1
 +-XX:CMSInitiatingOccupancyFraction=75
 +-XX:+UseCMSInitiatingOccupancyOnly
 +-XX:CMSWaitDuration=10000
 +-XX:+CMSParallelInitialMarkEnabled
 +-XX:+CMSEdenChunksRecordAlways
 +# some JVMs will fill up their heap when accessed via JMX, see CASSANDRA-6541
 +-XX:+CMSClassUnloadingEnabled
 +
 +### G1 Settings (experimental, comment previous section and uncomment section below to enable)
 +
 +## Use the Hotspot garbage-first collector.
 +#-XX:+UseG1GC
 +#
 +## Have the JVM do less remembered set work during STW, instead
 +## preferring concurrent GC. Reduces p99.9 latency.
 +#-XX:G1RSetUpdatingPauseTimePercent=5
 +#
 +## Main G1GC tunable: lowering the pause target will lower throughput and vise versa.
 +## 200ms is the JVM default and lowest viable setting
 +## 1000ms increases throughput. Keep it smaller than the timeouts in cassandra.yaml.
 +#-XX:MaxGCPauseMillis=500
 +
 +## Optional G1 Settings
 +
 +# Save CPU time on large (>= 16GB) heaps by delaying region scanning
 +# until the heap is 70% full. The default in Hotspot 8u40 is 40%.
 +#-XX:InitiatingHeapOccupancyPercent=70
 +
 +# For systems with > 8 cores, the default ParallelGCThreads is 5/8 the number of logical cores.
 +# Otherwise equal to the number of cores when 8 or less.
 +# Machines with > 10 cores should try setting these to <= full cores.
 +#-XX:ParallelGCThreads=16
 +# By default, ConcGCThreads is 1/4 of ParallelGCThreads.
 +# Setting both to the same value can reduce STW durations.
 +#-XX:ConcGCThreads=16
 +
 +### GC logging options -- uncomment to enable
 +
 +-XX:+PrintGCDetails
 +-XX:+PrintGCDateStamps
 +-XX:+PrintHeapAtGC
 +-XX:+PrintTenuringDistribution
 +-XX:+PrintGCApplicationStoppedTime
 +-XX:+PrintPromotionFailure
 +#-XX:PrintFLSStatistics=1
 +#-Xloggc:/var/log/cassandra/gc.log
 +-XX:+UseGCLogFileRotation
 +-XX:NumberOfGCLogFiles=10
 +-XX:GCLogFileSize=10M

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/redhat/cassandra.spec
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 4ed0f83,84f423a..832d0a7
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,12 -18,17 +18,11 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
--import java.util.Collections;
 -import java.util.concurrent.TimeUnit;
 -
 -import com.google.common.collect.Iterables;
 -import com.google.common.annotations.VisibleForTesting;
 -import org.slf4j.Logger;
 -import org.slf4j.LoggerFactory;
 +import java.util.List;
  
- import com.google.common.collect.Iterables;
- 
+ import org.apache.cassandra.config.CFMetaData;
  import org.apache.cassandra.cql3.functions.Function;
 -import org.apache.cassandra.db.ExpiringCell;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
@@@ -100,10 -137,13 +99,13 @@@ public class Attribute
          return LongType.instance.compose(tval);
      }
  
-     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+     public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
      {
          if (timeToLive == null)
-             return 0;
+         {
 -            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
 -            return metadata.getDefaultTimeToLive();
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
++            return metadata.params.defaultTimeToLive;
+         }
  
          ByteBuffer tval = timeToLive.bindAndGet(options);
          if (tval == null)
@@@ -125,9 -165,11 +127,11 @@@
          if (ttl < 0)
              throw new InvalidRequestException("A TTL must be greater or equal to 0, but was " + ttl);
  
 -        if (ttl > ExpiringCell.MAX_TTL)
 -            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 +        if (ttl > MAX_TTL)
 +            throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, MAX_TTL));
  
 -        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
++        ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+ 
          return ttl;
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 536681f,45908de..b5946bb
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -1379,13 -1610,12 +1379,13 @@@ public class ColumnFamilyStore implemen
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
++        return scrub(disableSnapshot, skipCorrupted, reinsertOverflowedTTL, false, checkData, jobs);
      }
  
 -    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
 +    @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean reinsertOverflowedTTL, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
      {
          // skip snapshot creation during scrub, SEE JIRA 5891
          if(!disableSnapshot)
@@@ -1393,7 -1623,7 +1393,7 @@@
  
          try
          {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
 -            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTL, jobs);
          }
          catch(Throwable t)
          {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
index 0000000,0000000..852dcb1
new file mode 100644
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/ExpirationDateOverflowHandling.java
@@@ -1,0 -1,0 +1,121 @@@
++/*
++ * 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;
++
++import java.util.concurrent.TimeUnit;
++
++import com.google.common.annotations.VisibleForTesting;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++import org.slf4j.helpers.MessageFormatter;
++
++import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.cql3.Attributes;
++import org.apache.cassandra.db.rows.BufferCell;
++import org.apache.cassandra.db.rows.Cell;
++import org.apache.cassandra.exceptions.InvalidRequestException;
++import org.apache.cassandra.service.ClientWarn;
++import org.apache.cassandra.utils.NoSpamLogger;
++
++public class ExpirationDateOverflowHandling
++{
++    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
++
++    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
++
++    public enum ExpirationDateOverflowPolicy
++    {
++        REJECT, CAP_NOWARN, CAP
++    }
++
++    @VisibleForTesting
++    public static ExpirationDateOverflowPolicy policy;
++
++    static {
++        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
++        try
++        {
++            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
++        }
++        catch (RuntimeException e)
++        {
++            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
++            policy = ExpirationDateOverflowPolicy.REJECT;
++        }
++    }
++
++    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
++                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
++                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
++                                                                          "is fixed. See CASSANDRA-14092 for more details.";
++
++    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
++                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
++                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
++                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
++
++    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
++    {
++        if (ttl == BufferCell.NO_TTL)
++            return;
++
++        // Check for localExpirationTime overflow (CASSANDRA-14092)
++        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
++        if (ttl + nowInSecs < 0)
++        {
++            switch (policy)
++            {
++                case CAP:
++                    ClientWarn.instance.warn(MessageFormatter.arrayFormat(MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING, new Object[] { metadata.ksName,
++                                                                                                                                   metadata.cfName,
++                                                                                                                                   isDefaultTTL? "default " : "", ttl })
++                                                             .getMessage());
++                case CAP_NOWARN:
++                    /**
++                     * Capping at this stage is basically not rejecting the request. The actual capping is done
++                     * by {@link #computeLocalExpirationTime(int, int)}, which converts the negative TTL
++                     * to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
++                     */
++                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES, TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
++                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
++                    return;
++
++                default:
++                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
++                                                                    isDefaultTTL? "default " : "", ttl));
++            }
++        }
++    }
++
++    /**
++     * This method computes the {@link Cell#localDeletionTime()}, maybe capping to the maximum representable value
++     * which is {@link Cell#MAX_DELETION_TIME}.
++     *
++     * Please note that the {@link ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy} is applied
++     * during {@link ExpirationDateOverflowHandling#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
++     * so if the request was not denied it means its expiration date should be capped.
++     *
++     * See CASSANDRA-14092
++     */
++    public static int computeLocalExpirationTime(int nowInSec, int timeToLive)
++    {
++        int localExpirationTime = nowInSec + timeToLive;
++        return localExpirationTime >= 0? localExpirationTime : Cell.MAX_DELETION_TIME;
++    }
++}


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


[16/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LivenessInfo.java
index 89e0578,0000000..f6c9b62
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@@ -1,369 -1,0 +1,375 @@@
 +/*
 + * 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;
 +
 +import java.util.Objects;
 +import java.security.MessageDigest;
 +
 +import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.db.rows.Cell;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * Stores the information relating to the liveness of the primary key columns of a row.
 + * <p>
 + * A {@code LivenessInfo} can first be empty. If it isn't, it contains at least a timestamp,
 + * which is the timestamp for the row primary key columns. On top of that, the info can be
 + * ttl'ed, in which case the {@code LivenessInfo} also has both a ttl and a local expiration time.
 + * <p>
 + * Please note that if a liveness info is ttl'ed, that expiration is <b>only</b> an expiration
 + * of the liveness info itself (so, of the timestamp), and once the info expires it becomes
 + * {@code EMPTY}. But if a row has a liveness info which expires, the rest of the row data is
 + * unaffected (of course, the rest of said row data might be ttl'ed on its own but this is
 + * separate).
 + */
 +public class LivenessInfo
 +{
 +    public static final long NO_TIMESTAMP = Long.MIN_VALUE;
-     public static final int NO_TTL = 0;
++    public static final int NO_TTL = Cell.NO_TTL;
 +    /**
 +     * Used as flag for representing an expired liveness.
 +     *
 +     * TTL per request is at most 20 yrs, so this shouldn't conflict
 +     * (See {@link org.apache.cassandra.cql3.Attributes#MAX_TTL})
 +     */
 +    public static final int EXPIRED_LIVENESS_TTL = Integer.MAX_VALUE;
-     public static final int NO_EXPIRATION_TIME = Integer.MAX_VALUE;
++    public static final int NO_EXPIRATION_TIME = Cell.NO_DELETION_TIME;
 +
 +    public static final LivenessInfo EMPTY = new LivenessInfo(NO_TIMESTAMP);
 +
 +    protected final long timestamp;
 +
 +    protected LivenessInfo(long timestamp)
 +    {
 +        this.timestamp = timestamp;
 +    }
 +
 +    public static LivenessInfo create(CFMetaData metadata, long timestamp, int nowInSec)
 +    {
 +        int defaultTTL = metadata.params.defaultTimeToLive;
 +        if (defaultTTL != NO_TTL)
 +            return expiring(timestamp, defaultTTL, nowInSec);
 +
 +        return new LivenessInfo(timestamp);
 +    }
 +
 +    public static LivenessInfo expiring(long timestamp, int ttl, int nowInSec)
 +    {
 +        assert ttl != EXPIRED_LIVENESS_TTL;
-         return new ExpiringLivenessInfo(timestamp, ttl, nowInSec + ttl);
++        return new ExpiringLivenessInfo(timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl));
 +    }
 +
 +    public static LivenessInfo create(CFMetaData metadata, long timestamp, int ttl, int nowInSec)
 +    {
 +        return ttl == NO_TTL
 +             ? create(metadata, timestamp, nowInSec)
 +             : expiring(timestamp, ttl, nowInSec);
 +    }
 +
 +    // Note that this ctor ignores the default table ttl and takes the expiration time, not the current time.
 +    // Use when you know that's what you want.
 +    public static LivenessInfo create(long timestamp, int ttl, int localExpirationTime)
 +    {
 +        if (ttl == EXPIRED_LIVENESS_TTL)
 +            return new ExpiredLivenessInfo(timestamp, ttl, localExpirationTime);
 +        return ttl == NO_TTL ? new LivenessInfo(timestamp) : new ExpiringLivenessInfo(timestamp, ttl, localExpirationTime);
 +    }
 +
 +    /**
 +     * Whether this liveness info is empty (has no timestamp).
 +     *
 +     * @return whether this liveness info is empty or not.
 +     */
 +    public boolean isEmpty()
 +    {
 +        return timestamp == NO_TIMESTAMP;
 +    }
 +
 +    /**
 +     * The timestamp for this liveness info.
 +     *
 +     * @return the liveness info timestamp (or {@link #NO_TIMESTAMP} if the info is empty).
 +     */
 +    public long timestamp()
 +    {
 +        return timestamp;
 +    }
 +
 +    /**
 +     * Whether the info has a ttl.
 +     */
 +    public boolean isExpiring()
 +    {
 +        return false;
 +    }
 +
 +    /**
 +     * The ttl (if any) on the row primary key columns or {@link #NO_TTL} if it is not
 +     * expiring.
 +     *
 +     * Please note that this value is the TTL that was set originally and is thus not
 +     * changing.
 +     */
 +    public int ttl()
 +    {
 +        return NO_TTL;
 +    }
 +
 +    /**
 +     * The expiration time (in seconds) if the info is expiring ({@link #NO_EXPIRATION_TIME} otherwise).
 +     *
 +     */
 +    public int localExpirationTime()
 +    {
 +        return NO_EXPIRATION_TIME;
 +    }
 +
 +    /**
 +     * Whether that info is still live.
 +     *
 +     * A {@code LivenessInfo} is live if it is either not expiring, or if its expiration time if after
 +     * {@code nowInSec}.
 +     *
 +     * @param nowInSec the current time in seconds.
 +     * @return whether this liveness info is live or not.
 +     */
 +    public boolean isLive(int nowInSec)
 +    {
 +        return !isEmpty();
 +    }
 +
 +    /**
 +     * Adds this liveness information to the provided digest.
 +     *
 +     * @param digest the digest to add this liveness information to.
 +     */
 +    public void digest(MessageDigest digest)
 +    {
 +        FBUtilities.updateWithLong(digest, timestamp());
 +    }
 +
 +    /**
 +     * Validate the data contained by this liveness information.
 +     *
 +     * @throws MarshalException if some of the data is corrupted.
 +     */
 +    public void validate()
 +    {
 +    }
 +
 +    /**
 +     * The size of the (useful) data this liveness information contains.
 +     *
 +     * @return the size of the data this liveness information contains.
 +     */
 +    public int dataSize()
 +    {
 +        return TypeSizes.sizeof(timestamp());
 +    }
 +
 +    /**
 +     * Whether this liveness information supersedes another one (that is
 +     * whether is has a greater timestamp than the other or not).
 +     *
 +     * </br>
 +     *
 +     * If timestamps are the same and none of them are expired livenessInfo,
 +     * livenessInfo with greater TTL supersedes another. It also means, if timestamps are the same,
 +     * ttl superseders no-ttl. This is the same rule as {@link Conflicts#resolveRegular}
 +     *
 +     * If timestamps are the same and one of them is expired livenessInfo. Expired livenessInfo
 +     * supersedes, ie. tombstone supersedes.
 +     *
 +     * If timestamps are the same and both of them are expired livenessInfo(Ideally it shouldn't happen),
 +     * greater localDeletionTime wins.
 +     *
 +     * @param other
 +     *            the {@code LivenessInfo} to compare this info to.
 +     *
 +     * @return whether this {@code LivenessInfo} supersedes {@code other}.
 +     */
 +    public boolean supersedes(LivenessInfo other)
 +    {
 +        if (timestamp != other.timestamp)
 +            return timestamp > other.timestamp;
 +        if (isExpired() ^ other.isExpired())
 +            return isExpired();
 +        if (isExpiring() == other.isExpiring())
 +            return localExpirationTime() > other.localExpirationTime();
 +        return isExpiring();
 +    }
 +
 +    protected boolean isExpired()
 +    {
 +        return false;
 +    }
 +
 +    /**
 +     * Returns a copy of this liveness info updated with the provided timestamp.
 +     *
 +     * @param newTimestamp the timestamp for the returned info.
 +     * @return if this liveness info has a timestamp, a copy of it with {@code newTimestamp}
 +     * as timestamp. If it has no timestamp however, this liveness info is returned
 +     * unchanged.
 +     */
 +    public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +    {
 +        return new LivenessInfo(newTimestamp);
 +    }
 +
++    public LivenessInfo withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return LivenessInfo.create(newTimestamp, ttl(), newLocalDeletionTime);
++    }
++
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("[ts=%d]", timestamp);
 +    }
 +
 +    @Override
 +    public boolean equals(Object other)
 +    {
 +        if(!(other instanceof LivenessInfo))
 +            return false;
 +
 +        LivenessInfo that = (LivenessInfo)other;
 +        return this.timestamp() == that.timestamp()
 +            && this.ttl() == that.ttl()
 +            && this.localExpirationTime() == that.localExpirationTime();
 +    }
 +
 +    @Override
 +    public int hashCode()
 +    {
 +        return Objects.hash(timestamp(), ttl(), localExpirationTime());
 +    }
 +
 +    /**
 +     * Effectively acts as a PK tombstone. This is used for Materialized Views to shadow
 +     * updated entries while co-existing with row tombstones.
 +     *
 +     * See {@link org.apache.cassandra.db.view.ViewUpdateGenerator#deleteOldEntryInternal}.
 +     */
 +    private static class ExpiredLivenessInfo extends ExpiringLivenessInfo
 +    {
 +        private ExpiredLivenessInfo(long timestamp, int ttl, int localExpirationTime)
 +        {
 +            super(timestamp, ttl, localExpirationTime);
 +            assert ttl == EXPIRED_LIVENESS_TTL;
 +            assert timestamp != NO_TIMESTAMP;
 +        }
 +
 +        @Override
 +        public boolean isExpired()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean isLive(int nowInSec)
 +        {
 +            // used as tombstone to shadow entire PK
 +            return false;
 +        }
 +
 +        @Override
 +        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +        {
 +            return new ExpiredLivenessInfo(newTimestamp, ttl(), localExpirationTime());
 +        }
 +    }
 +
 +    private static class ExpiringLivenessInfo extends LivenessInfo
 +    {
 +        private final int ttl;
 +        private final int localExpirationTime;
 +
 +        private ExpiringLivenessInfo(long timestamp, int ttl, int localExpirationTime)
 +        {
 +            super(timestamp);
 +            assert ttl != NO_TTL && localExpirationTime != NO_EXPIRATION_TIME;
 +            this.ttl = ttl;
 +            this.localExpirationTime = localExpirationTime;
 +        }
 +
 +        @Override
 +        public int ttl()
 +        {
 +            return ttl;
 +        }
 +
 +        @Override
 +        public int localExpirationTime()
 +        {
 +            return localExpirationTime;
 +        }
 +
 +        @Override
 +        public boolean isExpiring()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean isLive(int nowInSec)
 +        {
 +            return nowInSec < localExpirationTime;
 +        }
 +
 +        @Override
 +        public void digest(MessageDigest digest)
 +        {
 +            super.digest(digest);
 +            FBUtilities.updateWithInt(digest, localExpirationTime);
 +            FBUtilities.updateWithInt(digest, ttl);
 +        }
 +
 +        @Override
 +        public void validate()
 +        {
 +            if (ttl < 0)
 +                throw new MarshalException("A TTL should not be negative");
 +            if (localExpirationTime < 0)
 +                throw new MarshalException("A local expiration time should not be negative");
 +        }
 +
 +        @Override
 +        public int dataSize()
 +        {
 +            return super.dataSize()
 +                 + TypeSizes.sizeof(ttl)
 +                 + TypeSizes.sizeof(localExpirationTime);
 +
 +        }
 +
 +        @Override
 +        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +        {
 +            return new ExpiringLivenessInfo(newTimestamp, ttl, localExpirationTime);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("[ts=%d ttl=%d, let=%d]", timestamp, ttl, localExpirationTime);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index eaf6dab,d90abe9..1d54667
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -363,7 -348,13 +363,15 @@@ public class CompactionManager implemen
          }
      }
  
--    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
++                                           int jobs)
+     throws InterruptedException, ExecutionException
+     {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
++                                           final boolean reinsertOverflowedTTL, int jobs)
      throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@@ -377,7 -368,7 +385,7 @@@
              @Override
              public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
 -                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
++                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTL);
              }
          }, jobs, OperationType.SCRUB);
      }
@@@ -745,11 -736,11 +753,11 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL) throws IOException
      {
          CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTL))
          {
              scrubInfo = scrubber.getScrubInfo();
              metrics.beginCompaction(scrubInfo);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index c8e0c53,affee11..bc11504
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -35,8 -37,12 +35,9 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
 -import org.apache.cassandra.utils.ByteBufferUtil;
 -import org.apache.cassandra.utils.JVMStabilityInspector;
 -import org.apache.cassandra.utils.OutputHandler;
 -import org.apache.cassandra.utils.memory.HeapAllocator;
 -import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.*;
  import org.apache.cassandra.utils.concurrent.Refs;
++import org.apache.cassandra.utils.memory.HeapAllocator;
  
  public class Scrubber implements Closeable
  {
@@@ -45,7 -51,9 +46,8 @@@
      private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
+     private final boolean reinsertOverflowedTTLRows;
  
 -    private final CompactionController controller;
      private final boolean isCommutative;
      private final boolean isIndex;
      private final boolean checkData;
@@@ -65,38 -76,41 +67,47 @@@
      long currentRowPositionFromIndex;
      long nextRowPositionFromIndex;
  
 -    private final OutputHandler outputHandler;
+     private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
+ 
 -    private static final Comparator<Row> rowComparator = new Comparator<Row>()
 +    private final OutputHandler outputHandler;
 +
 +    private static final Comparator<Partition> partitionComparator = new Comparator<Partition>()
      {
 -         public int compare(Row r1, Row r2)
 +         public int compare(Partition r1, Partition r2)
           {
 -             return r1.key.compareTo(r2.key);
 +             return r1.partitionKey().compareTo(r2.partitionKey());
           }
      };
 -    private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
 +    private final SortedSet<Partition> outOfOrder = new TreeSet<>(partitionComparator);
  
      public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
++        this(cfs, transaction, skipCorrupted, checkData, false);
+     }
+ 
+     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
+         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
      @SuppressWarnings("resource")
 -    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
 +    public Scrubber(ColumnFamilyStore cfs,
 +                    LifecycleTransaction transaction,
 +                    boolean skipCorrupted,
 +                    OutputHandler outputHandler,
-                     boolean checkData) throws IOException
++                    boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
          this.transaction = transaction;
          this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 -
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
 +                                                                                                        sstable.descriptor.version,
 +                                                                                                        sstable.header);
- 
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
          // Calculate the expected compacted filesize
@@@ -134,19 -150,15 +145,22 @@@
  
          this.currentRowPositionFromIndex = 0;
          this.nextRowPositionFromIndex = 0;
+ 
+         if (reinsertOverflowedTTLRows)
+             outputHandler.output("Starting scrub with reinsert overflowed TTL option");
      }
  
 +    private UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
 +    {
 +        return checkData ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
 +    }
 +
      public void scrub()
      {
 +        List<SSTableReader> finished = new ArrayList<>();
 +        boolean completed = false;
          outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
 -        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, transaction.isOffline());
 +        try (SSTableRewriter writer = SSTableRewriter.construct(cfs, transaction, false, sstable.maxDataAge, transaction.isOffline());
               Refs<SSTableReader> refs = Refs.ref(Collections.singleton(sstable)))
          {
              nextIndexKey = indexAvailable() ? ByteBufferUtil.readWithShortLength(indexFile) : null;
@@@ -285,57 -311,66 +299,71 @@@
          }
          finally
          {
 -            controller.close();
 -            if (transaction.isOffline() && newSstable != null)
 -                newSstable.selfRef().release();
 +            if (transaction.isOffline())
 +                finished.forEach(sstable -> sstable.selfRef().release());
          }
  
 -        if (newSstable == null)
 -        {
 -            if (badRows > 0)
 -                outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
 -            else
 -                outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
 -        }
 -        else
 +        if (completed)
          {
+             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+             if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                 outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
              if (badRows > 0)
-                 outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
-             else
-                 outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
+                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
          }
 +        else
 +        {
-             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
 +            if (badRows > 0)
-                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
++                outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
++            else
++                outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
 +        }
      }
  
      @SuppressWarnings("resource")
      private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, SSTableRewriter writer)
      {
 -        // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
 -        // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
 -        // to the outOfOrderRows that will be later written to a new SSTable.
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
 -                                                              cfs.metadata.comparator.onDiskAtomComparator());
 -        if (prevKey != null && prevKey.compareTo(key) > 0)
 -        {
 -            saveOutOfOrderRow(prevKey, key, atoms);
 -            return false;
 -        }
 +        // OrderCheckerIterator will check, at iteration time, that the rows are in the proper order. If it detects
 +        // that one row is out of order, it will stop returning them. The remaining rows will be sorted and added
 +        // to the outOfOrder set that will be later written to a new SSTable.
-         OrderCheckerIterator sstableIterator = new OrderCheckerIterator(new RowMergingSSTableIterator(sstable, dataFile, key),
++        OrderCheckerIterator sstableIterator = new OrderCheckerIterator(getIterator(key),
 +                                                                        cfs.metadata.comparator);
  
 -        AbstractCompactedRow compactedRow = new LazilyCompactedRow(controller, Collections.singletonList(atoms));
 -        if (writer.tryAppend(compactedRow) == null)
 -            emptyRows++;
 -        else
 -            goodRows++;
 +        try (UnfilteredRowIterator iterator = withValidation(sstableIterator, dataFile.getPath()))
 +        {
 +            if (prevKey != null && prevKey.compareTo(key) > 0)
 +            {
 +                saveOutOfOrderRow(prevKey, key, iterator);
 +                return false;
 +            }
  
 -        if (atoms.hasOutOfOrderCells())
 -            saveOutOfOrderRow(key, atoms);
 +            if (writer.tryAppend(iterator) == null)
 +                emptyRows++;
 +            else
 +                goodRows++;
 +        }
 +
 +        if (sstableIterator.hasRowsOutOfOrder())
 +        {
 +            outputHandler.warn(String.format("Out of order rows found in partition: %s", key));
 +            outOfOrder.add(sstableIterator.getRowsOutOfOrder());
 +        }
  
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key)
++    private UnfilteredRowIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
 -        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
++        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(sstable, dataFile, key);
++        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(rowMergingIterator,
+                                                                                     outputHandler,
 -                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
++                                                                                    negativeLocalDeletionInfoMetrics) : rowMergingIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;
@@@ -477,49 -555,12 +505,55 @@@
          }
      }
  
+     public class NegativeLocalDeletionInfoMetrics
+     {
+         public volatile int fixedRows = 0;
+     }
+ 
      /**
 +     * During 2.x migration, under some circumstances rows might have gotten duplicated.
 +     * Merging iterator merges rows with same clustering.
 +     *
 +     * For more details, refer to CASSANDRA-12144.
 +     */
 +    private static class RowMergingSSTableIterator extends SSTableIdentityIterator
 +    {
 +        RowMergingSSTableIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key)
 +        {
 +            super(sstable, file, key);
 +        }
 +
 +        @Override
 +        protected Unfiltered doCompute()
 +        {
 +            if (!iterator.hasNext())
 +                return endOfData();
 +
 +            Unfiltered next = iterator.next();
 +            if (!next.isRow())
 +                return next;
 +
 +            while (iterator.hasNext())
 +            {
 +                Unfiltered peek = iterator.peek();
 +                // If there was a duplicate row, merge it.
 +                if (next.clustering().equals(peek.clustering()) && peek.isRow())
 +                {
 +                    iterator.next(); // Make sure that the peeked item was consumed.
 +                    next = Rows.merge((Row) next, (Row) peek, FBUtilities.nowInSeconds());
 +                }
 +                else
 +                {
 +                    break;
 +                }
 +            }
 +
 +            return next;
 +        }
++
 +    }
 +
 +    /**
       * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
       * cells order and collect the out of order cells to correct the problem.
       */
@@@ -571,53 -608,98 +605,199 @@@
          }
  
          @Override
 -        protected OnDiskAtom computeNext()
 +        public boolean isEmpty()
          {
 -            if (!iterator.hasNext())
 -                return endOfData();
 +            return iterator.isEmpty();
 +        }
  
 -            OnDiskAtom next = iterator.next();
 +        public void close()
 +        {
 +            iterator.close();
 +        }
  
 -            // If we detect that some cells are out of order we will store and sort the remaining once to insert them
 -            // in a separate SSTable.
 -            if (previous != null && comparator.compare(next, previous) < 0)
 -            {
 -                outOfOrderCells = collectOutOfOrderCells(next, iterator);
 -                return endOfData();
 -            }
 -            previous = next;
 -            return next;
 +        public DeletionTime partitionLevelDeletion()
 +        {
 +            return iterator.partitionLevelDeletion();
          }
  
 -        public boolean hasOutOfOrderCells()
 +        public EncodingStats stats()
          {
 -            return outOfOrderCells != null;
 +            return iterator.stats();
          }
  
 -        public ColumnFamily getOutOfOrderCells()
 +        public boolean hasRowsOutOfOrder()
          {
 -            return outOfOrderCells;
 +            return rowsOutOfOrder != null;
          }
  
 -        private static ColumnFamily collectOutOfOrderCells(OnDiskAtom atom, OnDiskAtomIterator iterator)
 +        public Partition getRowsOutOfOrder()
          {
 -            ColumnFamily cf = iterator.getColumnFamily().cloneMeShallow(ArrayBackedSortedColumns.factory, false);
 -            cf.addAtom(atom);
 -            while (iterator.hasNext())
 -                cf.addAtom(iterator.next());
 -            return cf;
 +            return rowsOutOfOrder;
 +        }
 +
 +        protected Unfiltered computeNext()
 +        {
 +            if (!iterator.hasNext())
 +                return endOfData();
 +
 +            Unfiltered next = iterator.next();
 +
 +            // If we detect that some rows are out of order we will store and sort the remaining ones to insert them
 +            // in a separate SSTable.
 +            if (previous != null && comparator.compare(next, previous) < 0)
 +            {
 +                rowsOutOfOrder = ImmutableBTreePartition.create(UnfilteredRowIterators.concat(next, iterator), false);
 +                return endOfData();
 +            }
 +            previous = next;
 +            return next;
          }
+     }
+ 
+     /**
 -     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
++     * This iterator converts negative {@link AbstractCell#localDeletionTime()} into {@link AbstractCell#MAX_DELETION_TIME}
+      *
+      * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+      */
 -    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
++    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
+     {
+         /**
+          * The decorated iterator.
+          */
 -        private final OnDiskAtomIterator iterator;
++        private final UnfilteredRowIterator iterator;
+ 
+         private final OutputHandler outputHandler;
+         private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+ 
 -        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
++        public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, OutputHandler outputHandler,
+                                                     NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+         {
+             this.iterator = iterator;
+             this.outputHandler = outputHandler;
+             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+         }
+ 
 -        public ColumnFamily getColumnFamily()
++        public CFMetaData metadata()
+         {
 -            return iterator.getColumnFamily();
++            return iterator.metadata();
+         }
+ 
 -        public DecoratedKey getKey()
++        public boolean isReverseOrder()
+         {
 -            return iterator.getKey();
++            return iterator.isReverseOrder();
+         }
+ 
 -        public void close() throws IOException
++        public PartitionColumns columns()
+         {
 -            iterator.close();
++            return iterator.columns();
++        }
++
++        public DecoratedKey partitionKey()
++        {
++            return iterator.partitionKey();
++        }
++
++        public Row staticRow()
++        {
++            return iterator.staticRow();
+         }
+ 
+         @Override
 -        protected OnDiskAtom computeNext()
++        public boolean isEmpty()
++        {
++            return iterator.isEmpty();
++        }
++
++        public void close()
++        {
++            iterator.close();
++        }
++
++        public DeletionTime partitionLevelDeletion()
++        {
++            return iterator.partitionLevelDeletion();
++        }
++
++        public EncodingStats stats()
++        {
++            return iterator.stats();
++        }
++
++        protected Unfiltered computeNext()
+         {
+             if (!iterator.hasNext())
+                 return endOfData();
+ 
 -            OnDiskAtom next = iterator.next();
++            Unfiltered next = iterator.next();
++            if (!next.isRow())
++                return next;
+ 
 -            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
++            if (hasNegativeLocalExpirationTime((Row) next))
+             {
 -                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
++                outputHandler.debug(String.format("Found row with negative local expiration time: %s", next.toString(metadata(), false)));
+                 negativeLocalExpirationTimeMetrics.fixedRows++;
 -                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
++                return fixNegativeLocalExpirationTime((Row) next);
+             }
+ 
+             return next;
+         }
++
++        private boolean hasNegativeLocalExpirationTime(Row next)
++        {
++            Row row = next;
++            if (row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0)
++            {
++                return true;
++            }
++
++            for (ColumnData cd : row)
++            {
++                if (cd.column().isSimple())
++                {
++                    Cell cell = (Cell)cd;
++                    if (cell.isExpiring() && cell.localDeletionTime() < 0)
++                        return true;
++                }
++                else
++                {
++                    ComplexColumnData complexData = (ComplexColumnData)cd;
++                    for (Cell cell : complexData)
++                    {
++                        if (cell.isExpiring() && cell.localDeletionTime() < 0)
++                            return true;
++                    }
++                }
++            }
++
++            return false;
++        }
 +
++        private Unfiltered fixNegativeLocalExpirationTime(Row row)
++        {
++            Row.Builder builder = HeapAllocator.instance.cloningBTreeRowBuilder();
++            builder.newRow(row.clustering());
++            builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0 ?
++                                              row.primaryKeyLivenessInfo().withUpdatedTimestampAndLocalDeletionTime(row.primaryKeyLivenessInfo().timestamp() + 1, AbstractCell.MAX_DELETION_TIME)
++                                              :row.primaryKeyLivenessInfo());
++            builder.addRowDeletion(row.deletion());
++            for (ColumnData cd : row)
++            {
++                if (cd.column().isSimple())
++                {
++                    Cell cell = (Cell)cd;
++                    builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
++                }
++                else
++                {
++                    ComplexColumnData complexData = (ComplexColumnData)cd;
++                    builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
++                    for (Cell cell : complexData)
++                    {
++                        builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
++                    }
++                }
++            }
++            return builder.build();
++        }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BufferCell.java
index 82ae02c,0000000..df2619c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@@ -1,365 -1,0 +1,370 @@@
 +/*
 + * 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.io.IOException;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.ByteType;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +
 +public class BufferCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnDefinition.regularDef("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
 +
 +    private final long timestamp;
 +    private final int ttl;
 +    private final int localDeletionTime;
 +
 +    private final ByteBuffer value;
 +    private final CellPath path;
 +
 +    public BufferCell(ColumnDefinition column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
 +    {
 +        super(column);
 +        assert column.isComplex() == (path != null);
 +        this.timestamp = timestamp;
 +        this.ttl = ttl;
 +        this.localDeletionTime = localDeletionTime;
 +        this.value = value;
 +        this.path = path;
 +    }
 +
 +    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value)
 +    {
 +        return live(metadata, column, timestamp, value, null);
 +    }
 +
 +    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value, CellPath path)
 +    {
 +        if (metadata.params.defaultTimeToLive != NO_TTL)
 +            return expiring(column, timestamp, metadata.params.defaultTimeToLive, FBUtilities.nowInSeconds(), value, path);
 +
 +        return new BufferCell(column, timestamp, NO_TTL, NO_DELETION_TIME, value, path);
 +    }
 +
 +    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value)
 +    {
 +        return expiring(column, timestamp, ttl, nowInSec, value, null);
 +    }
 +
 +    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
 +    {
 +        assert ttl != NO_TTL;
-         return new BufferCell(column, timestamp, ttl, nowInSec + ttl, value, path);
++        return new BufferCell(column, timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), value, path);
 +    }
 +
 +    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec)
 +    {
 +        return tombstone(column, timestamp, nowInSec, null);
 +    }
 +
 +    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec, CellPath path)
 +    {
 +        return new BufferCell(column, timestamp, NO_TTL, nowInSec, ByteBufferUtil.EMPTY_BYTE_BUFFER, path);
 +    }
 +
 +    public boolean isCounterCell()
 +    {
 +        return !isTombstone() && column.isCounterColumn();
 +    }
 +
 +    public boolean isLive(int nowInSec)
 +    {
 +        return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && nowInSec < localDeletionTime);
 +    }
 +
 +    public boolean isTombstone()
 +    {
 +        return localDeletionTime != NO_DELETION_TIME && ttl == NO_TTL;
 +    }
 +
 +    public boolean isExpiring()
 +    {
 +        return ttl != NO_TTL;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return timestamp;
 +    }
 +
 +    public int ttl()
 +    {
 +        return ttl;
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return localDeletionTime;
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        return value;
 +    }
 +
 +    public CellPath path()
 +    {
 +        return path;
 +    }
 +
 +    public Cell withUpdatedColumn(ColumnDefinition newColumn)
 +    {
 +        return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);
 +    }
 +
++    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return new BufferCell(column, newTimestamp, ttl, newLocalDeletionTime, value, path);
++    }
++
 +    public Cell copy(AbstractAllocator allocator)
 +    {
 +        if (!value.hasRemaining())
 +            return this;
 +
 +        return new BufferCell(column, timestamp, ttl, localDeletionTime, allocator.clone(value), path == null ? null : path.copy(allocator));
 +    }
 +
 +    public Cell markCounterLocalToBeCleared()
 +    {
 +        if (!isCounterCell())
 +            return this;
 +
 +        ByteBuffer marked = CounterContext.instance().markLocalToBeCleared(value());
 +        return marked == value() ? this : new BufferCell(column, timestamp, ttl, localDeletionTime, marked, path);
 +    }
 +
 +    public Cell purge(DeletionPurger purger, int nowInSec)
 +    {
 +        if (!isLive(nowInSec))
 +        {
 +            if (purger.shouldPurge(timestamp, localDeletionTime))
 +                return null;
 +
 +            // We slightly hijack purging to convert expired but not purgeable columns to tombstones. The reason we do that is
 +            // that once a column has expired it is equivalent to a tombstone but actually using a tombstone is more compact since
 +            // we don't keep the column value. The reason we do it here is that 1) it's somewhat related to dealing with tombstones
 +            // so hopefully not too surprising and 2) we want to this and purging at the same places, so it's simpler/more efficient
 +            // to do both here.
 +            if (isExpiring())
 +            {
 +                // Note that as long as the expiring column and the tombstone put together live longer than GC grace seconds,
 +                // we'll fulfil our responsibility to repair. See discussion at
 +                // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
 +                return BufferCell.tombstone(column, timestamp, localDeletionTime - ttl, path).purge(purger, nowInSec);
 +            }
 +        }
 +        return this;
 +    }
 +
 +    public Cell updateAllTimestamp(long newTimestamp)
 +    {
 +        return new BufferCell(column, isTombstone() ? newTimestamp - 1 : newTimestamp, ttl, localDeletionTime, value, path);
 +    }
 +
 +    public int dataSize()
 +    {
 +        return TypeSizes.sizeof(timestamp)
 +             + TypeSizes.sizeof(ttl)
 +             + TypeSizes.sizeof(localDeletionTime)
 +             + value.remaining()
 +             + (path == null ? 0 : path.dataSize());
 +    }
 +
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value) + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
 +    }
 +
 +    /**
 +     * The serialization format for cell is:
 +     *     [ flags ][ timestamp ][ deletion time ][    ttl    ][ path size ][ path ][ value size ][ value ]
 +     *     [   1b  ][ 8b (vint) ][   4b (vint)   ][ 4b (vint) ][ 4b (vint) ][  arb ][  4b (vint) ][  arb  ]
 +     *
 +     * where not all field are always present (in fact, only the [ flags ] are guaranteed to be present). The fields have the following
 +     * meaning:
 +     *   - [ flags ] is the cell flags. It is a byte for which each bit represents a flag whose meaning is explained below (*_MASK constants)
 +     *   - [ timestamp ] is the cell timestamp. Present unless the cell has the USE_TIMESTAMP_MASK.
 +     *   - [ deletion time]: the local deletion time for the cell. Present if either the cell is deleted (IS_DELETED_MASK)
 +     *       or it is expiring (IS_EXPIRING_MASK) but doesn't have the USE_ROW_TTL_MASK.
 +     *   - [ ttl ]: the ttl for the cell. Present if the row is expiring (IS_EXPIRING_MASK) but doesn't have the
 +     *       USE_ROW_TTL_MASK.
 +     *   - [ value size ] is the size of the [ value ] field. It's present unless either the cell has the HAS_EMPTY_VALUE_MASK, or the value
 +     *       for columns of this type have a fixed length.
 +     *   - [ path size ] is the size of the [ path ] field. Present iff this is the cell of a complex column.
 +     *   - [ value ]: the cell value, unless it has the HAS_EMPTY_VALUE_MASK.
 +     *   - [ path ]: the cell path if the column this is a cell of is complex.
 +     */
 +    static class Serializer implements Cell.Serializer
 +    {
 +        private final static int IS_DELETED_MASK             = 0x01; // Whether the cell is a tombstone or not.
 +        private final static int IS_EXPIRING_MASK            = 0x02; // Whether the cell is expiring.
 +        private final static int HAS_EMPTY_VALUE_MASK        = 0x04; // Wether the cell has an empty value. This will be the case for tombstone in particular.
 +        private final static int USE_ROW_TIMESTAMP_MASK      = 0x08; // Wether the cell has the same timestamp than the row this is a cell of.
 +        private final static int USE_ROW_TTL_MASK            = 0x10; // Wether the cell has the same ttl than the row this is a cell of.
 +
 +        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException
 +        {
 +            assert cell != null;
 +            boolean hasValue = cell.value().hasRemaining();
 +            boolean isDeleted = cell.isTombstone();
 +            boolean isExpiring = cell.isExpiring();
 +            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
 +            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
 +            int flags = 0;
 +            if (!hasValue)
 +                flags |= HAS_EMPTY_VALUE_MASK;
 +
 +            if (isDeleted)
 +                flags |= IS_DELETED_MASK;
 +            else if (isExpiring)
 +                flags |= IS_EXPIRING_MASK;
 +
 +            if (useRowTimestamp)
 +                flags |= USE_ROW_TIMESTAMP_MASK;
 +            if (useRowTTL)
 +                flags |= USE_ROW_TTL_MASK;
 +
 +            out.writeByte((byte)flags);
 +
 +            if (!useRowTimestamp)
 +                header.writeTimestamp(cell.timestamp(), out);
 +
 +            if ((isDeleted || isExpiring) && !useRowTTL)
 +                header.writeLocalDeletionTime(cell.localDeletionTime(), out);
 +            if (isExpiring && !useRowTTL)
 +                header.writeTTL(cell.ttl(), out);
 +
 +            if (column.isComplex())
 +                column.cellPathSerializer().serialize(cell.path(), out);
 +
 +            if (hasValue)
 +                header.getType(column).writeValue(cell.value(), out);
 +        }
 +
 +        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException
 +        {
 +            int flags = in.readUnsignedByte();
 +            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
 +            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
 +            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
 +            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
 +            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
 +
 +            long timestamp = useRowTimestamp ? rowLiveness.timestamp() : header.readTimestamp(in);
 +
 +            int localDeletionTime = useRowTTL
 +                                  ? rowLiveness.localExpirationTime()
 +                                  : (isDeleted || isExpiring ? header.readLocalDeletionTime(in) : NO_DELETION_TIME);
 +
 +            int ttl = useRowTTL ? rowLiveness.ttl() : (isExpiring ? header.readTTL(in) : NO_TTL);
 +
 +            CellPath path = column.isComplex()
 +                          ? column.cellPathSerializer().deserialize(in)
 +                          : null;
 +
 +            boolean isCounter = localDeletionTime == NO_DELETION_TIME && column.type.isCounter();
 +
 +            ByteBuffer value = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +            if (hasValue)
 +            {
 +                if (helper.canSkipValue(column) || (path != null && helper.canSkipValue(path)))
 +                {
 +                    header.getType(column).skipValue(in);
 +                }
 +                else
 +                {
 +                    value = header.getType(column).readValue(in, DatabaseDescriptor.getMaxValueSize());
 +                    if (isCounter)
 +                        value = helper.maybeClearCounterValue(value);
 +                }
 +            }
 +
 +            return new BufferCell(column, timestamp, ttl, localDeletionTime, value, path);
 +        }
 +
 +        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header)
 +        {
 +            long size = 1; // flags
 +            boolean hasValue = cell.value().hasRemaining();
 +            boolean isDeleted = cell.isTombstone();
 +            boolean isExpiring = cell.isExpiring();
 +            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
 +            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
 +
 +            if (!useRowTimestamp)
 +                size += header.timestampSerializedSize(cell.timestamp());
 +
 +            if ((isDeleted || isExpiring) && !useRowTTL)
 +                size += header.localDeletionTimeSerializedSize(cell.localDeletionTime());
 +            if (isExpiring && !useRowTTL)
 +                size += header.ttlSerializedSize(cell.ttl());
 +
 +            if (column.isComplex())
 +                size += column.cellPathSerializer().serializedSize(cell.path());
 +
 +            if (hasValue)
 +                size += header.getType(column).writtenLength(cell.value());
 +
 +            return size;
 +        }
 +
 +        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
 +        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException
 +        {
 +            int flags = in.readUnsignedByte();
 +            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
 +            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
 +            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
 +            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
 +            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
 +
 +            if (!useRowTimestamp)
 +                header.skipTimestamp(in);
 +
 +            if (!useRowTTL && (isDeleted || isExpiring))
 +                header.skipLocalDeletionTime(in);
 +
 +            if (!useRowTTL && isExpiring)
 +                header.skipTTL(in);
 +
 +            if (column.isComplex())
 +                column.cellPathSerializer().skip(in);
 +
 +            if (hasValue)
 +                header.getType(column).skipValue(in);
 +
 +            return true;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/Cell.java
index d10cc74,0000000..c69e11f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@@ -1,157 -1,0 +1,166 @@@
 +/*
 + * 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.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Comparator;
 +
++import com.google.common.annotations.VisibleForTesting;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.Attributes;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +
 +/**
 + * A cell is our atomic unit for a single value of a single column.
 + * <p>
 + * A cell always holds at least a timestamp that gives us how the cell reconcile. We then
 + * have 3 main types of cells:
 + *   1) live regular cells: those will also have a value and, if for a complex column, a path.
 + *   2) expiring cells: on top of regular cells, those have a ttl and a local deletion time (when they are expired).
 + *   3) tombstone cells: those won't have value, but they have a local deletion time (when the tombstone was created).
 + */
 +public abstract class Cell extends ColumnData
 +{
 +    public static final int NO_TTL = 0;
 +    public static final int NO_DELETION_TIME = Integer.MAX_VALUE;
++    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
 +
 +    public final static Comparator<Cell> comparator = (c1, c2) ->
 +    {
 +        int cmp = c1.column().compareTo(c2.column());
 +        if (cmp != 0)
 +            return cmp;
 +
 +        Comparator<CellPath> pathComparator = c1.column().cellPathComparator();
 +        return pathComparator == null ? 0 : pathComparator.compare(c1.path(), c2.path());
 +    };
 +
 +    public static final Serializer serializer = new BufferCell.Serializer();
 +
 +    protected Cell(ColumnDefinition column)
 +    {
 +        super(column);
 +    }
 +
 +    /**
 +     * Whether the cell is a counter cell or not.
 +     *
 +     * @return whether the cell is a counter cell or not.
 +     */
 +    public abstract boolean isCounterCell();
 +
 +    /**
 +     * The cell value.
 +     *
 +     * @return the cell value.
 +     */
 +    public abstract ByteBuffer value();
 +
 +    /**
 +     * The cell timestamp.
 +     * <p>
 +     * @return the cell timestamp.
 +     */
 +    public abstract long timestamp();
 +
 +    /**
 +     * The cell ttl.
 +     *
 +     * @return the cell ttl, or {@code NO_TTL} if the cell isn't an expiring one.
 +     */
 +    public abstract int ttl();
 +
 +    /**
 +     * The cell local deletion time.
 +     *
 +     * @return the cell local deletion time, or {@code NO_DELETION_TIME} if the cell is neither
 +     * a tombstone nor an expiring one.
 +     */
 +    public abstract int localDeletionTime();
 +
 +    /**
 +     * Whether the cell is a tombstone or not.
 +     *
 +     * @return whether the cell is a tombstone or not.
 +     */
 +    public abstract boolean isTombstone();
 +
 +    /**
 +     * Whether the cell is an expiring one or not.
 +     * <p>
 +     * Note that this only correspond to whether the cell liveness info
 +     * have a TTL or not, but doesn't tells whether the cell is already expired
 +     * or not. You should use {@link #isLive} for that latter information.
 +     *
 +     * @return whether the cell is an expiring one or not.
 +     */
 +    public abstract boolean isExpiring();
 +
 +    /**
 +     * Whether the cell is live or not given the current time.
 +     *
 +     * @param nowInSec the current time in seconds. This is used to
 +     * decide if an expiring cell is expired or live.
 +     * @return whether the cell is live or not at {@code nowInSec}.
 +     */
 +    public abstract boolean isLive(int nowInSec);
 +
 +    /**
 +     * For cells belonging to complex types (non-frozen collection and UDT), the
 +     * path to the cell.
 +     *
 +     * @return the cell path for cells of complex column, and {@code null} for other cells.
 +     */
 +    public abstract CellPath path();
 +
 +    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
 +
 +    public abstract Cell withUpdatedValue(ByteBuffer newValue);
 +
++    public abstract Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
++
 +    public abstract Cell copy(AbstractAllocator allocator);
 +
 +    @Override
 +    // Overrides super type to provide a more precise return type.
 +    public abstract Cell markCounterLocalToBeCleared();
 +
 +    @Override
 +    // Overrides super type to provide a more precise return type.
 +    public abstract Cell purge(DeletionPurger purger, int nowInSec);
 +
 +    public interface Serializer
 +    {
 +        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException;
 +
 +        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException;
 +
 +        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header);
 +
 +        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
 +        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index e5a50dd,2c9ac4d..cf8e257
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2705,22 -2599,28 +2705,27 @@@ public class StorageService extends Not
          return status.statusCode;
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
 +        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
 +        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, tables);
+     }
+ 
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
 -                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
 -            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
++            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, reinsertOverflowedTTL, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7344ca8,f336bcc..10d47f7
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -262,14 -262,17 +262,17 @@@ public interface StorageServiceMBean ex
       * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
       */
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+ 
      /**
       * Verify (checksums of) the given keyspace.
 -     * If columnFamilies array is empty, all CFs are verified.
 +     * If tableNames array is empty, all CFs are verified.
       *
       * The entire sstable will be read to ensure each cell validates if extendedVerify is true
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 6ad791d,8bdf9dc..2ab0330
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -332,9 -315,9 +332,9 @@@ public class ThriftValidatio
              if (isCommutative)
                  throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
 -            validateColumnData(metadata, key, null, cosc.column);
 +            validateColumnData(metadata, null, cosc.column);
          }
  
          if (cosc.super_column != null)
@@@ -374,11 -357,13 +374,13 @@@
              if (column.ttl <= 0)
                  throw new org.apache.cassandra.exceptions.InvalidRequestException("ttl must be positive");
  
 -            if (column.ttl > ExpiringCell.MAX_TTL)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
 -            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
 +            if (column.ttl > Attributes.MAX_TTL)
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, Attributes.MAX_TTL));
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
          }
          else
          {
 -            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
              // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
              assert column.ttl == 0;
          }
@@@ -450,9 -435,9 +452,9 @@@
      /**
       * Validates the data part of the column (everything in the column object but the name, which is assumed to be valid)
       */
 -    public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
 +    public static void validateColumnData(CFMetaData metadata, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
      {
-         validateTtl(column);
+         validateTtl(metadata, column);
          if (!column.isSetValue())
              throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
          if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 172b505,17bef02..0d3c078
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -235,35 -228,27 +235,35 @@@ public class NodeProbe implements AutoC
  
      public void close() throws IOException
      {
 -        jmxc.close();
 +        try
 +        {
 +            jmxc.close();
 +        }
 +        catch (ConnectException e)
 +        {
 +            // result of 'stopdaemon' command - i.e. if close() call fails, the daemon is shutdown
 +            System.out.println("Cassandra has shutdown.");
 +        }
      }
  
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
 +        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, tables);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables);
 -        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
++        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTL, jobs, keyspaceName, tables);
      }
  
 -    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
      }
  
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
 +        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames);
      }
  
      private void checkJobs(PrintStream out, int jobs)
@@@ -288,19 -267,13 +288,19 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         switch (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables))
 -        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
++        switch (ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTL, jobs, keyspaceName, tables))
          {
 -            failed = true;
 -            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +            case 1:
 +                failed = true;
 +                out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +                break;
 +            case 2:
 +                failed = true;
 +                out.println("Failed marking some sstables compacting in keyspace "+keyspaceName+", check server logs for more information");
 +                break;
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 6076e32,8319014..19af957
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@@ -90,12 -65,13 +90,13 @@@ public class SSTableMetadataViewe
                  {
                      out.printf("Minimum timestamp: %s%n", stats.minTimestamp);
                      out.printf("Maximum timestamp: %s%n", stats.maxTimestamp);
++                    out.printf("SSTable min local deletion time: %s%n", stats.minLocalDeletionTime);
                      out.printf("SSTable max local deletion time: %s%n", stats.maxLocalDeletionTime);
                      out.printf("Compression ratio: %s%n", stats.compressionRatio);
 -                    out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
 +                    out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000) - gcgs));
                      out.printf("SSTable Level: %d%n", stats.sstableLevel);
                      out.printf("Repaired at: %d%n", stats.repairedAt);
 -                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
 -                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
 +                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
                      out.println("Estimated tombstone drop times:");
                      for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 4249430,f5e84c5..4778d72
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -122,7 -129,7 +129,7 @@@ public class StandaloneScrubbe
                      try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
                          txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
 -                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinserOverflowedTTL))
                          {
                              scrubber.scrub();
                          }
@@@ -199,6 -207,7 +206,7 @@@
          public boolean manifestCheckOnly;
          public boolean skipCorrupted;
          public boolean noValidate;
 -        public boolean reinsertOverflowedTTL;
++        public boolean reinserOverflowedTTL;
  
          private Options(String keyspaceName, String cfName)
          {
@@@ -239,6 -248,7 +247,7 @@@
                  opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                  opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                  opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
 -                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
++                opts.reinserOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
  
                  return opts;
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 2345a85,50224a0..ead2fd4
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -48,11 -49,16 +49,16 @@@ public class Scrub extends NodeToolCm
                     description = "Do not validate columns using column validator")
      private boolean noValidation = false;
  
 -    @Option(title = "jobs",
 -            name = {"-j", "--jobs"},
 -            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 -    private int jobs = 2;
 -
+     @Option(title = "reinsert_overflowed_ttl",
+     name = {"r", "--reinsert-overflowed-ttl"},
+     description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+     private boolean reinsertOverflowedTTL = false;
+ 
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
      @Override
      public void execute(NodeProbe probe)
      {
@@@ -63,13 -69,11 +69,13 @@@
          {
              try
              {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, tableNames);
 -                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 -            } catch (IllegalArgumentException e)
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, tableNames);
 +            }
 +            catch (IllegalArgumentException e)
              {
                  throw e;
 -            } catch (Exception e)
 +            }
 +            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during scrubbing", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-CompressionInfo.db
index 0000000,0000000..d759cec
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
index 0000000,0000000..e7a72da
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
index 0000000,0000000..a3c633a
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++203700622

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Index.db
index 0000000,0000000..d742724
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
index 0000000,0000000..faf367b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
index 0000000,0000000..45113dc
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++CompressionInfo.db
++Data.db
++Summary.db
++Filter.db
++Statistics.db
++TOC.txt
++Digest.crc32
++Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-CompressionInfo.db
index 0000000,0000000..1759c09
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
index 0000000,0000000..c1de572
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
index 0000000,0000000..0403b5b
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++82785930

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Index.db
index 0000000,0000000..a0477eb
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
index 0000000,0000000..e9d6577
new file mode 100644
Binary files differ


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


[19/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
index 0000000,0000000..45113dc
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++CompressionInfo.db
++Data.db
++Summary.db
++Filter.db
++Statistics.db
++TOC.txt
++Digest.crc32
++Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-CompressionInfo.db
index 0000000,0000000..b4de068
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
index 0000000,0000000..e96f772
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
index 0000000,0000000..459804b
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++3064924389

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Index.db
index 0000000,0000000..807a27b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
index 0000000,0000000..1ee01e6
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
index 0000000,0000000..f445537
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Summary.db
++TOC.txt
++Filter.db
++Index.db
++Digest.crc32
++CompressionInfo.db
++Data.db
++Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-CompressionInfo.db
index 0000000,0000000..5d22c04
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
index 0000000,0000000..a22a7a3
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
index 0000000,0000000..db7a6c7
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++1803989939

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Index.db
index 0000000,0000000..6397b5e
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
index 0000000,0000000..4ee9294
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
index 0000000,0000000..f445537
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Summary.db
++TOC.txt
++Filter.db
++Index.db
++Digest.crc32
++CompressionInfo.db
++Data.db
++Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 9f375d4,b1eaac1..fc70974
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -5,20 -11,33 +11,33 @@@ import static org.junit.Assert.fail
  
  import org.apache.cassandra.cql3.Attributes;
  import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
 -import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
 -import org.apache.cassandra.db.ExpiringCell;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
+ import org.apache.cassandra.db.Keyspace;
++import org.apache.cassandra.db.rows.AbstractCell;
  import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
  import org.junit.Test;
  
  public class TTLTest extends CQLTester
  {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
 -    public static int MAX_TTL = ExpiringCell.MAX_TTL;
++    public static int MAX_TTL = Attributes.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
  
      @Test
      public void testTTLPerRequestLimit() throws Throwable
      {
          createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
-         // insert
-         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", Attributes.MAX_TTL); // max ttl
-         int ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > Attributes.MAX_TTL - 10);
+         // insert with low TTL should not be denied
 -        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
++        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10);
  
          try
          {
@@@ -41,10 -60,8 +60,8 @@@
          }
          execute("TRUNCATE %s");
  
-         // update
-         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", Attributes.MAX_TTL); // max ttl
-         ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > Attributes.MAX_TTL - 10);
+         // insert with low TTL should not be denied
 -        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
++        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5);
  
          try
          {
@@@ -91,14 -109,297 +108,288 @@@
          {
              assertTrue(e.getCause()
                          .getMessage()
-                         .contains("default_time_to_live must be less than or equal to " + Attributes.MAX_TTL + " (got "
-                                 + (Attributes.MAX_TTL + 1) + ")"));
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
          }
  
-         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + Attributes.MAX_TTL);
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
 -    public void testRejectExpirationDateOverflowPolicy() throws Throwable
++    public void testCapWarnExpirationOverflowPolicy() throws Throwable
++    {
++        // We don't test that the actual warn is logged here, only on dtest
++        testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.CAP);
++    }
++
++    @Test
++    public void testCapNoWarnExpirationOverflowPolicy() throws Throwable
++    {
++        testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.CAP_NOWARN);
++    }
++
++    @Test
++    public void testCapNoWarnExpirationOverflowPolicyDefaultTTL() throws Throwable
++    {
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.policy.CAP_NOWARN;
++        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
 +        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
-         int ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > 10000 - 10); // within 10 second
++        checkTTLIsCapped("i");
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.policy.REJECT;
++    }
++
++    @Test
++    public void testRejectExpirationOverflowPolicy() throws Throwable
+     {
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
++        //ExpirationDateOverflowHandling.expirationDateOverflowPolicy = ExpirationDateOverflowHandling.expirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
 -    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
++    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
 -        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
 -        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
 -        checkTTLIsCapped("i");
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
++        baseTestRecoverOverflowedExpiration(false, false);
++        baseTestRecoverOverflowedExpiration(true, false);
++        baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
 -    @Test
 -    public void testCapExpirationDatePolicyPerRequest() throws Throwable
++    public void testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy policy) throws Throwable
+     {
 -        // Test cap policy
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
++        ExpirationDateOverflowHandling.policy = policy;
+ 
+         // simple column, clustering, flush
 -        baseCapExpirationDateOverflowTest(true, true, true);
++        testCapExpirationDateOverflowPolicy(true, true, true);
+         // simple column, clustering, noflush
 -        baseCapExpirationDateOverflowTest(true, true, false);
++        testCapExpirationDateOverflowPolicy(true, true, false);
+         // simple column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(true, false, true);
++        testCapExpirationDateOverflowPolicy(true, false, true);
+         // simple column, noclustering, noflush
 -        baseCapExpirationDateOverflowTest(true, false, false);
++        testCapExpirationDateOverflowPolicy(true, false, false);
+         // complex column, clustering, flush
 -        baseCapExpirationDateOverflowTest(false, true, true);
++        testCapExpirationDateOverflowPolicy(false, true, true);
+         // complex column, clustering, noflush
 -        baseCapExpirationDateOverflowTest(false, true, false);
++        testCapExpirationDateOverflowPolicy(false, true, false);
+         // complex column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(false, false, true);
++        testCapExpirationDateOverflowPolicy(false, false, true);
+         // complex column, noclustering, noflush
 -        baseCapExpirationDateOverflowTest(false, false, false);
 -        // complex column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(false, false, false);
++        testCapExpirationDateOverflowPolicy(false, false, false);
+ 
+         // Return to previous policy
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
 -    }
 -
 -    @Test
 -    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
 -    {
 -        baseTestRecoverOverflowedExpiration(false, false);
 -        baseTestRecoverOverflowedExpiration(true, false);
 -        baseTestRecoverOverflowedExpiration(true, true);
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
 -    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
++    public void testCapExpirationDateOverflowPolicy(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
 -        if (simple)
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b int, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b int)");
 -        }
 -        else
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b set<text>)");
 -        }
++        createTable(simple, clustering);
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
 -            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
++            execute("INSERT INTO %s (k, a) VALUES (?, ?) USING TTL " + MAX_TTL, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
++    private void createTable(boolean simple, boolean clustering)
++    {
++        if (simple)
++        {
++            if (clustering)
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
++            else
++                createTable("create table %s (k int primary key, a int, b int)");
++        }
++        else
++        {
++            if (clustering)
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
++            else
++                createTable("create table %s (k int primary key, a int, b set<text>)");
++        }
++    }
++
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
 -            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
      }
  
+     /**
 -     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
 -     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
++     * Verify that the computed TTL is equal to the maximum allowed ttl given the
++     * {@link AbstractCell#localDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
 -        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
++        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s WHERE k = 1");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
 -     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
++     * allowed expiration time {@link org.apache.cassandra.db.rows.Cell#MAX_DELETION_TIME} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
 -        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
++        return AbstractCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
+         createTable(simple, clustering);
+ 
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
+         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getSSTables().size());
++        assertEquals(0, cfs.getLiveSSTables().size());
+ 
+         copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
 -            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
++            cfs.scrub(true, false, true, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+             else
+                 assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+             else
+                 assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
+             assertEmpty(execute("SELECT * from %s"));
+         }
+     }
+ 
+     private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).getDirectories().getCFDirectories().iterator().next();
+         File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
+     private static File getTableDir(String table, boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering)
 -    {
 -        if (simple)
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b int, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b int)");
 -        }
 -        else
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b set<text>)");
 -        }
 -    }
 -
 -    private static File getTableDir(boolean simple, boolean clustering)
 -    {
 -        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
 -    }
 -
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 08336a1,9b1ede4..fc2faea
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -113,14 -127,18 +113,14 @@@ public class ScrubTes
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 1);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
  
--        CompactionManager.instance.performScrub(cfs, false, true, 2);
++        CompactionManager.instance.performScrub(cfs, false, true, false, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
      }
  
      @Test
@@@ -617,9 -763,9 +617,9 @@@
                  boolean failure = !scrubs[i];
                  if (failure)
                  { //make sure the next scrub fails
 -                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                    overrideWithGarbage(indexCfs.getLiveSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
                  }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 -                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, false, true, false,0);
                  assertEquals(failure ?
                               CompactionManager.AllSSTableOpStatus.ABORTED :
                               CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
@@@ -629,118 -775,8 +629,118 @@@
  
  
          // check index is still working
 -        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 -        assertNotNull(rows);
 -        assertEquals(numRows / 2, rows.size());
 +        assertOrdered(Util.cmd(cfs).filterOn(colName, Operator.EQ, 1L).build(), numRows / 2);
 +    }
 +
 +    private static SSTableMultiWriter createTestWriter(Descriptor descriptor, long keyCount, CFMetaData metadata, LifecycleTransaction txn)
 +    {
 +        SerializationHeader header = new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
 +        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(0);
 +        return new TestMultiWriter(new TestWriter(descriptor, keyCount, 0, metadata, collector, header, txn));
 +    }
 +
 +    private static class TestMultiWriter extends SimpleSSTableMultiWriter
 +    {
 +        TestMultiWriter(SSTableWriter writer)
 +        {
 +            super(writer);
 +        }
 +    }
 +
 +    /**
 +     * Test writer that allows to write out of order SSTable.
 +     */
 +    private static class TestWriter extends BigTableWriter
 +    {
 +        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata,
 +                   MetadataCollector collector, SerializationHeader header, LifecycleTransaction txn)
 +        {
 +            super(descriptor, keyCount, repairedAt, metadata, collector, header, txn);
 +        }
 +
 +        @Override
 +        protected long beforeAppend(DecoratedKey decoratedKey)
 +        {
 +            return dataFile.position();
 +        }
 +    }
 +
 +    /**
 +     * Tests with invalid sstables (containing duplicate entries in 2.0 and 3.0 storage format),
 +     * that were caused by upgrading from 2.x with duplicate range tombstones.
 +     *
 +     * See CASSANDRA-12144 for details.
 +     */
 +    @Test
 +    public void testFilterOutDuplicates() throws Exception
 +    {
 +        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".cf_with_duplicates_3_0 (a int, b int, c int, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
 +
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("cf_with_duplicates_3_0");
 +
 +        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
 +                                           "Keyspace1",
 +                                           "cf_with_duplicates_3_0");
 +
 +        for (String filename : new String[]{ "mb-3-big-CompressionInfo.db",
 +                                             "mb-3-big-Digest.crc32",
 +                                             "mb-3-big-Index.db",
 +                                             "mb-3-big-Summary.db",
 +                                             "mb-3-big-Data.db",
 +                                             "mb-3-big-Filter.db",
 +                                             "mb-3-big-Statistics.db",
 +                                             "mb-3-big-TOC.txt" })
 +        {
 +            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
 +        }
 +
 +        cfs.loadNewSSTables();
 +
-         cfs.scrub(true, true, true, 1);
++        cfs.scrub(true, true, false, false, false, 1);
 +
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
 +        assertEquals(1, rs.size());
 +        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".cf_with_duplicates_3_0 WHERE a=1 AND b =2", KEYSPACE));
 +        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
 +        assertEquals(0, rs.size());
 +    }
 +
 +    @Test
 +    public void testUpgradeSstablesWithDuplicates() throws Exception
 +    {
 +        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +        String cf = "cf_with_duplicates_2_0";
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".%s (a int, b int, c int, PRIMARY KEY (a, b))", KEYSPACE, cf), ConsistencyLevel.ONE);
 +
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cf);
 +
 +        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
 +                                           "Keyspace1",
 +                                           cf);
 +
 +        for (String filename : new String[]{ "lb-1-big-CompressionInfo.db",
 +                                             "lb-1-big-Data.db",
 +                                             "lb-1-big-Digest.adler32",
 +                                             "lb-1-big-Filter.db",
 +                                             "lb-1-big-Index.db",
 +                                             "lb-1-big-Statistics.db",
 +                                             "lb-1-big-Summary.db",
 +                                             "lb-1-big-TOC.txt" })
 +        {
 +            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
 +        }
 +
 +        cfs.loadNewSSTables();
 +
 +        cfs.sstablesRewrite(true, 1);
 +
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE, cf));
 +        assertEquals(1, rs.size());
 +        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".%s WHERE a=1 AND b =2", KEYSPACE, cf));
 +        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE, cf));
 +        assertEquals(0, rs.size());
      }
  }


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


[25/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LegacyLayout.java
index 2117dd6,0000000..912d591
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@@ -1,2504 -1,0 +1,2509 @@@
 +/*
 + * 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;
 +
 +import java.io.DataInput;
 +import java.io.IOException;
 +import java.io.IOError;
 +import java.nio.ByteBuffer;
 +import java.security.MessageDigest;
 +import java.util.*;
 +
 +import org.apache.cassandra.cql3.SuperColumnCompatibility;
++import org.apache.cassandra.thrift.Column;
 +import org.apache.cassandra.utils.AbstractIterator;
 +import com.google.common.collect.Iterators;
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.PeekingIterator;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.utils.*;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 +
 +/**
 + * Functions to deal with the old format.
 + */
 +public abstract class LegacyLayout
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LegacyLayout.class);
 +
 +    public final static int MAX_CELL_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
 +
 +    public final static int STATIC_PREFIX = 0xFFFF;
 +
 +    public final static int DELETION_MASK        = 0x01;
 +    public final static int EXPIRATION_MASK      = 0x02;
 +    public final static int COUNTER_MASK         = 0x04;
 +    public final static int COUNTER_UPDATE_MASK  = 0x08;
 +    private final static int RANGE_TOMBSTONE_MASK = 0x10;
 +
 +    private LegacyLayout() {}
 +
 +    public static AbstractType<?> makeLegacyComparator(CFMetaData metadata)
 +    {
 +        ClusteringComparator comparator = metadata.comparator;
 +        if (!metadata.isCompound())
 +        {
 +            assert comparator.size() == 1;
 +            return comparator.subtype(0);
 +        }
 +
 +        boolean hasCollections = metadata.hasCollectionColumns() || metadata.hasDroppedCollectionColumns();
 +        List<AbstractType<?>> types = new ArrayList<>(comparator.size() + (metadata.isDense() ? 0 : 1) + (hasCollections ? 1 : 0));
 +
 +        types.addAll(comparator.subtypes());
 +
 +        if (!metadata.isDense())
 +        {
 +            types.add(UTF8Type.instance);
 +
 +            if (hasCollections)
 +            {
 +                Map<ByteBuffer, CollectionType> defined = new HashMap<>();
 +
 +                for (CFMetaData.DroppedColumn def : metadata.getDroppedColumns().values())
 +                    if (def.type instanceof CollectionType && def.type.isMultiCell())
 +                        defined.put(bytes(def.name), (CollectionType) def.type);
 +
 +                for (ColumnDefinition def : metadata.partitionColumns())
 +                    if (def.type instanceof CollectionType && def.type.isMultiCell())
 +                        defined.put(def.name.bytes, (CollectionType) def.type);
 +
 +                types.add(ColumnToCollectionType.getInstance(defined));
 +            }
 +        }
 +        return CompositeType.getInstance(types);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer cellname)
 +    throws UnknownColumnException
 +    {
 +        assert cellname != null;
 +        if (metadata.isSuper())
 +        {
 +            assert superColumnName != null;
 +            return decodeForSuperColumn(metadata, new Clustering(superColumnName), cellname);
 +        }
 +
 +        assert superColumnName == null;
 +        return decodeCellName(metadata, cellname);
 +    }
 +
 +    private static LegacyCellName decodeForSuperColumn(CFMetaData metadata, Clustering clustering, ByteBuffer subcol)
 +    {
 +        ColumnDefinition def = metadata.getColumnDefinition(subcol);
 +        if (def != null)
 +        {
 +            // it's a statically defined subcolumn
 +            return new LegacyCellName(clustering, def, null);
 +        }
 +
 +        def = metadata.compactValueColumn();
 +        assert def != null && def.type instanceof MapType;
 +        return new LegacyCellName(clustering, def, subcol);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException
 +    {
 +        return decodeCellName(metadata, cellname, false);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException
 +    {
 +        Clustering clustering = decodeClustering(metadata, cellname);
 +
 +        if (metadata.isSuper())
 +            return decodeForSuperColumn(metadata, clustering, CompositeType.extractComponent(cellname, 1));
 +
 +        if (metadata.isDense() || (metadata.isCompactTable() && readAllAsDynamic))
 +            return new LegacyCellName(clustering, metadata.compactValueColumn(), null);
 +
 +        ByteBuffer column = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size()) : cellname;
 +        if (column == null)
 +        {
 +            // Tables for composite 2ndary indexes used to be compound but dense, but we've transformed them into regular tables
 +            // (non compact ones) but with no regular column (i.e. we only care about the clustering). So we'll get here
 +            // in that case, and what we want to return is basically a row marker.
 +            if (metadata.partitionColumns().isEmpty())
 +                return new LegacyCellName(clustering, null, null);
 +
 +            // Otherwise, we shouldn't get there
 +            throw new IllegalArgumentException("No column name component found in cell name");
 +        }
 +
 +        // Row marker, this is ok
 +        if (!column.hasRemaining())
 +            return new LegacyCellName(clustering, null, null);
 +
 +        ColumnDefinition def = metadata.getColumnDefinition(column);
 +        if ((def == null) || def.isPrimaryKeyColumn())
 +        {
 +            // If it's a compact table, it means the column is in fact a "dynamic" one
 +            if (metadata.isCompactTable())
 +                return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null);
 +
 +            if (def == null)
 +                throw new UnknownColumnException(metadata, column);
 +            else
 +                throw new IllegalArgumentException("Cannot add primary key column to partition update");
 +        }
 +
 +        ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null;
 +
 +        // Note that because static compact columns are translated to static defs in the new world order, we need to force a static
 +        // clustering if the definition is static (as it might not be in this case).
 +        return new LegacyCellName(def.isStatic() ? Clustering.STATIC_CLUSTERING : clustering, def, collectionElement);
 +    }
 +
 +    public static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, boolean isStart)
 +    {
 +        if (!bound.hasRemaining())
 +            return isStart ? LegacyBound.BOTTOM : LegacyBound.TOP;
 +
 +        if (!metadata.isCompound())
 +        {
 +            // The non compound case is a lot easier, in that there is no EOC nor collection to worry about, so dealing
 +            // with that first.
 +            return new LegacyBound(isStart ? Slice.Bound.inclusiveStartOf(bound) : Slice.Bound.inclusiveEndOf(bound), false, null);
 +        }
 +
 +        int clusteringSize = metadata.comparator.size();
 +
 +        List<ByteBuffer> components = CompositeType.splitName(bound);
 +        byte eoc = CompositeType.lastEOC(bound);
 +
 +        // There can be  more components than the clustering size only in the case this is the bound of a collection
 +        // range tombstone. In which case, there is exactly one more component, and that component is the name of the
 +        // collection being selected/deleted.
 +        assert components.size() <= clusteringSize || (!metadata.isCompactTable() && components.size() == clusteringSize + 1);
 +
 +        ColumnDefinition collectionName = null;
 +        if (components.size() > clusteringSize)
 +            collectionName = metadata.getColumnDefinition(components.remove(clusteringSize));
 +
 +        boolean isInclusive;
 +        if (isStart)
 +        {
 +            isInclusive = eoc <= 0;
 +        }
 +        else
 +        {
 +            isInclusive = eoc >= 0;
 +
 +            // for an end bound, if we only have a prefix of all the components and the final EOC is zero,
 +            // then it should only match up to the prefix but no further, that is, it is an inclusive bound
 +            // of the exact prefix but an exclusive bound of anything beyond it, so adding an empty
 +            // composite value ensures this behavior, see CASSANDRA-12423 for more details
 +            if (eoc == 0 && components.size() < clusteringSize)
 +            {
 +                components.add(ByteBufferUtil.EMPTY_BYTE_BUFFER);
 +                isInclusive = false;
 +            }
 +        }
 +
 +        Slice.Bound.Kind boundKind = Slice.Bound.boundKind(isStart, isInclusive);
 +        Slice.Bound sb = Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
 +        return new LegacyBound(sb, metadata.isCompound() && CompositeType.isStaticName(bound), collectionName);
 +    }
 +
 +    public static ByteBuffer encodeBound(CFMetaData metadata, Slice.Bound bound, boolean isStart)
 +    {
 +        if (bound == Slice.Bound.BOTTOM || bound == Slice.Bound.TOP || metadata.comparator.size() == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        ClusteringPrefix clustering = bound.clustering();
 +
 +        if (!metadata.isCompound())
 +        {
 +            assert clustering.size() == 1;
 +            return clustering.get(0);
 +        }
 +
 +        CompositeType ctype = CompositeType.getInstance(metadata.comparator.subtypes());
 +        CompositeType.Builder builder = ctype.builder();
 +        for (int i = 0; i < clustering.size(); i++)
 +            builder.add(clustering.get(i));
 +
 +        if (isStart)
 +            return bound.isInclusive() ? builder.build() : builder.buildAsEndOfRange();
 +        else
 +            return bound.isInclusive() ? builder.buildAsEndOfRange() : builder.build();
 +    }
 +
 +    public static ByteBuffer encodeCellName(CFMetaData metadata, ClusteringPrefix clustering, ByteBuffer columnName, ByteBuffer collectionElement)
 +    {
 +        boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
 +
 +        if (!metadata.isCompound())
 +        {
 +            if (isStatic)
 +                return columnName;
 +
 +            assert clustering.size() == 1 : "Expected clustering size to be 1, but was " + clustering.size();
 +            return clustering.get(0);
 +        }
 +
 +        // We use comparator.size() rather than clustering.size() because of static clusterings
 +        int clusteringSize = metadata.comparator.size();
 +        int size = clusteringSize + (metadata.isDense() ? 0 : 1) + (collectionElement == null ? 0 : 1);
 +        if (metadata.isSuper())
 +            size = clusteringSize + 1;
 +        ByteBuffer[] values = new ByteBuffer[size];
 +        for (int i = 0; i < clusteringSize; i++)
 +        {
 +            if (isStatic)
 +            {
 +                values[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +                continue;
 +            }
 +
 +            ByteBuffer v = clustering.get(i);
 +            // we can have null (only for dense compound tables for backward compatibility reasons) but that
 +            // means we're done and should stop there as far as building the composite is concerned.
 +            if (v == null)
 +                return CompositeType.build(Arrays.copyOfRange(values, 0, i));
 +
 +            values[i] = v;
 +        }
 +
 +        if (metadata.isSuper())
 +        {
 +            // We need to set the "column" (in thrift terms) name, i.e. the value corresponding to the subcomparator.
 +            // What it is depends if this a cell for a declared "static" column or a "dynamic" column part of the
 +            // super-column internal map.
 +            assert columnName != null; // This should never be null for supercolumns, see decodeForSuperColumn() above
 +            values[clusteringSize] = columnName.equals(SuperColumnCompatibility.SUPER_COLUMN_MAP_COLUMN)
 +                                   ? collectionElement
 +                                   : columnName;
 +        }
 +        else
 +        {
 +            if (!metadata.isDense())
 +                values[clusteringSize] = columnName;
 +            if (collectionElement != null)
 +                values[clusteringSize + 1] = collectionElement;
 +        }
 +
 +        return CompositeType.build(isStatic, values);
 +    }
 +
 +    public static Clustering decodeClustering(CFMetaData metadata, ByteBuffer value)
 +    {
 +        int csize = metadata.comparator.size();
 +        if (csize == 0)
 +            return Clustering.EMPTY;
 +
 +        if (metadata.isCompound() && CompositeType.isStaticName(value))
 +            return Clustering.STATIC_CLUSTERING;
 +
 +        List<ByteBuffer> components = metadata.isCompound()
 +                                    ? CompositeType.splitName(value)
 +                                    : Collections.singletonList(value);
 +
 +        return new Clustering(components.subList(0, Math.min(csize, components.size())).toArray(new ByteBuffer[csize]));
 +    }
 +
 +    public static ByteBuffer encodeClustering(CFMetaData metadata, ClusteringPrefix clustering)
 +    {
 +        if (clustering.size() == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        if (!metadata.isCompound())
 +        {
 +            assert clustering.size() == 1;
 +            return clustering.get(0);
 +        }
 +
 +        ByteBuffer[] values = new ByteBuffer[clustering.size()];
 +        for (int i = 0; i < clustering.size(); i++)
 +            values[i] = clustering.get(i);
 +        return CompositeType.build(values);
 +    }
 +
 +    /**
 +     * The maximum number of cells to include per partition when converting to the old format.
 +     * <p>
 +     * We already apply the limit during the actual query, but for queries that counts cells and not rows (thrift queries
 +     * and distinct queries as far as old nodes are concerned), we may still include a little bit more than requested
 +     * because {@link DataLimits} always include full rows. So if the limit ends in the middle of a queried row, the
 +     * full row will be part of our result. This would confuse old nodes however so we make sure to truncate it to
 +     * what's expected before writting it on the wire.
 +     *
 +     * @param command the read commmand for which to determine the maximum cells per partition. This can be {@code null}
 +     * in which case {@code Integer.MAX_VALUE} is returned.
 +     * @return the maximum number of cells per partition that should be enforced according to the read command if
 +     * post-query limitation are in order (see above). This will be {@code Integer.MAX_VALUE} if no such limits are
 +     * necessary.
 +     */
 +    private static int maxCellsPerPartition(ReadCommand command)
 +    {
 +        if (command == null)
 +            return Integer.MAX_VALUE;
 +
 +        DataLimits limits = command.limits();
 +
 +        // There is 2 types of DISTINCT queries: those that includes only the partition key, and those that include static columns.
 +        // On old nodes, the latter expects the first row in term of CQL count, which is what we already have and there is no additional
 +        // limit to apply. The former however expect only one cell per partition and rely on it (See CASSANDRA-10762).
 +        if (limits.isDistinct())
 +            return command.columnFilter().fetchedColumns().statics.isEmpty() ? 1 : Integer.MAX_VALUE;
 +
 +        switch (limits.kind())
 +        {
 +            case THRIFT_LIMIT:
 +            case SUPER_COLUMN_COUNTING_LIMIT:
 +                return limits.perPartitionCount();
 +            default:
 +                return Integer.MAX_VALUE;
 +        }
 +    }
 +
 +    // For serializing to old wire format
 +    public static LegacyUnfilteredPartition fromUnfilteredRowIterator(ReadCommand command, UnfilteredRowIterator iterator)
 +    {
 +        // we need to extract the range tombstone so materialize the partition. Since this is
 +        // used for the on-wire format, this is not worst than it used to be.
 +        final ImmutableBTreePartition partition = ImmutableBTreePartition.create(iterator);
 +        DeletionInfo info = partition.deletionInfo();
 +        Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> pair = fromRowIterator(partition.metadata(), partition.iterator(), partition.staticRow());
 +
 +        LegacyLayout.LegacyRangeTombstoneList rtl = pair.left;
 +
 +        // Processing the cell iterator results in the LegacyRangeTombstoneList being populated, so we do this
 +        // before we use the LegacyRangeTombstoneList at all
 +        List<LegacyLayout.LegacyCell> cells = Lists.newArrayList(pair.right);
 +
 +        int maxCellsPerPartition = maxCellsPerPartition(command);
 +        if (cells.size() > maxCellsPerPartition)
 +            cells = cells.subList(0, maxCellsPerPartition);
 +
 +        // The LegacyRangeTombstoneList already has range tombstones for the single-row deletions and complex
 +        // deletions.  Go through our normal range tombstones and add then to the LegacyRTL so that the range
 +        // tombstones all get merged and sorted properly.
 +        if (info.hasRanges())
 +        {
 +            Iterator<RangeTombstone> rangeTombstoneIterator = info.rangeIterator(false);
 +            while (rangeTombstoneIterator.hasNext())
 +            {
 +                RangeTombstone rt = rangeTombstoneIterator.next();
 +                Slice slice = rt.deletedSlice();
 +                LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(slice.start(), false, null);
 +                LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(slice.end(), false, null);
 +                rtl.add(start, end, rt.deletionTime().markedForDeleteAt(), rt.deletionTime().localDeletionTime());
 +            }
 +        }
 +
 +        return new LegacyUnfilteredPartition(info.getPartitionDeletion(), rtl, cells);
 +    }
 +
 +    public static void serializeAsLegacyPartition(ReadCommand command, UnfilteredRowIterator partition, DataOutputPlus out, int version) throws IOException
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        out.writeBoolean(true);
 +
 +        LegacyLayout.LegacyUnfilteredPartition legacyPartition = LegacyLayout.fromUnfilteredRowIterator(command, partition);
 +
 +        UUIDSerializer.serializer.serialize(partition.metadata().cfId, out, version);
 +        DeletionTime.serializer.serialize(legacyPartition.partitionDeletion, out);
 +
 +        legacyPartition.rangeTombstones.serialize(out, partition.metadata());
 +
 +        // begin cell serialization
 +        out.writeInt(legacyPartition.cells.size());
 +        for (LegacyLayout.LegacyCell cell : legacyPartition.cells)
 +        {
 +            ByteBufferUtil.writeWithShortLength(cell.name.encode(partition.metadata()), out);
 +            out.writeByte(cell.serializationFlags());
 +            if (cell.isExpiring())
 +            {
 +                out.writeInt(cell.ttl);
 +                out.writeInt(cell.localDeletionTime);
 +            }
 +            else if (cell.isTombstone())
 +            {
 +                out.writeLong(cell.timestamp);
 +                out.writeInt(TypeSizes.sizeof(cell.localDeletionTime));
 +                out.writeInt(cell.localDeletionTime);
 +                continue;
 +            }
 +            else if (cell.isCounterUpdate())
 +            {
 +                out.writeLong(cell.timestamp);
 +                long count = CounterContext.instance().getLocalCount(cell.value);
 +                ByteBufferUtil.writeWithLength(ByteBufferUtil.bytes(count), out);
 +                continue;
 +            }
 +            else if (cell.isCounter())
 +            {
 +                out.writeLong(Long.MIN_VALUE);  // timestampOfLastDelete (not used, and MIN_VALUE is the default)
 +            }
 +
 +            out.writeLong(cell.timestamp);
 +            ByteBufferUtil.writeWithLength(cell.value, out);
 +        }
 +    }
 +
 +    // For the old wire format
 +    // Note: this can return null if an empty partition is serialized!
 +    public static UnfilteredRowIterator deserializeLegacyPartition(DataInputPlus in, int version, SerializationHelper.Flag flag, ByteBuffer key) throws IOException
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        // This is only used in mutation, and mutation have never allowed "null" column families
 +        boolean present = in.readBoolean();
 +        if (!present)
 +            return null;
 +
 +        CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
 +        LegacyDeletionInfo info = LegacyDeletionInfo.deserialize(metadata, in);
 +        int size = in.readInt();
 +        Iterator<LegacyCell> cells = deserializeCells(metadata, in, flag, size);
 +        SerializationHelper helper = new SerializationHelper(metadata, version, flag);
 +        return onWireCellstoUnfilteredRowIterator(metadata, metadata.partitioner.decorateKey(key), info, cells, false, helper);
 +    }
 +
 +    // For the old wire format
 +    public static long serializedSizeAsLegacyPartition(ReadCommand command, UnfilteredRowIterator partition, int version)
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        if (partition.isEmpty())
 +            return TypeSizes.sizeof(false);
 +
 +        long size = TypeSizes.sizeof(true);
 +
 +        LegacyLayout.LegacyUnfilteredPartition legacyPartition = LegacyLayout.fromUnfilteredRowIterator(command, partition);
 +
 +        size += UUIDSerializer.serializer.serializedSize(partition.metadata().cfId, version);
 +        size += DeletionTime.serializer.serializedSize(legacyPartition.partitionDeletion);
 +        size += legacyPartition.rangeTombstones.serializedSize(partition.metadata());
 +
 +        // begin cell serialization
 +        size += TypeSizes.sizeof(legacyPartition.cells.size());
 +        for (LegacyLayout.LegacyCell cell : legacyPartition.cells)
 +        {
 +            size += ByteBufferUtil.serializedSizeWithShortLength(cell.name.encode(partition.metadata()));
 +            size += 1;  // serialization flags
 +            if (cell.isExpiring())
 +            {
 +                size += TypeSizes.sizeof(cell.ttl);
 +                size += TypeSizes.sizeof(cell.localDeletionTime);
 +            }
 +            else if (cell.isTombstone())
 +            {
 +                size += TypeSizes.sizeof(cell.timestamp);
 +                // localDeletionTime replaces cell.value as the body
 +                size += TypeSizes.sizeof(TypeSizes.sizeof(cell.localDeletionTime));
 +                size += TypeSizes.sizeof(cell.localDeletionTime);
 +                continue;
 +            }
 +            else if (cell.isCounterUpdate())
 +            {
 +                size += TypeSizes.sizeof(cell.timestamp);
 +                long count = CounterContext.instance().getLocalCount(cell.value);
 +                size += ByteBufferUtil.serializedSizeWithLength(ByteBufferUtil.bytes(count));
 +                continue;
 +            }
 +            else if (cell.isCounter())
 +            {
 +                size += TypeSizes.sizeof(Long.MIN_VALUE);  // timestampOfLastDelete
 +            }
 +
 +            size += TypeSizes.sizeof(cell.timestamp);
 +            size += ByteBufferUtil.serializedSizeWithLength(cell.value);
 +        }
 +
 +        return size;
 +    }
 +
 +    // For thrift sake
 +    public static UnfilteredRowIterator toUnfilteredRowIterator(CFMetaData metadata,
 +                                                                DecoratedKey key,
 +                                                                LegacyDeletionInfo delInfo,
 +                                                                Iterator<LegacyCell> cells)
 +    {
 +        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
 +        return toUnfilteredRowIterator(metadata, key, delInfo, cells, false, helper);
 +    }
 +
 +    // For deserializing old wire format
 +    public static UnfilteredRowIterator onWireCellstoUnfilteredRowIterator(CFMetaData metadata,
 +                                                                           DecoratedKey key,
 +                                                                           LegacyDeletionInfo delInfo,
 +                                                                           Iterator<LegacyCell> cells,
 +                                                                           boolean reversed,
 +                                                                           SerializationHelper helper)
 +    {
 +
 +        // If the table is a static compact, the "column_metadata" are now internally encoded as
 +        // static. This has already been recognized by decodeCellName, but it means the cells
 +        // provided are not in the expected order (the "static" cells are not necessarily at the front).
 +        // So sort them to make sure toUnfilteredRowIterator works as expected.
 +        // Further, if the query is reversed, then the on-wire format still has cells in non-reversed
 +        // order, but we need to have them reverse in the final UnfilteredRowIterator. So reverse them.
 +        if (metadata.isStaticCompactTable() || reversed)
 +        {
 +            List<LegacyCell> l = new ArrayList<>();
 +            Iterators.addAll(l, cells);
 +            Collections.sort(l, legacyCellComparator(metadata, reversed));
 +            cells = l.iterator();
 +        }
 +
 +        return toUnfilteredRowIterator(metadata, key, delInfo, cells, reversed, helper);
 +    }
 +
 +    private static UnfilteredRowIterator toUnfilteredRowIterator(CFMetaData metadata,
 +                                                                 DecoratedKey key,
 +                                                                 LegacyDeletionInfo delInfo,
 +                                                                 Iterator<LegacyCell> cells,
 +                                                                 boolean reversed,
 +                                                                 SerializationHelper helper)
 +    {
 +        // A reducer that basically does nothing, we know the 2 merged iterators can't have conflicting atoms (since we merge cells with range tombstones).
 +        MergeIterator.Reducer<LegacyAtom, LegacyAtom> reducer = new MergeIterator.Reducer<LegacyAtom, LegacyAtom>()
 +        {
 +            private LegacyAtom atom;
 +
 +            public void reduce(int idx, LegacyAtom current)
 +            {
 +                // We're merging cell with range tombstones, so we should always only have a single atom to reduce.
 +                assert atom == null;
 +                atom = current;
 +            }
 +
 +            protected LegacyAtom getReduced()
 +            {
 +                return atom;
 +            }
 +
 +            protected void onKeyChange()
 +            {
 +                atom = null;
 +            }
 +        };
 +        List<Iterator<LegacyAtom>> iterators = Arrays.asList(asLegacyAtomIterator(cells), asLegacyAtomIterator(delInfo.inRowRangeTombstones()));
 +        PeekingIterator<LegacyAtom> atoms = Iterators.peekingIterator(MergeIterator.get(iterators, legacyAtomComparator(metadata), reducer));
 +
 +        // Check if we have some static
 +        Row staticRow = atoms.hasNext() && atoms.peek().isStatic()
 +                      ? getNextRow(CellGrouper.staticGrouper(metadata, helper), atoms)
 +                      : Rows.EMPTY_STATIC_ROW;
 +
 +        Iterator<Row> rows = convertToRows(new CellGrouper(metadata, helper), atoms);
 +        Iterator<RangeTombstone> ranges = delInfo.deletionInfo.rangeIterator(reversed);
 +        return new RowAndDeletionMergeIterator(metadata,
 +                                               key,
 +                                               delInfo.deletionInfo.getPartitionDeletion(),
 +                                               ColumnFilter.all(metadata),
 +                                               staticRow,
 +                                               reversed,
 +                                               EncodingStats.NO_STATS,
 +                                               rows,
 +                                               ranges,
 +                                               true);
 +    }
 +
 +    public static Row extractStaticColumns(CFMetaData metadata, DataInputPlus in, Columns statics) throws IOException
 +    {
 +        assert !statics.isEmpty();
 +        assert metadata.isCompactTable();
 +
 +        if (metadata.isSuper())
 +            // TODO: there is in practice nothing to do here, but we need to handle the column_metadata for super columns somewhere else
 +            throw new UnsupportedOperationException();
 +
 +        Set<ByteBuffer> columnsToFetch = new HashSet<>(statics.size());
 +        for (ColumnDefinition column : statics)
 +            columnsToFetch.add(column.name.bytes);
 +
 +        Row.Builder builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
 +        builder.newRow(Clustering.STATIC_CLUSTERING);
 +
 +        boolean foundOne = false;
 +        LegacyAtom atom;
 +        while ((atom = readLegacyAtomSkippingUnknownColumn(metadata,in)) != null)
 +        {
 +            if (atom.isCell())
 +            {
 +                LegacyCell cell = atom.asCell();
 +                if (!columnsToFetch.contains(cell.name.encode(metadata)))
 +                    continue;
 +
 +                foundOne = true;
 +                builder.addCell(new BufferCell(cell.name.column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, null));
 +            }
 +            else
 +            {
 +                LegacyRangeTombstone tombstone = atom.asRangeTombstone();
 +                // TODO: we need to track tombstones and potentially ignore cells that are
 +                // shadowed (or even better, replace them by tombstones).
 +                throw new UnsupportedOperationException();
 +            }
 +        }
 +
 +        return foundOne ? builder.build() : Rows.EMPTY_STATIC_ROW;
 +    }
 +
 +    private static LegacyAtom readLegacyAtomSkippingUnknownColumn(CFMetaData metadata, DataInputPlus in)
 +    throws IOException
 +    {
 +        while (true)
 +        {
 +            try
 +            {
 +                return readLegacyAtom(metadata, in, false);
 +            }
 +            catch (UnknownColumnException e)
 +            {
 +                // Simply skip, as the method name implies.
 +            }
 +        }
 +
 +    }
 +
 +    private static Row getNextRow(CellGrouper grouper, PeekingIterator<? extends LegacyAtom> cells)
 +    {
 +        if (!cells.hasNext())
 +            return null;
 +
 +        grouper.reset();
 +        while (cells.hasNext() && grouper.addAtom(cells.peek()))
 +        {
 +            // We've added the cell already in the grouper, so just skip it
 +            cells.next();
 +        }
 +        return grouper.getRow();
 +    }
 +
 +    @SuppressWarnings("unchecked")
 +    private static Iterator<LegacyAtom> asLegacyAtomIterator(Iterator<? extends LegacyAtom> iter)
 +    {
 +        return (Iterator<LegacyAtom>)iter;
 +    }
 +
 +    private static Iterator<Row> convertToRows(final CellGrouper grouper, final PeekingIterator<LegacyAtom> atoms)
 +    {
 +        return new AbstractIterator<Row>()
 +        {
 +            protected Row computeNext()
 +            {
 +                if (!atoms.hasNext())
 +                    return endOfData();
 +
 +                return getNextRow(grouper, atoms);
 +            }
 +        };
 +    }
 +
 +    public static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRowIterator(final RowIterator iterator)
 +    {
 +        return fromRowIterator(iterator.metadata(), iterator, iterator.staticRow());
 +    }
 +
 +    private static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRowIterator(final CFMetaData metadata, final Iterator<Row> iterator, final Row staticRow)
 +    {
 +        LegacyRangeTombstoneList deletions = new LegacyRangeTombstoneList(new LegacyBoundComparator(metadata.comparator), 10);
 +        Iterator<LegacyCell> cells = new AbstractIterator<LegacyCell>()
 +        {
 +            private Iterator<LegacyCell> currentRow = initializeRow();
 +
 +            private Iterator<LegacyCell> initializeRow()
 +            {
 +                if (staticRow == null || staticRow.isEmpty())
 +                    return Collections.<LegacyLayout.LegacyCell>emptyIterator();
 +
 +                Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> row = fromRow(metadata, staticRow);
 +                deletions.addAll(row.left);
 +                return row.right;
 +            }
 +
 +            protected LegacyCell computeNext()
 +            {
 +                while (true)
 +                {
 +                    if (currentRow.hasNext())
 +                        return currentRow.next();
 +
 +                    if (!iterator.hasNext())
 +                        return endOfData();
 +
 +                    Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> row = fromRow(metadata, iterator.next());
 +                    deletions.addAll(row.left);
 +                    currentRow = row.right;
 +                }
 +            }
 +        };
 +
 +        return Pair.create(deletions, cells);
 +    }
 +
 +    private static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRow(final CFMetaData metadata, final Row row)
 +    {
 +        // convert any complex deletions or row deletion into normal range tombstones so that we can build and send a proper RangeTombstoneList
 +        // to legacy nodes
 +        LegacyRangeTombstoneList deletions = new LegacyRangeTombstoneList(new LegacyBoundComparator(metadata.comparator), 10);
 +
 +        if (!row.deletion().isLive())
 +        {
 +            Clustering clustering = row.clustering();
 +            Slice.Bound startBound = Slice.Bound.inclusiveStartOf(clustering);
 +            Slice.Bound endBound = Slice.Bound.inclusiveEndOf(clustering);
 +
 +            LegacyBound start = new LegacyLayout.LegacyBound(startBound, false, null);
 +            LegacyBound end = new LegacyLayout.LegacyBound(endBound, false, null);
 +
 +            deletions.add(start, end, row.deletion().time().markedForDeleteAt(), row.deletion().time().localDeletionTime());
 +        }
 +
 +        for (ColumnData cd : row)
 +        {
 +            ColumnDefinition col = cd.column();
 +            if (col.isSimple())
 +                continue;
 +
 +            DeletionTime delTime = ((ComplexColumnData)cd).complexDeletion();
 +            if (!delTime.isLive())
 +            {
 +                Clustering clustering = row.clustering();
 +
 +                Slice.Bound startBound = Slice.Bound.inclusiveStartOf(clustering);
 +                Slice.Bound endBound = Slice.Bound.inclusiveEndOf(clustering);
 +
 +                LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(startBound, col.isStatic(), col);
 +                LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(endBound, col.isStatic(), col);
 +
 +                deletions.add(start, end, delTime.markedForDeleteAt(), delTime.localDeletionTime());
 +            }
 +        }
 +
 +        Iterator<LegacyCell> cells = new AbstractIterator<LegacyCell>()
 +        {
 +            private final Iterator<Cell> cells = row.cellsInLegacyOrder(metadata, false).iterator();
 +            // we don't have (and shouldn't have) row markers for compact tables.
 +            private boolean hasReturnedRowMarker = metadata.isCompactTable();
 +
 +            protected LegacyCell computeNext()
 +            {
 +                if (!hasReturnedRowMarker)
 +                {
 +                    hasReturnedRowMarker = true;
 +
 +                    // don't include a row marker if there's no timestamp on the primary key; this is the 3.0+ equivalent
 +                    // of a row marker
 +                    if (!row.primaryKeyLivenessInfo().isEmpty())
 +                    {
 +                        LegacyCellName cellName = new LegacyCellName(row.clustering(), null, null);
 +                        LivenessInfo info = row.primaryKeyLivenessInfo();
 +                        return new LegacyCell(info.isExpiring() ? LegacyCell.Kind.EXPIRING : LegacyCell.Kind.REGULAR, cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER, info.timestamp(), info.localExpirationTime(), info.ttl());
 +                    }
 +                }
 +
 +                if (!cells.hasNext())
 +                    return endOfData();
 +
 +                return makeLegacyCell(row.clustering(), cells.next());
 +            }
 +        };
 +        return Pair.create(deletions, cells);
 +    }
 +
 +    private static LegacyCell makeLegacyCell(Clustering clustering, Cell cell)
 +    {
 +        LegacyCell.Kind kind;
 +        if (cell.isCounterCell())
 +            kind = LegacyCell.Kind.COUNTER;
 +        else if (cell.isTombstone())
 +            kind = LegacyCell.Kind.DELETED;
 +        else if (cell.isExpiring())
 +            kind = LegacyCell.Kind.EXPIRING;
 +        else
 +            kind = LegacyCell.Kind.REGULAR;
 +
 +        CellPath path = cell.path();
 +        assert path == null || path.size() == 1;
 +        LegacyCellName name = new LegacyCellName(clustering, cell.column(), path == null ? null : path.get(0));
 +        return new LegacyCell(kind, name, cell.value(), cell.timestamp(), cell.localDeletionTime(), cell.ttl());
 +    }
 +
 +    public static RowIterator toRowIterator(final CFMetaData metadata,
 +                                            final DecoratedKey key,
 +                                            final Iterator<LegacyCell> cells,
 +                                            final int nowInSec)
 +    {
 +        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
 +        return UnfilteredRowIterators.filter(toUnfilteredRowIterator(metadata, key, LegacyDeletionInfo.live(), cells, false, helper), nowInSec);
 +    }
 +
 +    public static Comparator<LegacyCell> legacyCellComparator(CFMetaData metadata)
 +    {
 +        return legacyCellComparator(metadata, false);
 +    }
 +
 +    public static Comparator<LegacyCell> legacyCellComparator(final CFMetaData metadata, final boolean reversed)
 +    {
 +        final Comparator<LegacyCellName> cellNameComparator = legacyCellNameComparator(metadata, reversed);
 +        return new Comparator<LegacyCell>()
 +        {
 +            public int compare(LegacyCell cell1, LegacyCell cell2)
 +            {
 +                LegacyCellName c1 = cell1.name;
 +                LegacyCellName c2 = cell2.name;
 +
 +                int c = cellNameComparator.compare(c1, c2);
 +                if (c != 0)
 +                    return c;
 +
 +                // The actual sorting when the cellname is equal doesn't matter, we just want to make
 +                // sure the cells are not considered equal.
 +                if (cell1.timestamp != cell2.timestamp)
 +                    return cell1.timestamp < cell2.timestamp ? -1 : 1;
 +
 +                if (cell1.localDeletionTime != cell2.localDeletionTime)
 +                    return cell1.localDeletionTime < cell2.localDeletionTime ? -1 : 1;
 +
 +                return cell1.value.compareTo(cell2.value);
 +            }
 +        };
 +    }
 +
 +    // Note that this doesn't exactly compare cells as they were pre-3.0 because within a row they sort columns like
 +    // in 3.0, that is, with simple columns before complex columns. In other words, this comparator makes sure cells
 +    // are in the proper order to convert them to actual 3.0 rows.
 +    public static Comparator<LegacyCellName> legacyCellNameComparator(final CFMetaData metadata, final boolean reversed)
 +    {
 +        return new Comparator<LegacyCellName>()
 +        {
 +            public int compare(LegacyCellName c1, LegacyCellName c2)
 +            {
 +                // Compare clustering first
 +                if (c1.clustering == Clustering.STATIC_CLUSTERING)
 +                {
 +                    if (c2.clustering != Clustering.STATIC_CLUSTERING)
 +                        return -1;
 +                }
 +                else if (c2.clustering == Clustering.STATIC_CLUSTERING)
 +                {
 +                    return 1;
 +                }
 +                else
 +                {
 +                    int c = metadata.comparator.compare(c1.clustering, c2.clustering);
 +                    if (c != 0)
 +                        return reversed ? -c : c;
 +                }
 +
 +                // Note that when reversed, we only care about the clustering being reversed, so it's ok
 +                // not to take reversed into account below.
 +
 +                // Then check the column name
 +                if (c1.column != c2.column)
 +                {
 +                    // A null for the column means it's a row marker
 +                    if (c1.column == null)
 +                        return -1;
 +                    if (c2.column == null)
 +                        return 1;
 +
 +                    assert c1.column.isRegular() || c1.column.isStatic();
 +                    assert c2.column.isRegular() || c2.column.isStatic();
 +                    int cmp = c1.column.compareTo(c2.column);
 +                    if (cmp != 0)
 +                        return cmp;
 +                }
 +
 +                assert (c1.collectionElement == null) == (c2.collectionElement == null);
 +
 +                if (c1.collectionElement != null)
 +                {
 +                    AbstractType<?> colCmp = ((CollectionType)c1.column.type).nameComparator();
 +                    return colCmp.compare(c1.collectionElement, c2.collectionElement);
 +                }
 +                return 0;
 +            }
 +        };
 +    }
 +
 +    private static boolean equalValues(ClusteringPrefix c1, ClusteringPrefix c2, ClusteringComparator comparator)
 +    {
 +        assert c1.size() == c2.size();
 +        for (int i = 0; i < c1.size(); i++)
 +        {
 +            if (comparator.compareComponent(i, c1.get(i), c2.get(i)) != 0)
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    private static Comparator<LegacyAtom> legacyAtomComparator(CFMetaData metadata)
 +    {
 +        return (o1, o2) ->
 +        {
 +            // First we want to compare by clustering, but we have to be careful with range tombstone, because
 +            // we can have collection deletion and we want those to sort properly just before the column they
 +            // delete, not before the whole row.
 +            // We also want to special case static so they sort before any non-static. Note in particular that
 +            // this special casing is important in the case of one of the Atom being Slice.Bound.BOTTOM: we want
 +            // it to sort after the static as we deal with static first in toUnfilteredAtomIterator and having
 +            // Slice.Bound.BOTTOM first would mess that up (note that static deletion is handled through a specific
 +            // static tombstone, see LegacyDeletionInfo.add()).
 +            if (o1.isStatic() != o2.isStatic())
 +                return o1.isStatic() ? -1 : 1;
 +
 +            ClusteringPrefix c1 = o1.clustering();
 +            ClusteringPrefix c2 = o2.clustering();
 +
 +            int clusteringComparison;
 +            if (c1.size() != c2.size() || (o1.isCell() == o2.isCell()) || !equalValues(c1, c2, metadata.comparator))
 +            {
 +                clusteringComparison = metadata.comparator.compare(c1, c2);
 +            }
 +            else
 +            {
 +                // one is a cell and one is a range tombstone, and both have the same prefix size (that is, the
 +                // range tombstone is either a row deletion or a collection deletion).
 +                LegacyRangeTombstone rt = o1.isCell() ? o2.asRangeTombstone() : o1.asRangeTombstone();
 +                clusteringComparison = rt.isCollectionTombstone()
 +                                       ? 0
 +                                       : metadata.comparator.compare(c1, c2);
 +            }
 +
 +            // Note that if both are range tombstones and have the same clustering, then they are equal.
 +            if (clusteringComparison != 0)
 +                return clusteringComparison;
 +
 +            if (o1.isCell())
 +            {
 +                LegacyCell cell1 = o1.asCell();
 +                if (o2.isCell())
 +                {
 +                    LegacyCell cell2 = o2.asCell();
 +                    // Check for row marker cells
 +                    if (cell1.name.column == null)
 +                        return cell2.name.column == null ? 0 : -1;
 +                    return cell2.name.column == null ? 1 : cell1.name.column.compareTo(cell2.name.column);
 +                }
 +
 +                LegacyRangeTombstone rt2 = o2.asRangeTombstone();
 +                assert rt2.isCollectionTombstone(); // otherwise, we shouldn't have got a clustering equality
 +                if (cell1.name.column == null)
 +                    return -1;
 +                int cmp = cell1.name.column.compareTo(rt2.start.collectionName);
 +                // If both are for the same column, then the RT should come first
 +                return cmp == 0 ? 1 : cmp;
 +            }
 +            else
 +            {
 +                assert o2.isCell();
 +                LegacyCell cell2 = o2.asCell();
 +
 +                LegacyRangeTombstone rt1 = o1.asRangeTombstone();
 +                assert rt1.isCollectionTombstone(); // otherwise, we shouldn't have got a clustering equality
 +
 +                if (cell2.name.column == null)
 +                    return 1;
 +
 +                int cmp = rt1.start.collectionName.compareTo(cell2.name.column);
 +                // If both are for the same column, then the RT should come first
 +                return cmp == 0 ? -1 : cmp;
 +            }
 +        };
 +    }
 +
 +    public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, boolean readAllAsDynamic)
 +    throws IOException, UnknownColumnException
 +    {
 +        ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
 +        if (!cellname.hasRemaining())
 +            return null; // END_OF_ROW
 +
 +        try
 +        {
 +            int b = in.readUnsignedByte();
 +            return (b & RANGE_TOMBSTONE_MASK) != 0
 +                   ? readLegacyRangeTombstoneBody(metadata, in, cellname)
 +                   : readLegacyCellBody(metadata, in, cellname, b, SerializationHelper.Flag.LOCAL, readAllAsDynamic);
 +        }
 +        catch (UnknownColumnException e)
 +        {
 +            // We legitimately can get here in 2 cases:
 +            // 1) for system tables, because we've unceremoniously removed columns (without registering them as dropped)
 +            // 2) for dropped columns.
 +            // In any other case, there is a mismatch between the schema and the data, and we complain loudly in
 +            // that case. Note that if we are in a legit case of an unknown column, we want to simply skip that cell,
 +            // but we don't do this here and re-throw the exception because the calling code sometimes has to know
 +            // about this happening. This does mean code calling this method should handle this case properly.
 +            if (!metadata.ksName.equals(SystemKeyspace.NAME) && metadata.getDroppedColumnDefinition(e.columnName) == null)
 +                throw new IllegalStateException(String.format("Got cell for unknown column %s in sstable of %s.%s: " +
 +                                                              "This suggest a problem with the schema which doesn't list " +
 +                                                              "this column. Even if that column was dropped, it should have " +
 +                                                              "been listed as such", metadata.ksName, metadata.cfName, UTF8Type.instance.compose(e.columnName)), e);
 +
 +            throw e;
 +        }
 +    }
 +
 +    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException
 +    {
 +        ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
 +        int b = in.readUnsignedByte();
 +        return readLegacyCellBody(metadata, in, cellname, b, flag, false);
 +    }
 +
 +    public static LegacyCell readLegacyCellBody(CFMetaData metadata, DataInput in, ByteBuffer cellname, int mask, SerializationHelper.Flag flag, boolean readAllAsDynamic)
 +    throws IOException, UnknownColumnException
 +    {
 +        // Note that we want to call decodeCellName only after we've deserialized other parts, since it can throw
 +        // and we want to throw only after having deserialized the full cell.
 +        if ((mask & COUNTER_MASK) != 0)
 +        {
 +            in.readLong(); // timestampOfLastDelete: this has been unused for a long time so we ignore it
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            if (flag == SerializationHelper.Flag.FROM_REMOTE || (flag == SerializationHelper.Flag.LOCAL && CounterContext.instance().shouldClearLocal(value)))
 +                value = CounterContext.instance().clearAllLocal(value);
 +            return new LegacyCell(LegacyCell.Kind.COUNTER, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +        else if ((mask & EXPIRATION_MASK) != 0)
 +        {
 +            int ttl = in.readInt();
 +            int expiration = in.readInt();
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            return new LegacyCell(LegacyCell.Kind.EXPIRING, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, expiration, ttl);
 +        }
 +        else
 +        {
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            LegacyCellName name = decodeCellName(metadata, cellname, readAllAsDynamic);
 +            return (mask & COUNTER_UPDATE_MASK) != 0
 +                ? new LegacyCell(LegacyCell.Kind.COUNTER, name, CounterContext.instance().createUpdate(ByteBufferUtil.toLong(value)), ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
 +                : ((mask & DELETION_MASK) == 0
 +                        ? new LegacyCell(LegacyCell.Kind.REGULAR, name, value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
 +                        : new LegacyCell(LegacyCell.Kind.DELETED, name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ts, ByteBufferUtil.toInt(value), Cell.NO_TTL));
 +        }
 +    }
 +
 +    public static LegacyRangeTombstone readLegacyRangeTombstoneBody(CFMetaData metadata, DataInputPlus in, ByteBuffer boundname) throws IOException
 +    {
 +        LegacyBound min = decodeBound(metadata, boundname, true);
 +        LegacyBound max = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
 +        DeletionTime dt = DeletionTime.serializer.deserialize(in);
 +        return new LegacyRangeTombstone(min, max, dt);
 +    }
 +
 +    public static Iterator<LegacyCell> deserializeCells(final CFMetaData metadata,
 +                                                        final DataInput in,
 +                                                        final SerializationHelper.Flag flag,
 +                                                        final int size)
 +    {
 +        return new AbstractIterator<LegacyCell>()
 +        {
 +            private int i = 0;
 +
 +            protected LegacyCell computeNext()
 +            {
 +                if (i >= size)
 +                    return endOfData();
 +
 +                ++i;
 +                try
 +                {
 +                    return readLegacyCell(metadata, in, flag);
 +                }
 +                catch (UnknownColumnException e)
 +                {
 +                    // We can get there if we read a cell for a dropped column, and if that is the case,
 +                    // then simply ignore the cell is fine. But also not that we ignore if it's the
 +                    // system keyspace because for those table we actually remove columns without registering
 +                    // them in the dropped columns
 +                    if (metadata.ksName.equals(SystemKeyspace.NAME) || metadata.getDroppedColumnDefinition(e.columnName) != null)
 +                        return computeNext();
 +                    else
 +                        throw new IOError(e);
 +                }
 +                catch (IOException e)
 +                {
 +                    throw new IOError(e);
 +                }
 +            }
 +        };
 +    }
 +
 +    public static class CellGrouper
 +    {
 +        /**
 +         * The fake TTL used for expired rows that have been compacted.
 +         */
 +        private static final int FAKE_TTL = 1;
 +
 +        public final CFMetaData metadata;
 +        private final boolean isStatic;
 +        private final SerializationHelper helper;
 +        private final Row.Builder builder;
 +        private Clustering clustering;
 +
 +        private LegacyRangeTombstone rowDeletion;
 +        private LegacyRangeTombstone collectionDeletion;
 +
 +        public CellGrouper(CFMetaData metadata, SerializationHelper helper)
 +        {
 +            this(metadata, helper, false);
 +        }
 +
 +        private CellGrouper(CFMetaData metadata, SerializationHelper helper, boolean isStatic)
 +        {
 +            this.metadata = metadata;
 +            this.isStatic = isStatic;
 +            this.helper = helper;
 +            // We cannot use a sorted builder because we don't have exactly the same ordering in 3.0 and pre-3.0. More precisely, within a row, we
 +            // store all simple columns before the complex ones in 3.0, which we use to sort everything sorted by the column name before. Note however
 +            // that the unsorted builder won't have to reconcile cells, so the exact value we pass for nowInSec doesn't matter.
 +            this.builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
 +        }
 +
 +        public static CellGrouper staticGrouper(CFMetaData metadata, SerializationHelper helper)
 +        {
 +            return new CellGrouper(metadata, helper, true);
 +        }
 +
 +        public void reset()
 +        {
 +            this.clustering = null;
 +            this.rowDeletion = null;
 +            this.collectionDeletion = null;
 +        }
 +
 +        public boolean addAtom(LegacyAtom atom)
 +        {
 +            return atom.isCell()
 +                 ? addCell(atom.asCell())
 +                 : addRangeTombstone(atom.asRangeTombstone());
 +        }
 +
 +        public boolean addCell(LegacyCell cell)
 +        {
 +            if (clustering == null)
 +            {
 +                clustering = cell.name.clustering;
 +                assert !isStatic || clustering == Clustering.STATIC_CLUSTERING;
 +                builder.newRow(clustering);
 +            }
 +            else if (!clustering.equals(cell.name.clustering))
 +            {
 +                return false;
 +            }
 +
 +            // Ignore shadowed cells
 +            if (rowDeletion != null && rowDeletion.deletionTime.deletes(cell.timestamp))
 +                return true;
 +
 +            ColumnDefinition column = cell.name.column;
 +            if (column == null)
 +            {
 +                // It's the row marker
 +                assert !cell.value.hasRemaining();
 +                // In 2.1, the row marker expired cell might have been converted into a deleted one by compaction.
 +                // If we do not set the primary key liveness info for this row and it does not contains any regular columns
 +                // the row will be empty. To avoid that, we reuse the localDeletionTime but use a fake TTL.
 +                // The only time in 2.x that we actually delete a row marker is in 2i tables, so in that case we do
 +                // want to actually propagate the row deletion. (CASSANDRA-13320)
 +                if (!cell.isTombstone())
 +                    builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, cell.ttl, cell.localDeletionTime));
 +                else if (metadata.isIndex())
 +                    builder.addRowDeletion(Row.Deletion.regular(new DeletionTime(cell.timestamp, cell.localDeletionTime)));
 +                else
 +                    builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, FAKE_TTL, cell.localDeletionTime));
 +            }
 +            else
 +            {
 +                if (collectionDeletion != null && collectionDeletion.start.collectionName.name.equals(column.name) && collectionDeletion.deletionTime.deletes(cell.timestamp))
 +                    return true;
 +
 +                if (helper.includes(column))
 +                {
 +                    CellPath path = null;
 +                    if (column.isComplex())
 +                    {
 +                        // Recalling startOfComplexColumn for every cell is a big inefficient, but it's ok in practice
 +                        // and it's simpler. And since 1) this only matter for super column selection in thrift in
 +                        // practice and 2) is only used during upgrade, it's probably worth keeping things simple.
 +                        helper.startOfComplexColumn(column);
 +                        path = cell.name.collectionElement == null ? null : CellPath.create(cell.name.collectionElement);
 +                        if (!helper.includes(path))
 +                            return true;
 +                    }
 +                    Cell c = new BufferCell(column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, path);
 +                    if (!helper.isDropped(c, column.isComplex()))
 +                        builder.addCell(c);
 +                    if (column.isComplex())
 +                    {
 +                        helper.endOfComplexColumn();
 +                    }
 +                }
 +            }
 +            return true;
 +        }
 +
 +        private boolean addRangeTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (tombstone.isRowDeletion(metadata))
 +                return addRowTombstone(tombstone);
 +            else if (tombstone.isCollectionTombstone())
 +                return addCollectionTombstone(tombstone);
 +            else
 +                return addGenericRangeTombstone(tombstone);
 +        }
 +
 +        private boolean addRowTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (clustering != null)
 +            {
 +                // If we're already in the row, there might be a chance that there were two range tombstones
 +                // written, as 2.x storage format does not guarantee just one range tombstone, unlike 3.x.
 +                // We have to make sure that clustering matches, which would mean that tombstone is for the
 +                // same row.
 +                if (rowDeletion != null && clustering.equals(tombstone.start.getAsClustering(metadata)))
 +                {
 +                    // If the tombstone superceeds the previous delete, we discard the previous one
 +                    if (tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
 +                    {
 +                        builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
 +                        rowDeletion = tombstone;
 +                    }
 +                    return true;
 +                }
 +
 +                // If we're already within a row and there was no delete written before that one, it can't be the same one
 +                return false;
 +            }
 +
 +            clustering = tombstone.start.getAsClustering(metadata);
 +            builder.newRow(clustering);
 +            builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
 +            rowDeletion = tombstone;
 +
 +            return true;
 +        }
 +
 +        private boolean addCollectionTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (!helper.includes(tombstone.start.collectionName))
 +                return false; // see CASSANDRA-13109
 +
 +            if (clustering == null)
 +            {
 +                clustering = tombstone.start.getAsClustering(metadata);
 +                builder.newRow(clustering);
 +            }
 +            else if (!clustering.equals(tombstone.start.getAsClustering(metadata)))
 +            {
 +                return false;
 +            }
 +
 +            builder.addComplexDeletion(tombstone.start.collectionName, tombstone.deletionTime);
 +            if (rowDeletion == null || tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
 +                collectionDeletion = tombstone;
 +
 +            return true;
 +        }
 +
 +        private boolean addGenericRangeTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            /*
 +             * We can see a non-collection, non-row deletion in two scenarios:
 +             *
 +             * 1. Most commonly, the tombstone's start bound is bigger than current row's clustering, which means that
 +             *    the current row is over, and we should move on to the next row or RT;
 +             *
 +             * 2. Less commonly, the tombstone's start bound is smaller than current row's clustering, which means that
 +             *    we've crossed an index boundary and are seeing a non-closed RT from the previous block, repeated;
 +             *    we should ignore it and stay in the current row.
 +             *
 +             *  In either case, clustering should be non-null, or we shouldn't have gotten to this method at all
 +             *  However, to be absolutely SURE we're in case two above, we check here.
 +             */
 +            return clustering != null && metadata.comparator.compare(clustering, tombstone.start.bound.clustering()) > 0;
 +        }
 +
 +        public Row getRow()
 +        {
 +            return builder.build();
 +        }
 +    }
 +
 +    public static class LegacyUnfilteredPartition
 +    {
 +        public final DeletionTime partitionDeletion;
 +        public final LegacyRangeTombstoneList rangeTombstones;
 +        public final List<LegacyCell> cells;
 +
 +        private LegacyUnfilteredPartition(DeletionTime partitionDeletion, LegacyRangeTombstoneList rangeTombstones, List<LegacyCell> cells)
 +        {
 +            this.partitionDeletion = partitionDeletion;
 +            this.rangeTombstones = rangeTombstones;
 +            this.cells = cells;
 +        }
 +
 +        public void digest(CFMetaData metadata, MessageDigest digest)
 +        {
 +            for (LegacyCell cell : cells)
 +            {
 +                digest.update(cell.name.encode(metadata).duplicate());
 +
 +                if (cell.isCounter())
 +                    CounterContext.instance().updateDigest(digest, cell.value);
 +                else
 +                    digest.update(cell.value.duplicate());
 +
 +                FBUtilities.updateWithLong(digest, cell.timestamp);
 +                FBUtilities.updateWithByte(digest, cell.serializationFlags());
 +
 +                if (cell.isExpiring())
 +                    FBUtilities.updateWithInt(digest, cell.ttl);
 +
 +                if (cell.isCounter())
 +                {
 +                    // Counters used to have the timestampOfLastDelete field, which we stopped using long ago and has been hard-coded
 +                    // to Long.MIN_VALUE but was still taken into account in 2.2 counter digests (to maintain backward compatibility
 +                    // in the first place).
 +                    FBUtilities.updateWithLong(digest, Long.MIN_VALUE);
 +                }
 +            }
 +
 +            if (partitionDeletion.markedForDeleteAt() != Long.MIN_VALUE)
 +                digest.update(ByteBufferUtil.bytes(partitionDeletion.markedForDeleteAt()));
 +
 +            if (!rangeTombstones.isEmpty())
 +                rangeTombstones.updateDigest(digest);
 +        }
 +    }
 +
 +    public static class LegacyCellName
 +    {
 +        public final Clustering clustering;
 +        public final ColumnDefinition column;
 +        public final ByteBuffer collectionElement;
 +
 +        private LegacyCellName(Clustering clustering, ColumnDefinition column, ByteBuffer collectionElement)
 +        {
 +            this.clustering = clustering;
 +            this.column = column;
 +            this.collectionElement = collectionElement;
 +        }
 +
 +        public ByteBuffer encode(CFMetaData metadata)
 +        {
 +            return encodeCellName(metadata, clustering, column == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : column.name.bytes, collectionElement);
 +        }
 +
 +        public ByteBuffer superColumnSubName()
 +        {
 +            assert collectionElement != null;
 +            return collectionElement;
 +        }
 +
 +        public ByteBuffer superColumnName()
 +        {
 +            return clustering.get(0);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            for (int i = 0; i < clustering.size(); i++)
 +                sb.append(i > 0 ? ":" : "").append(clustering.get(i) == null ? "null" : ByteBufferUtil.bytesToHex(clustering.get(i)));
 +            return String.format("Cellname(clustering=%s, column=%s, collElt=%s)", sb.toString(), column == null ? "null" : column.name, collectionElement == null ? "null" : ByteBufferUtil.bytesToHex(collectionElement));
 +        }
 +    }
 +
 +    public static class LegacyBound
 +    {
 +        public static final LegacyBound BOTTOM = new LegacyBound(Slice.Bound.BOTTOM, false, null);
 +        public static final LegacyBound TOP = new LegacyBound(Slice.Bound.TOP, false, null);
 +
 +        public final Slice.Bound bound;
 +        public final boolean isStatic;
 +        public final ColumnDefinition collectionName;
 +
 +        public LegacyBound(Slice.Bound bound, boolean isStatic, ColumnDefinition collectionName)
 +        {
 +            this.bound = bound;
 +            this.isStatic = isStatic;
 +            this.collectionName = collectionName;
 +        }
 +
 +        public Clustering getAsClustering(CFMetaData metadata)
 +        {
 +            if (isStatic)
 +                return Clustering.STATIC_CLUSTERING;
 +
 +            assert bound.size() == metadata.comparator.size();
 +            ByteBuffer[] values = new ByteBuffer[bound.size()];
 +            for (int i = 0; i < bound.size(); i++)
 +                values[i] = bound.get(i);
 +            return new Clustering(values);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            sb.append(bound.kind()).append('(');
 +            for (int i = 0; i < bound.size(); i++)
 +                sb.append(i > 0 ? ":" : "").append(bound.get(i) == null ? "null" : ByteBufferUtil.bytesToHex(bound.get(i)));
 +            sb.append(')');
 +            return String.format("Bound(%s, collection=%s)", sb.toString(), collectionName == null ? "null" : collectionName.name);
 +        }
 +    }
 +
 +    public interface LegacyAtom
 +    {
 +        public boolean isCell();
 +
 +        public ClusteringPrefix clustering();
 +        public boolean isStatic();
 +
 +        public LegacyCell asCell();
 +        public LegacyRangeTombstone asRangeTombstone();
 +    }
 +
 +    /**
 +     * A legacy cell.
 +     * <p>
 +     * This is used as a temporary object to facilitate dealing with the legacy format, this
 +     * is not meant to be optimal.
 +     */
 +    public static class LegacyCell implements LegacyAtom
 +    {
 +        private final static int DELETION_MASK        = 0x01;
 +        private final static int EXPIRATION_MASK      = 0x02;
 +        private final static int COUNTER_MASK         = 0x04;
 +        private final static int COUNTER_UPDATE_MASK  = 0x08;
 +        private final static int RANGE_TOMBSTONE_MASK = 0x10;
 +
 +        public enum Kind { REGULAR, EXPIRING, DELETED, COUNTER }
 +
 +        public final Kind kind;
 +
 +        public final LegacyCellName name;
 +        public final ByteBuffer value;
 +
 +        public final long timestamp;
 +        public final int localDeletionTime;
 +        public final int ttl;
 +
 +        private LegacyCell(Kind kind, LegacyCellName name, ByteBuffer value, long timestamp, int localDeletionTime, int ttl)
 +        {
 +            this.kind = kind;
 +            this.name = name;
 +            this.value = value;
 +            this.timestamp = timestamp;
 +            this.localDeletionTime = localDeletionTime;
 +            this.ttl = ttl;
 +        }
 +
 +        public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp)
 +        throws UnknownColumnException
 +        {
 +            return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +
 +        public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp, int ttl, int nowInSec)
 +        throws UnknownColumnException
 +        {
-             return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, nowInSec + ttl, ttl);
++            /*
++             * CASSANDRA-14092: Max expiration date capping is maybe performed here, expiration overflow policy application
++             * is done at {@link org.apache.cassandra.thrift.ThriftValidation#validateTtl(CFMetaData, Column)}
++             */
++            return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), ttl);
 +        }
 +
 +        public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
 +        throws UnknownColumnException
 +        {
 +            return new LegacyCell(Kind.DELETED, decodeCellName(metadata, superColumnName, name), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, nowInSec, LivenessInfo.NO_TTL);
 +        }
 +
 +        public static LegacyCell counterUpdate(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long value)
 +        throws UnknownColumnException
 +        {
 +            // See UpdateParameters.addCounter() for more details on this
 +            ByteBuffer counterValue = CounterContext.instance().createUpdate(value);
 +            return counter(decodeCellName(metadata, superColumnName, name), counterValue);
 +        }
 +
 +        public static LegacyCell counter(LegacyCellName name, ByteBuffer value)
 +        {
 +            return new LegacyCell(Kind.COUNTER, name, value, FBUtilities.timestampMicros(), Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +
 +        public byte serializationFlags()
 +        {
 +            if (isExpiring())
 +                return EXPIRATION_MASK;
 +            if (isTombstone())
 +                return DELETION_MASK;
 +            if (isCounterUpdate())
 +                return COUNTER_UPDATE_MASK;
 +            if (isCounter())
 +                return COUNTER_MASK;
 +            return 0;
 +        }
 +
 +        public boolean isCounterUpdate()
 +        {
 +            // See UpdateParameters.addCounter() for more details on this
 +            return isCounter() && CounterContext.instance().isUpdate(value);
 +        }
 +
 +        public ClusteringPrefix clustering()
 +        {
 +            return name.clustering;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return name.clustering == Clustering.STATIC_CLUSTERING;
 +        }
 +
 +        public boolean isCell()
 +        {
 +            return true;
 +        }
 +
 +        public LegacyCell asCell()
 +        {
 +            return this;
 +        }
 +
 +        public LegacyRangeTombstone asRangeTombstone()
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        public boolean isCounter()
 +        {
 +            return kind == Kind.COUNTER;
 +        }
 +
 +        public boolean isExpiring()
 +        {
 +            return kind == Kind.EXPIRING;
 +        }
 +
 +        public boolean isTombstone()
 +        {
 +            return kind == Kind.DELETED;
 +        }
 +
 +        public boolean isLive(int nowInSec)
 +        {
 +            if (isTombstone())
 +                return false;
 +
 +            return !isExpiring() || nowInSec < localDeletionTime;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("LegacyCell(%s, name=%s, v=%s, ts=%s, ldt=%s, ttl=%s)", kind, name, ByteBufferUtil.bytesToHex(value), timestamp, localDeletionTime, ttl);
 +        }
 +    }
 +
 +    /**
 +     * A legacy range tombstone.
 +     * <p>
 +     * This is used as a temporary object to facilitate dealing with the legacy format, this
 +     * is not meant to be optimal.
 +     */
 +    public static class LegacyRangeTombstone implements LegacyAtom
 +    {
 +        public final LegacyBound start;
 +        public final LegacyBound stop;
 +        public final DeletionTime deletionTime;
 +
 +        public LegacyRangeTombstone(LegacyBound start, LegacyBound stop, DeletionTime deletionTime)
 +        {
 +            // Because of the way RangeTombstoneList work, we can have a tombstone where only one of
 +            // the bound has a collectionName. That happens if we have a big tombstone A (spanning one
 +            // or multiple rows) and a collection tombstone B. In that case, RangeTombstoneList will
 +            // split this into 3 RTs: the first one from the beginning of A to the beginning of B,
 +            // then B, then a third one from the end of B to the end of A. To make this simpler, if
 +            // we detect that case we transform the 1st and 3rd tombstone so they don't end in the middle
 +            // of a row (which is still correct).
 +            if ((start.collectionName == null) != (stop.collectionName == null))
 +            {
 +                if (start.collectionName == null)
 +                    stop = new LegacyBound(stop.bound, stop.isStatic, null);
 +                else
 +                    start = new LegacyBound(start.bound, start.isStatic, null);
 +            }
 +            else if (!Objects.equals(start.collectionName, stop.collectionName))
 +            {
 +                // We're in the similar but slightly more complex case where on top of the big tombstone
 +                // A, we have 2 (or more) collection tombstones B and C within A. So we also end up with
 +                // a tombstone that goes between the end of B and the start of C.
 +                start = new LegacyBound(start.bound, start.isStatic, null);
 +                stop = new LegacyBound(stop.bound, stop.isStatic, null);
 +            }
 +
 +            this.start = start;
 +            this.stop = stop;
 +            this.deletionTime = deletionTime;
 +        }
 +
 +        public ClusteringPrefix clustering()
 +        {
 +            return start.bound;
 +        }
 +
 +        public LegacyRangeTombstone withNewStart(LegacyBound newStart)
 +        {
 +            return new LegacyRangeTombstone(newStart, stop, deletionTime);
 +        }
 +
 +        public LegacyRangeTombstone withNewEnd(LegacyBound newStop)
 +        {
 +            return new LegacyRangeTombstone(start, newStop, deletionTime);
 +        }
 +
 +        public boolean isCell()
 +        {
 +            return false;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return start.isStatic || stop.isStatic;
 +        }
 +
 +        public LegacyCell asCell()
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        public LegacyRangeTombstone asRangeTombstone()
 +        {
 +            return this;
 +        }
 +
 +        public boolean isCollectionTombstone()
 +        {
 +            return start.collectionName != null;
 +        }
 +
 +        public boolean isRowDeletion(CFMetaData metadata)
 +        {
 +            if (start.collectionName != null
 +                || stop.collectionName != null
 +                || start.bound.size() != metadata.comparator.size()
 +                || stop.bound.size() != metadata.comparator.size())
 +                return false;
 +
 +            for (int i = 0; i < start.bound.size(); i++)
 +                if (!Objects.equals(start.bound.get(i), stop.bound.get(i)))
 +                    return false;
 +            return true;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("RT(%s-%s, %s)", start, stop, deletionTime);
 +        }
 +    }
 +
 +    public static class LegacyDeletionInfo
 +    {
 +        public final MutableDeletionInfo deletionInfo;
 +        public final List<LegacyRangeTombstone> inRowTombstones = new ArrayList<>();
 +
 +        private LegacyDeletionInfo(MutableDeletionInfo deletionInfo)
 +        {
 +            this.deletionInfo = deletionInfo;
 +        }
 +
 +        public static LegacyDeletionInfo live()
 +        {
 +            return new LegacyDeletionInfo(MutableDeletionInfo.live());
 +        }
 +
 +        public void add(DeletionTime topLevel)
 +        {
 +            deletionInfo.add(topLevel);
 +        }
 +
 +        private static Slice.Bound staticBound(CFMetaData metadata, boolean isStart)
 +        {
 +            // In pre-3.0 nodes, static row started by a clustering with all empty values so we
 +            // preserve that here. Note that in practice, it doesn't really matter since the rest
 +            // of the code will ignore the bound for RT that have their static flag set.
 +            ByteBuffer[] values = new ByteBuffer[metadata.comparator.size()];
 +            for (int i = 0; i < values.length; i++)
 +                values[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +            return isStart
 +                 ? Slice.Bound.inclusiveStartOf(values)
 +                 : Slice.Bound.inclusiveEndOf(values);
 +        }
 +
 +        public void add(CFMetaData metadata, LegacyRangeTombstone tombstone)
 +        {
 +            if (metadata.hasStaticColumns())
 +            {
 +                /*
 +                 * For table having static columns we have to deal with the following cases:
 +                 *  1. the end of the tombstone is static (in which case either the start is static or is BOTTOM, which is the same
 +                 *     for our consideration). This mean that either the range only delete the static row, or that it's a collection
 +                 *     tombstone of a static collection. In both case, we just add the tombstone to the inRowTombstones.
 +                 *  2. only the start is static. There is then 2 subcase: either the start is inclusive, and that mean we include the
 +                 *     static row and more (so we add an inRowTombstone for the static and deal with the rest normally). Or the start
 +                 *     is exclusive, and that means we explicitely exclude the static (in which case we can just add the tombstone
 +                 *     as if it started at BOTTOM).
 +                 *  3. none of the bound are static but the start is BOTTOM. This means we intended to delete the static row so we
 +                 *     need to add it to the inRowTombstones (and otherwise handle the range normally).
 +                 */
 +                if (tombstone.stop.isStatic)
 +                {
 +                    // If the start is BOTTOM, we replace it by the beginning of the starting row so as to not confuse the
 +                    // RangeTombstone.isRowDeletion() method
 +                    if (tombstone.start == LegacyBound.BOTTOM)
 +                        tombstone = tombstone.withNewStart(new LegacyBound(staticBound(metadata, true), true, null));
 +                    inRowTombstones.add(tombstone);
 +                    return;
 +                }
 +
 +                if (tombstone.start.isStatic)
 +                {
 +                    if (tombstone.start.bound.isInclusive())
 +                        inRowTombstones.add(tombstone.withNewEnd(new LegacyBound(staticBound(metadata, false), true, null)));
 +
 +                    tombstone = tombstone.withNewStart(LegacyBound.BOTTOM);
 +                }
 +                else if (tombstone.start == LegacyBound.BOTTOM)
 +                {
 +                    inRowTombstones.add(new LegacyRangeTombstone(new LegacyBound(staticBound(metadata, true), true, null),
 +                                                                 new LegacyBound(staticBound(metadata, false), true, null),
 +                                                                 tombstone.deletionTime));
 +                }
 +            }
 +
 +            if (tombstone.isCollectionTombstone() || tombstone.isRowDeletion(metadata))
 +                inRowTombstones.add(tombstone);
 +            else
 +                add(metadata, new RangeTombstone(Slice.make(tombstone.start.bound, tombstone.stop.bound), tombstone.deletionTime));
 +        }
 +
 +        public void add(CFMetaData metadata, RangeTombstone tombstone)
 +        {
 +            deletionInfo.add(tombstone, metadata.comparator);
 +        }
 +
 +        public Iterator<LegacyRangeTombstone> inRowRangeTombstones()
 +        {
 +            return inRowTombstones.iterator();
 +        }
 +
 +        public static LegacyDeletionInfo deserialize(CFMetaData metadata, DataInputPlus in) throws IOException
 +        {
 +            DeletionTime topLevel = DeletionTime.serializer.deserialize(in);
 +
 +            int rangeCount = in.readInt();
 +            if (rangeCount == 0)
 +                return new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
 +
 +            LegacyDeletionInfo delInfo = new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
 +            for (int i = 0; i < rangeCount; i++)
 +            {
 +                LegacyBound start = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
 +                LegacyBound end = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
 +                int delTime =  in.readInt();
 +                long markedAt = in.readLong();
 +
 +                delInfo.add(metadata, new LegacyRangeTombstone(start, end, new DeletionTime(markedAt, delTime)));
 +            }
 +            return delInfo;
 +        }
 +    }
 +
 +    /**
 +     * A helper class for LegacyRangeTombstoneList.  This replaces the Comparator<Composite> that RTL used before 3.0.
 +     */
 +    private static class LegacyBoundComparator implements Comparator<LegacyBound>
 +    {
 +        ClusteringComparator clusteringComparator;
 +
 +        public LegacyBoundComparator(ClusteringComparator clusteringComparator)
 +        {
 +            this.clusteringComparator = clusteringComparator;
 +        }
 +
 +        public int compare(LegacyBound a, LegacyBound b)
 +        {
 +            // In the legacy sorting, BOTTOM comes before anything else
 +            if (a == LegacyBound.BOTTOM)
 +                return b == LegacyBound.BOTTOM ? 0 : -1;
 +            if (b == LegacyBound.BOTTOM)
 +                return 1;
 +
 +            // Excluding BOTTOM, statics are always before anything else.
 +            if (a.isStatic != b.isStatic)
 +                return a.isStatic ? -1 : 1;
 +
 +            // We have to be careful with bound comparison because of collections. Namely, if the 2 bounds represent the
 +            // same prefix, then we should take the collectionName into account before taking the bounds kind
 +            // (ClusteringPrefix.Kind). This means we can't really call ClusteringComparator.compare() directly.
 +            // For instance, if
 +            //    a is (bound=INCL_START_BOUND('x'), collectionName='d')
 +            //    b is (bound=INCL_END_BOUND('x'),   collectionName='c')
 +            // Ten b < a since the element 'c' of collection 'x' comes before element 'd', but calling
 +            // clusteringComparator.compare(a.bound, b.bound) returns -1.
 +            // See CASSANDRA-13125 for details.
 +            int sa = a.bound.size();
 +            int sb = b.bound.size();
 +            for (int i = 0; i < Math.min(sa, sb); i++)
 +            {
 +                int cmp = clusteringComparator.compareComponent(i, a.bound.get(i), b.bound.get(i));
 +                if (cmp != 0)
 +                    return cmp;
 +            }
 +
 +            if (sa != sb)
 +                return sa < sb ? a.bound.kind().comparedToClustering : -b.bound.kind().comparedToClustering;
 +
 +            // Both bound represent the same prefix, compare the collection names
 +            // If one has a collection name and the other doesn't, the other comes before as it points to the beginning of the row.
 +            if ((a.collectionName == null) != (b.collectionName == null))
 +                return a.collectionName == null ? -1 : 1;
 +
 +            // If they both have a collection, compare that first
 +            if (a.collectionName != null)
 +            {
 +                int cmp = UTF8Type.instance.compare(a.collectionName.name.bytes, b.collectionName.name.bytes);
 +                if (cmp != 0)
 +                    return cmp;
 +            }
 +
 +            // Lastly, if everything so far is equal, compare their clustering kind
 +            return ClusteringPrefix.Kind.compare(a.bound.kind(), b.bound.kind());
 +        }
 +    }
 +
 +    /**
 +     * Almost an entire copy of RangeTombstoneList from C* 2.1.  The main difference is that LegacyBoundComparator
 +     * is used in place of Comparator<Composite> (because Composite doesn't exist any more).
 +     *
 +     * This class is needed to allow us to convert single-row deletions and complex deletions into range tombstones
 +     * and properly merge them into the normal set of range tombstones.
 +     */
 +    public static class LegacyRangeTombstoneList
 +    {
 +        private final LegacyBoundComparator comparator;
 +
 +        // Note: we don't want to use a List for the markedAts and delTimes to avoid boxin

<TRUNCATED>

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


[23/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
index 0000000,0000000..45113dc
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++CompressionInfo.db
++Data.db
++Summary.db
++Filter.db
++Statistics.db
++TOC.txt
++Digest.crc32
++Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-CompressionInfo.db
index 0000000,0000000..b4de068
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
index 0000000,0000000..e96f772
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
index 0000000,0000000..459804b
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++3064924389

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Index.db
index 0000000,0000000..807a27b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
index 0000000,0000000..1ee01e6
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
index 0000000,0000000..f445537
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Summary.db
++TOC.txt
++Filter.db
++Index.db
++Digest.crc32
++CompressionInfo.db
++Data.db
++Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-CompressionInfo.db
index 0000000,0000000..5d22c04
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
index 0000000,0000000..a22a7a3
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
index 0000000,0000000..db7a6c7
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++1803989939

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Index.db
index 0000000,0000000..6397b5e
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
index 0000000,0000000..4ee9294
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
index 0000000,0000000..f445537
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Summary.db
++TOC.txt
++Filter.db
++Index.db
++Digest.crc32
++CompressionInfo.db
++Data.db
++Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 9f375d4,b1eaac1..fc70974
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -5,20 -11,33 +11,33 @@@ import static org.junit.Assert.fail
  
  import org.apache.cassandra.cql3.Attributes;
  import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
 -import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
 -import org.apache.cassandra.db.ExpiringCell;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
+ import org.apache.cassandra.db.Keyspace;
++import org.apache.cassandra.db.rows.AbstractCell;
  import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
  import org.junit.Test;
  
  public class TTLTest extends CQLTester
  {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
 -    public static int MAX_TTL = ExpiringCell.MAX_TTL;
++    public static int MAX_TTL = Attributes.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
  
      @Test
      public void testTTLPerRequestLimit() throws Throwable
      {
          createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
-         // insert
-         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", Attributes.MAX_TTL); // max ttl
-         int ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > Attributes.MAX_TTL - 10);
+         // insert with low TTL should not be denied
 -        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
++        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10);
  
          try
          {
@@@ -41,10 -60,8 +60,8 @@@
          }
          execute("TRUNCATE %s");
  
-         // update
-         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", Attributes.MAX_TTL); // max ttl
-         ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > Attributes.MAX_TTL - 10);
+         // insert with low TTL should not be denied
 -        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
++        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5);
  
          try
          {
@@@ -91,14 -109,297 +108,288 @@@
          {
              assertTrue(e.getCause()
                          .getMessage()
-                         .contains("default_time_to_live must be less than or equal to " + Attributes.MAX_TTL + " (got "
-                                 + (Attributes.MAX_TTL + 1) + ")"));
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
          }
  
-         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + Attributes.MAX_TTL);
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
 -    public void testRejectExpirationDateOverflowPolicy() throws Throwable
++    public void testCapWarnExpirationOverflowPolicy() throws Throwable
++    {
++        // We don't test that the actual warn is logged here, only on dtest
++        testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.CAP);
++    }
++
++    @Test
++    public void testCapNoWarnExpirationOverflowPolicy() throws Throwable
++    {
++        testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.CAP_NOWARN);
++    }
++
++    @Test
++    public void testCapNoWarnExpirationOverflowPolicyDefaultTTL() throws Throwable
++    {
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.policy.CAP_NOWARN;
++        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
 +        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
-         int ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > 10000 - 10); // within 10 second
++        checkTTLIsCapped("i");
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.policy.REJECT;
++    }
++
++    @Test
++    public void testRejectExpirationOverflowPolicy() throws Throwable
+     {
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
++        //ExpirationDateOverflowHandling.expirationDateOverflowPolicy = ExpirationDateOverflowHandling.expirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
 -    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
++    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
 -        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
 -        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
 -        checkTTLIsCapped("i");
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
++        baseTestRecoverOverflowedExpiration(false, false);
++        baseTestRecoverOverflowedExpiration(true, false);
++        baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
 -    @Test
 -    public void testCapExpirationDatePolicyPerRequest() throws Throwable
++    public void testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy policy) throws Throwable
+     {
 -        // Test cap policy
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
++        ExpirationDateOverflowHandling.policy = policy;
+ 
+         // simple column, clustering, flush
 -        baseCapExpirationDateOverflowTest(true, true, true);
++        testCapExpirationDateOverflowPolicy(true, true, true);
+         // simple column, clustering, noflush
 -        baseCapExpirationDateOverflowTest(true, true, false);
++        testCapExpirationDateOverflowPolicy(true, true, false);
+         // simple column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(true, false, true);
++        testCapExpirationDateOverflowPolicy(true, false, true);
+         // simple column, noclustering, noflush
 -        baseCapExpirationDateOverflowTest(true, false, false);
++        testCapExpirationDateOverflowPolicy(true, false, false);
+         // complex column, clustering, flush
 -        baseCapExpirationDateOverflowTest(false, true, true);
++        testCapExpirationDateOverflowPolicy(false, true, true);
+         // complex column, clustering, noflush
 -        baseCapExpirationDateOverflowTest(false, true, false);
++        testCapExpirationDateOverflowPolicy(false, true, false);
+         // complex column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(false, false, true);
++        testCapExpirationDateOverflowPolicy(false, false, true);
+         // complex column, noclustering, noflush
 -        baseCapExpirationDateOverflowTest(false, false, false);
 -        // complex column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(false, false, false);
++        testCapExpirationDateOverflowPolicy(false, false, false);
+ 
+         // Return to previous policy
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
 -    }
 -
 -    @Test
 -    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
 -    {
 -        baseTestRecoverOverflowedExpiration(false, false);
 -        baseTestRecoverOverflowedExpiration(true, false);
 -        baseTestRecoverOverflowedExpiration(true, true);
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
 -    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
++    public void testCapExpirationDateOverflowPolicy(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
 -        if (simple)
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b int, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b int)");
 -        }
 -        else
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b set<text>)");
 -        }
++        createTable(simple, clustering);
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
 -            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
++            execute("INSERT INTO %s (k, a) VALUES (?, ?) USING TTL " + MAX_TTL, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
++    private void createTable(boolean simple, boolean clustering)
++    {
++        if (simple)
++        {
++            if (clustering)
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
++            else
++                createTable("create table %s (k int primary key, a int, b int)");
++        }
++        else
++        {
++            if (clustering)
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
++            else
++                createTable("create table %s (k int primary key, a int, b set<text>)");
++        }
++    }
++
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
 -            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
      }
  
+     /**
 -     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
 -     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
++     * Verify that the computed TTL is equal to the maximum allowed ttl given the
++     * {@link AbstractCell#localDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
 -        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
++        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s WHERE k = 1");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
 -     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
++     * allowed expiration time {@link org.apache.cassandra.db.rows.Cell#MAX_DELETION_TIME} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
 -        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
++        return AbstractCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
+         createTable(simple, clustering);
+ 
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
+         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getSSTables().size());
++        assertEquals(0, cfs.getLiveSSTables().size());
+ 
+         copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
 -            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
++            cfs.scrub(true, false, true, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+             else
+                 assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+             else
+                 assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
+             assertEmpty(execute("SELECT * from %s"));
+         }
+     }
+ 
+     private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).getDirectories().getCFDirectories().iterator().next();
+         File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
+     private static File getTableDir(String table, boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering)
 -    {
 -        if (simple)
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b int, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b int)");
 -        }
 -        else
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b set<text>)");
 -        }
 -    }
 -
 -    private static File getTableDir(boolean simple, boolean clustering)
 -    {
 -        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
 -    }
 -
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 08336a1,9b1ede4..fc2faea
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -113,14 -127,18 +113,14 @@@ public class ScrubTes
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 1);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
  
--        CompactionManager.instance.performScrub(cfs, false, true, 2);
++        CompactionManager.instance.performScrub(cfs, false, true, false, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
      }
  
      @Test
@@@ -617,9 -763,9 +617,9 @@@
                  boolean failure = !scrubs[i];
                  if (failure)
                  { //make sure the next scrub fails
 -                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                    overrideWithGarbage(indexCfs.getLiveSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
                  }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 -                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, false, true, false,0);
                  assertEquals(failure ?
                               CompactionManager.AllSSTableOpStatus.ABORTED :
                               CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
@@@ -629,118 -775,8 +629,118 @@@
  
  
          // check index is still working
 -        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 -        assertNotNull(rows);
 -        assertEquals(numRows / 2, rows.size());
 +        assertOrdered(Util.cmd(cfs).filterOn(colName, Operator.EQ, 1L).build(), numRows / 2);
 +    }
 +
 +    private static SSTableMultiWriter createTestWriter(Descriptor descriptor, long keyCount, CFMetaData metadata, LifecycleTransaction txn)
 +    {
 +        SerializationHeader header = new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
 +        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(0);
 +        return new TestMultiWriter(new TestWriter(descriptor, keyCount, 0, metadata, collector, header, txn));
 +    }
 +
 +    private static class TestMultiWriter extends SimpleSSTableMultiWriter
 +    {
 +        TestMultiWriter(SSTableWriter writer)
 +        {
 +            super(writer);
 +        }
 +    }
 +
 +    /**
 +     * Test writer that allows to write out of order SSTable.
 +     */
 +    private static class TestWriter extends BigTableWriter
 +    {
 +        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata,
 +                   MetadataCollector collector, SerializationHeader header, LifecycleTransaction txn)
 +        {
 +            super(descriptor, keyCount, repairedAt, metadata, collector, header, txn);
 +        }
 +
 +        @Override
 +        protected long beforeAppend(DecoratedKey decoratedKey)
 +        {
 +            return dataFile.position();
 +        }
 +    }
 +
 +    /**
 +     * Tests with invalid sstables (containing duplicate entries in 2.0 and 3.0 storage format),
 +     * that were caused by upgrading from 2.x with duplicate range tombstones.
 +     *
 +     * See CASSANDRA-12144 for details.
 +     */
 +    @Test
 +    public void testFilterOutDuplicates() throws Exception
 +    {
 +        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".cf_with_duplicates_3_0 (a int, b int, c int, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
 +
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("cf_with_duplicates_3_0");
 +
 +        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
 +                                           "Keyspace1",
 +                                           "cf_with_duplicates_3_0");
 +
 +        for (String filename : new String[]{ "mb-3-big-CompressionInfo.db",
 +                                             "mb-3-big-Digest.crc32",
 +                                             "mb-3-big-Index.db",
 +                                             "mb-3-big-Summary.db",
 +                                             "mb-3-big-Data.db",
 +                                             "mb-3-big-Filter.db",
 +                                             "mb-3-big-Statistics.db",
 +                                             "mb-3-big-TOC.txt" })
 +        {
 +            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
 +        }
 +
 +        cfs.loadNewSSTables();
 +
-         cfs.scrub(true, true, true, 1);
++        cfs.scrub(true, true, false, false, false, 1);
 +
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
 +        assertEquals(1, rs.size());
 +        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".cf_with_duplicates_3_0 WHERE a=1 AND b =2", KEYSPACE));
 +        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
 +        assertEquals(0, rs.size());
 +    }
 +
 +    @Test
 +    public void testUpgradeSstablesWithDuplicates() throws Exception
 +    {
 +        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +        String cf = "cf_with_duplicates_2_0";
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".%s (a int, b int, c int, PRIMARY KEY (a, b))", KEYSPACE, cf), ConsistencyLevel.ONE);
 +
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cf);
 +
 +        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
 +                                           "Keyspace1",
 +                                           cf);
 +
 +        for (String filename : new String[]{ "lb-1-big-CompressionInfo.db",
 +                                             "lb-1-big-Data.db",
 +                                             "lb-1-big-Digest.adler32",
 +                                             "lb-1-big-Filter.db",
 +                                             "lb-1-big-Index.db",
 +                                             "lb-1-big-Statistics.db",
 +                                             "lb-1-big-Summary.db",
 +                                             "lb-1-big-TOC.txt" })
 +        {
 +            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
 +        }
 +
 +        cfs.loadNewSSTables();
 +
 +        cfs.sstablesRewrite(true, 1);
 +
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE, cf));
 +        assertEquals(1, rs.size());
 +        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".%s WHERE a=1 AND b =2", KEYSPACE, cf));
 +        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE, cf));
 +        assertEquals(0, rs.size());
      }
  }


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


[13/29] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by pa...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-3.0
Commit: 1602e606348959aead18531cb8027afb15f276e7
Parents: aa831c9 b294943
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:53:20 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:54:56 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  79 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  11 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  14 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/lb-1-big-Data.db                     | Bin 0 -> 103 bytes
 .../table1/lb-1-big-Digest.adler32              |   1 +
 .../table1/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table1/lb-1-big-Statistics.db               | Bin 0 -> 4466 bytes
 .../table1/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table1/lb-1-big-TOC.txt                     |   8 +
 .../table2/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/lb-1-big-Data.db                     | Bin 0 -> 98 bytes
 .../table2/lb-1-big-Digest.adler32              |   1 +
 .../table2/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table2/lb-1-big-Statistics.db               | Bin 0 -> 4478 bytes
 .../table2/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table2/lb-1-big-TOC.txt                     |   8 +
 .../table3/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table3/lb-1-big-Digest.adler32              |   1 +
 .../table3/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table3/lb-1-big-Statistics.db               | Bin 0 -> 4482 bytes
 .../table3/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table3/lb-1-big-TOC.txt                     |   8 +
 .../table4/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table4/lb-1-big-Digest.adler32              |   1 +
 .../table4/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table4/lb-1-big-Statistics.db               | Bin 0 -> 4494 bytes
 .../table4/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table4/lb-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 405 +++++++++++++++++++
 .../unit/org/apache/cassandra/db/ScrubTest.java |   2 +-
 58 files changed, 847 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0f6e61c,9332354..82da6ad
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -2.1.20
 +2.2.12
 + * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
 + * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
 + * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
 + * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 +Merged from 2.1:
+  * Protect against overflow of local expiration time (CASSANDRA-14092)
   * More PEP8 compliance for cqlsh (CASSANDRA-14021)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 5747941,fb6b4ee..4fe3508
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,15 -38,9 +38,16 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
 -   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 -
 -2.1.19
++    - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +      options are used. See CASSANDRA-13006 for more details.
 +    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +      for more details.
 +
 +2.2.11
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/debian/rules
----------------------------------------------------------------------
diff --cc debian/rules
index 35f5a51,70db61c..ff1d64d
--- a/debian/rules
+++ b/debian/rules
@@@ -63,7 -64,7 +63,7 @@@ binary-indep: build instal
  	dh_testroot
  	dh_installchangelogs
  	dh_installinit -u'start 50 2 3 4 5 . stop 50 0 1 6 .'
- 	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html
 -	dh_installdocs README.asc CHANGES.txt NEWS.txt
++	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html CASSANDRA-14092.txt
  	dh_installexamples tools/*.yaml
  	dh_bash-completion
  	dh_compress

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/redhat/cassandra.spec
----------------------------------------------------------------------
diff --cc redhat/cassandra.spec
index 0d4b271,a3f09b0..07c3dc5
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@@ -114,10 -113,10 +114,10 @@@ exit 
  
  %files
  %defattr(0644,root,root,0755)
--%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt
 -%attr(755,root,root) %{_bindir}/cassandra-cli
++%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt
  %attr(755,root,root) %{_bindir}/cassandra-stress
  %attr(755,root,root) %{_bindir}/cqlsh
 +%attr(755,root,root) %{_bindir}/cqlsh.py
  %attr(755,root,root) %{_bindir}/debug-cql
  %attr(755,root,root) %{_bindir}/nodetool
  %attr(755,root,root) %{_bindir}/sstablekeys

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 7b38e9f,23571ca..84f423a
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,17 -18,19 +18,23 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
 +import java.util.Collections;
+ import java.util.concurrent.TimeUnit;
  
 +import com.google.common.collect.Iterables;
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.db.ExpiringCell;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
  import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.NoSpamLogger;
  
  /**
   * Utility class for the Parser to gather attributes for modification

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 7e36e11,2989b9d..45908de
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -905,26 -879,13 +905,26 @@@ public class ColumnFamilyStore implemen
                                                  previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
 -            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
 -            flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 -            postFlushExecutor.submit(task);
 +            ListenableFutureTask<Void> flushTask = ListenableFutureTask.create(flush, null);
 +            flushExecutor.execute(flushTask);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
 +            postFlushExecutor.execute(task);
  
              @SuppressWarnings("unchecked")
-             ListenableFuture<ReplayPosition> future = 
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future =
 +                    // If either of the two tasks errors out, resulting future must also error out.
 +                    // Combine the two futures and only return post-flush result after both have completed.
 +                    // Note that flushTask will always yield null, but Futures.allAsList is
 +                    // order preserving, which is why the transform function returns the result
 +                    // from item 1 in it's input list (i.e. what was yielded by task).
 +                    Futures.transform(Futures.allAsList(flushTask, task),
 +                                      new Function<List<Object>, ReplayPosition>()
 +                                      {
 +                                          public ReplayPosition apply(List<Object> input)
 +                                          {
 +                                              return (ReplayPosition) input.get(1);
 +                                          }
 +                                      });
              return future;
          }
      }
@@@ -1610,29 -1516,48 +1610,28 @@@
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        // skip snapshot creation during scrub, SEE JIRA 5891
 -        if(!disableSnapshot)
 -            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 -    }
 -
 -    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
 -    {
 -        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
 -    }
 -
 -    public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
 -    {
 -        assert !sstables.isEmpty();
 -        data.markObsolete(sstables, compactionType);
 -    }
 -
 -    void replaceFlushed(Memtable memtable, SSTableReader sstable)
 -    {
 -        compactionStrategyWrapper.replaceFlushed(memtable, sstable);
 -    }
 -
 -    public boolean isValid()
 -    {
 -        return valid;
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
      }
  
-     @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public long getMemtableColumnsCount()
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
 -        return metric.memtableColumnsCount.value();
 -    }
 +        // skip snapshot creation during scrub, SEE JIRA 5891
 +        if(!disableSnapshot)
 +            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
  
 -    public long getMemtableDataSize()
 -    {
 -        return metric.memtableOnHeapSize.value();
 -    }
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
  
 -    public int getMemtableSwitchCount()
 -    {
 -        return (int) metric.memtableSwitchCount.count();
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 3350b20,6e3634a..d90abe9
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -348,9 -358,16 +348,15 @@@ public class CompactionManager implemen
          }
      }
  
 -    @Deprecated
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
 +    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
 -                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    throws InterruptedException, ExecutionException
+     {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -360,30 -377,11 +366,30 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
+                 scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
              }
 -        }, jobs);
 +        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
 +        }, 0, OperationType.VERIFY);
      }
  
      public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
@@@ -730,14 -717,14 +736,14 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
      {
 -        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 +        CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
 -        metrics.beginCompaction(scrubInfo);
 -        try
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
          {
 +            scrubInfo = scrubber.getScrubInfo();
 +            metrics.beginCompaction(scrubInfo);
              scrubber.scrub();
          }
          finally

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index b6b20fb,6d4537c..affee11
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -22,33 -22,30 +22,36 @@@ import java.io.*
  import java.util.*;
  
  import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.base.Predicates;
  import com.google.common.base.Throwables;
  import com.google.common.collect.AbstractIterator;
 -import com.google.common.collect.Sets;
  
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+ import org.apache.cassandra.db.composites.CellNames;
  import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.OutputHandler;
+ import org.apache.cassandra.utils.memory.HeapAllocator;
 +import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.concurrent.Refs;
  
  public class Scrubber implements Closeable
  {
      private final ColumnFamilyStore cfs;
      private final SSTableReader sstable;
 +    private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
 -    public final boolean validateColumns;
+     private final boolean reinsertOverflowedTTLRows;
  
      private final CompactionController controller;
      private final boolean isCommutative;
@@@ -84,20 -81,27 +88,28 @@@
      };
      private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
 +    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
+     }
+ 
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
 -        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
 +    @SuppressWarnings("resource")
-     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData) throws IOException
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
 -        this.sstable = sstable;
 +        this.transaction = transaction;
 +        this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.isOffline = isOffline;
 -        this.validateColumns = checkData;
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
  
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
@@@ -322,7 -339,7 +339,7 @@@
          // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
          // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
          // to the outOfOrderRows that will be later written to a new SSTable.
-         OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, checkData),
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
++        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
                                                                cfs.metadata.comparator.onDiskAtomComparator());
          if (prevKey != null && prevKey.compareTo(key) > 0)
          {
@@@ -342,6 -359,18 +359,18 @@@
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
++    private OnDiskAtomIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
++        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
+         return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                     outputHandler,
+                                                                                     negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 6896062,d718765..e416c7b
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@@ -31,8 -32,8 +32,9 @@@ import org.apache.cassandra.cql3.Attrib
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.context.CounterContext;
  import org.apache.cassandra.dht.IPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 +import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.CounterId;
  import org.apache.cassandra.utils.Pair;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 1ecedac,a7a8ca7..2c9ac4d
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2611,10 -2415,16 +2611,16 @@@ public class StorageService extends Not
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                      int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 761eed6,90c0fb5..f336bcc
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -265,15 -274,10 +265,18 @@@ public interface StorageServiceMBean ex
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++
 +    /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Rewrite all sstables to the latest version.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d735676,10e7185..8bdf9dc
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -312,9 -313,9 +313,9 @@@ public class ThriftValidatio
          if (cosc.column != null)
          {
              if (isCommutative)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
              validateColumnData(metadata, key, null, cosc.column);
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 24c5874,fcd4110..17bef02
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -236,16 -243,11 +236,16 @@@ public class NodeProbe implements AutoC
          return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
@@@ -267,22 -268,13 +267,22 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+         if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index a486a13,59d13d5..f5e84c5
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -119,10 -115,10 +126,10 @@@ public class StandaloneScrubbe
              {
                  for (SSTableReader sstable : sstables)
                  {
 -                    try
 +                    try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
 -                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
 -                        try
 +                        txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
                          {
                              scrubber.scrub();
                          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index dafe8d1,0000000..50224a0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,76 -1,0 +1,82 @@@
 +/*
 + * 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.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
++import org.apache.cassandra.tools.StandaloneScrubber;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
++    @Option(title = "reinsert_overflowed_ttl",
++    name = {"r", "--reinsert-overflowed-ttl"},
++    description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
++    private boolean reinsertOverflowedTTL = false;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
index 0000000,0000000..d7cc13b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
index 0000000,0000000..51213c2
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
index 0000000,0000000..d5b12df
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2292388625

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
index 0000000,0000000..3ab96ee
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
index 0000000,0000000..e8cc7e0
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
index 0000000,0000000..38373b4
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
index 0000000,0000000..762a229
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
index 0000000,0000000..ae89849
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3829731931

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
index 0000000,0000000..38a6e4c
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
index 0000000,0000000..64dab43
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
index 0000000,0000000..04a7384
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
index 0000000,0000000..33145df
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
index 0000000,0000000..2a542cd
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3574474340

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
index 0000000,0000000..5fb34e8
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
index 0000000,0000000..51203ae
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
index 0000000,0000000..c814fef
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
index 0000000,0000000..f40e71f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
index 0000000,0000000..e6675e4
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2405377913

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
index 0000000,0000000..8291383
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
index 0000000,0000000..2217c2d
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 0000000,ab4ef21..b1eaac1
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -1,0 -1,410 +1,405 @@@
+ package org.apache.cassandra.cql3.validation.operations;
+ 
+ import java.io.File;
+ import java.io.FileInputStream;
+ import java.io.FileOutputStream;
+ import java.io.IOException;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
+ 
+ import org.apache.cassandra.cql3.Attributes;
+ import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
+ import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.ExpiringCell;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
+ import org.junit.Test;
+ 
+ public class TTLTest extends CQLTester
+ {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
+     public static int MAX_TTL = ExpiringCell.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
+ 
+     @Test
+     public void testTTLPerRequestLimit() throws Throwable
+     {
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         // insert with low TTL should not be denied
+         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+         execute("TRUNCATE %s");
+ 
+         // insert with low TTL should not be denied
+         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+     }
+ 
+ 
+     @Test
+     public void testTTLDefaultLimit() throws Throwable
+     {
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live cannot be smaller than 0"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                         + (MAX_TTL + 1));
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
+         }
+ 
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
+     public void testRejectExpirationDateOverflowPolicy() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         checkTTLIsCapped("i");
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyPerRequest() throws Throwable
+     {
+         // Test cap policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+ 
+         // simple column, clustering, flush
+         baseCapExpirationDateOverflowTest(true, true, true);
+         // simple column, clustering, noflush
+         baseCapExpirationDateOverflowTest(true, true, false);
+         // simple column, noclustering, flush
+         baseCapExpirationDateOverflowTest(true, false, true);
+         // simple column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(true, false, false);
+         // complex column, clustering, flush
+         baseCapExpirationDateOverflowTest(false, true, true);
+         // complex column, clustering, noflush
+         baseCapExpirationDateOverflowTest(false, true, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, true);
+         // complex column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(false, false, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, false);
+ 
+         // Return to previous policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        createTable(true, true);
 -        createTable(true, false);
 -        createTable(false, true);
 -        createTable(false, false);
 -
+         baseTestRecoverOverflowedExpiration(false, false);
+         baseTestRecoverOverflowedExpiration(true, false);
+         baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
+     public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
+         if (simple)
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
+     }
+ 
+     /**
+      * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+      * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
+         UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+      * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+         return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
++        createTable(simple, clustering);
++
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getLiveSSTableCount());
++        assertEquals(0, cfs.getSSTables().size());
+ 
 -        copySSTablesToTableDir(simple, clustering);
++        copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
+             cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -            {
 -                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
 -                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
 -
 -            }
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
 -            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
++            assertEmpty(execute("SELECT * from %s"));
+         }
 -        cfs.truncateBlocking(); //cleanup for next tests
+     }
+ 
 -    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
++    private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
 -        File sourceDir = getTableDir(simple, clustering);
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering) throws Throwable
++    private static File getTableDir(String table, boolean simple, boolean clustering)
++    {
++        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
++    }
++
++    private void createTable(boolean simple, boolean clustering)
+     {
+         if (simple)
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+     }
+ 
+     private static File getTableDir(boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 4cca7ff,4efd082..9b1ede4
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -654,129 -565,4 +654,129 @@@ public class ScrubTes
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }


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


[09/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
new file mode 100644
index 0000000..ab4ef21
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -0,0 +1,410 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.BufferExpiringCell;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+public class TTLTest extends CQLTester
+{
+    public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+
+    public static int MAX_TTL = ExpiringCell.MAX_TTL;
+
+    public static final String SIMPLE_NOCLUSTERING = "table1";
+    public static final String SIMPLE_CLUSTERING = "table2";
+    public static final String COMPLEX_NOCLUSTERING = "table3";
+    public static final String COMPLEX_CLUSTERING = "table4";
+
+    @Test
+    public void testTTLPerRequestLimit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        // insert with low TTL should not be denied
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+        execute("TRUNCATE %s");
+
+        // insert with low TTL should not be denied
+        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+    }
+
+
+    @Test
+    public void testTTLDefaultLimit() throws Throwable
+    {
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live cannot be smaller than 0"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                        + (MAX_TTL + 1));
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                  + (MAX_TTL + 1) + ")"));
+        }
+
+        // table with default low TTL should not be denied
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+    }
+
+    @Test
+    public void testRejectExpirationDateOverflowPolicy() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+            execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        checkTTLIsCapped("i");
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyPerRequest() throws Throwable
+    {
+        // Test cap policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+
+        // simple column, clustering, flush
+        baseCapExpirationDateOverflowTest(true, true, true);
+        // simple column, clustering, noflush
+        baseCapExpirationDateOverflowTest(true, true, false);
+        // simple column, noclustering, flush
+        baseCapExpirationDateOverflowTest(true, false, true);
+        // simple column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(true, false, false);
+        // complex column, clustering, flush
+        baseCapExpirationDateOverflowTest(false, true, true);
+        // complex column, clustering, noflush
+        baseCapExpirationDateOverflowTest(false, true, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, true);
+        // complex column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(false, false, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, false);
+
+        // Return to previous policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+    {
+        createTable(true, true);
+        createTable(true, false);
+        createTable(false, true);
+        createTable(false, false);
+
+        baseTestRecoverOverflowedExpiration(false, false);
+        baseTestRecoverOverflowedExpiration(true, false);
+        baseTestRecoverOverflowedExpiration(true, true);
+    }
+
+    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+    {
+        // Create Table
+        if (simple)
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b int)");
+        }
+        else
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b set<text>)");
+        }
+
+        // Insert data with INSERT and UPDATE
+        if (simple)
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+            if (clustering)
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+            else
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+        }
+        else
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+            if (clustering)
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+            else
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+        }
+
+        // Maybe Flush
+        Keyspace ks = Keyspace.open(keyspace());
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+        // Verify data
+        verifyData(simple);
+
+        // Maybe major compact
+        if (flush)
+        {
+            // Major compact and check data is still present
+            ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+
+            // Verify data again
+            verifyData(simple);
+        }
+    }
+
+    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        // simple column, clustering
+        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+        // simple column, noclustering
+        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+        // complex column, clustering
+        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+        // complex column, noclustering
+        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+    }
+
+    private void verifyData(boolean simple) throws Throwable
+    {
+        if (simple)
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+        }
+        else
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        // Cannot retrieve TTL from collections
+        if (simple)
+            checkTTLIsCapped("b");
+    }
+
+    /**
+     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+     */
+    private void checkTTLIsCapped(String field) throws Throwable
+    {
+
+        // TTL is computed dynamically from row expiration time, so if it is
+        // equal or higher to the minimum max TTL we compute before the query
+        // we are fine.
+        int minMaxTTL = computeMaxTTL();
+        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+        for (UntypedResultSet.Row row : execute)
+        {
+            int ttl = row.getInt("ttl(" + field + ")");
+            assertTrue(ttl >= minMaxTTL);
+        }
+    }
+
+    /**
+     * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+     */
+    private int computeMaxTTL()
+    {
+        int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+    }
+
+    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        if (reinsertOverflowedTTL)
+        {
+            assert runScrub;
+        }
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
+
+        assertEquals(0, cfs.getLiveSSTableCount());
+
+        copySSTablesToTableDir(simple, clustering);
+
+        cfs.loadNewSSTables();
+
+        if (runScrub)
+        {
+            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+        }
+
+        if (reinsertOverflowedTTL)
+        {
+            if (simple)
+            {
+                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
+                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
+
+            }
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+
+            cfs.forceMajorCompaction();
+
+            if (simple)
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        else
+        {
+            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
+        }
+        cfs.truncateBlocking(); //cleanup for next tests
+    }
+
+    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
+    {
+        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
+        File sourceDir = getTableDir(simple, clustering);
+        for (File file : sourceDir.listFiles())
+        {
+            copyFile(file, destDir);
+        }
+    }
+
+    private void createTable(boolean simple, boolean clustering) throws Throwable
+    {
+        if (simple)
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
+        }
+        else
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
+        }
+    }
+
+    private static File getTableDir(boolean simple, boolean clustering)
+    {
+        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+    }
+
+    private static void copyFile(File src, File dest) throws IOException
+    {
+        byte[] buf = new byte[65536];
+        if (src.isFile())
+        {
+            File target = new File(dest, src.getName());
+            int rd;
+            FileInputStream is = new FileInputStream(src);
+            FileOutputStream os = new FileOutputStream(target);
+            while ((rd = is.read(buf)) >= 0)
+                os.write(buf, 0, rd);
+        }
+    }
+
+    public static String getTableName(boolean simple, boolean clustering)
+    {
+        if (simple)
+            return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+        else
+            return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+    }
+}


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


[02/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
Protect against overflow of local expiration time

Patch by Paulo Motta; Reviewed by Sam Tunnicliffe for CASSANDRA-14092


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

Branch: refs/heads/cassandra-2.1
Commit: b2949439ec62077128103540e42570238520f4ee
Parents: 5ba9e6d
Author: Paulo Motta <pa...@gmail.com>
Authored: Thu Feb 1 04:01:28 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:33:50 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  23 +-
 .../cassandra/cql/AbstractModification.java     |   4 +
 .../org/apache/cassandra/cql/Attributes.java    |  19 +
 .../apache/cassandra/cql/BatchStatement.java    |   4 +
 .../org/apache/cassandra/cql/CFPropDefs.java    |   7 +
 .../org/apache/cassandra/cql3/Attributes.java   |  81 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   4 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  15 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |  17 +-
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  12 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  23 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 ...test_keyspace-table1-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table1-ka-1-Data.db       | Bin 0 -> 103 bytes
 .../cql_test_keyspace-table1-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table1-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table1-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table1-ka-1-Statistics.db | Bin 0 -> 4454 bytes
 .../cql_test_keyspace-table1-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table1-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table2-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table2-ka-1-Data.db       | Bin 0 -> 96 bytes
 .../cql_test_keyspace-table2-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table2-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table2-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table2-ka-1-Statistics.db | Bin 0 -> 4466 bytes
 .../cql_test_keyspace-table2-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table2-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table3-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table3-ka-1-Data.db       | Bin 0 -> 182 bytes
 .../cql_test_keyspace-table3-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table3-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table3-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table3-ka-1-Statistics.db | Bin 0 -> 4470 bytes
 .../cql_test_keyspace-table3-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table3-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table4-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table4-ka-1-Data.db       | Bin 0 -> 181 bytes
 .../cql_test_keyspace-table4-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table4-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table4-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table4-ka-1-Statistics.db | Bin 0 -> 4482 bytes
 .../cql_test_keyspace-table4-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table4-ka-1-TOC.txt       |   8 +
 .../cql3/validation/operations/TTLTest.java     | 410 +++++++++++++++++++
 59 files changed, 898 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --git a/CASSANDRA-14092.txt b/CASSANDRA-14092.txt
new file mode 100644
index 0000000..5ac872c
--- /dev/null
+++ b/CASSANDRA-14092.txt
@@ -0,0 +1,81 @@
+CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+---------------------------------------------
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+after this date are not currently supported.
+
+# Expiration Date Overflow Policy
+
+We plan to lift this limitation in newer versions, but while the fix is not available,
+operators can decide which policy to apply when dealing with inserts with TTL exceeding
+the maximum supported expiration date:
+  -     REJECT: this is the default policy and will reject any requests with expiration
+                date timestamp after 2038-01-19T03:14:06+00:00.
+  -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                2038-01-19T03:14:06+00:00 and the client will receive a warning.
+  - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+
+These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
+startup option.
+
+# Potential data loss on earlier versions
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+INSERTS with TTL expiring after the maximum supported date, causing the expiration
+time field to overflow and the records to expire immediately. Expired records due
+to overflow will not be queryable and will be permanently removed after a compaction.
+
+2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+since an AssertionError is thrown during INSERT when the expiration time field overflows
+on these versions. When assertions are disabled then it is possible to INSERT entries
+with overflowed local expiration time and even the earlier versions are subject to data
+loss due to this bug.
+
+This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+where this issue is addressed as soon as possible.
+
+# Data Recovery
+
+SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+with overflowed expiration time that were backed up or did not go through compaction can
+be recovered by reinserting overflowed entries with a valid expiration time and a higher
+timestamp, since tombstones may have been generated with the original timestamp.
+
+To find out if an SSTable has an entry with overflowed expiration, inspect it with the
+sstable2json tool and look for a negative "local deletion time" field. SSTables in this
+condition should be backed up immediately, as they are subject to data loss during
+compaction.
+
+A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+rows with overflowed expiration time with the maximum expiration date of
+2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+for recovery of SSTables via scrub:
+
+- Offline scrub:
+   - Clone the data directory tree to another location, keeping only the folders and the
+     contents of the system tables.
+   - Clone the configuration directory to another location, setting the data_file_directories
+     property to the cloned data directory in the cloned cassandra.yaml.
+   - Copy the affected SSTables to the cloned data location of the affected table.
+   - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+   - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+         WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+         compaction, so the data with overflowed will be removed - make sure to back up
+         your SSTables before running scrub.
+   - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+   - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+
+- Online scrub:
+   - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+     as otherwise, the data may be removed permanently during compaction.
+   - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+   - run "nodetool refresh" to load the SSTables.
+   - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+   - Re-enable compactions after verifying that scrub recovered the missing entries.
+
+See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e17093d..9332354 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.20
+ * Protect against overflow of local expiration time (CASSANDRA-14092)
  * More PEP8 compliance for cqlsh (CASSANDRA-14021)
  * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 796a424..fb6b4ee 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -1,3 +1,23 @@
+PLEASE READ: MAXIMUM TTL EXPIRATION DATE NOTICE (CASSANDRA-14092)
+------------------------------------------------------------------
+(General upgrading instructions are available in the next section)
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that inserts with TTL thatl expire after
+this date are not currently supported. By default, INSERTS with TTL exceeding the
+maximum supported date are rejected, but it's possible to choose a different
+ expiration overflow policy. See CASSANDRA-14092.txt for more details.
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x) there was no protection against INSERTS
+with TTL expiring after the maximum supported date, causing the expiration time
+field to overflow and the records to expire immediately. Clusters in the 2.X and
+lower series are not subject to this when assertions are enabled. Backed up SSTables
+can be potentially recovered and recovery instructions can be found on the
+CASSANDRA-14092.txt file.
+
+If you use or plan to use very large TTLS (10 to 20 years), read CASSANDRA-14092.txt
+for more information.
+
 GENERAL UPGRADING ADVICE FOR ANY VERSION
 ========================================
 
@@ -18,8 +38,7 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
+   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 
 2.1.19
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/AbstractModification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
index 8da2611..e98764b 100644
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ b/src/java/org/apache/cassandra/cql/AbstractModification.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.cql;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.thrift.ThriftClientState;
+import org.w3c.dom.Attr;
 
 public abstract class AbstractModification
 {
@@ -89,6 +92,7 @@ public abstract class AbstractModification
 
     public int getTimeToLive()
     {
+        Attributes.maybeApplyExpirationDateOverflowPolicy(keyspace, columnFamily, timeToLive);
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
index faee3b8..c1c37ef 100644
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ b/src/java/org/apache/cassandra/cql/Attributes.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.cql;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
  * Class to contain attributes for statements
@@ -73,4 +76,20 @@ public class Attributes
         return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
     }
 
+    public static void maybeApplyExpirationDateOverflowPolicy(String keyspace, String columnFamily, Integer timeToLive)
+    {
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
+        if (metadata != null)
+        {
+            try
+            {
+                org.apache.cassandra.cql3.Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, timeToLive, false);
+            }
+            catch (InvalidRequestException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
index b141bcc..e5a95b8 100644
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql/BatchStatement.java
@@ -72,6 +72,10 @@ public class BatchStatement
 
     public int getTimeToLive()
     {
+        for (AbstractModification statement : statements)
+        {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(statement.keyspace, statement.columnFamily, timeToLive);
+        }
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
index f65cb94..a0c8d0d 100644
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql/CFPropDefs.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -191,6 +192,12 @@ public class CFPropDefs {
                         KW_DEFAULT_TIME_TO_LIVE,
                         0,
                         CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
+
+            if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+                throw new InvalidRequestException(String.format("%s must be less than or equal to %d (got %s)",
+                                                                KW_DEFAULT_TIME_TO_LIVE,
+                                                                ExpiringCell.MAX_TTL,
+                                                                defaultTimeToLive));
         }
 
         CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index 435757b..23571ca 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.NoSpamLogger;
 
 /**
  * Utility class for the Parser to gather attributes for modification
@@ -32,6 +38,41 @@ import org.apache.cassandra.serializers.MarshalException;
  */
 public class Attributes
 {
+    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
+
+    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
+
+    public enum ExpirationDateOverflowPolicy
+    {
+        REJECT, CAP
+    }
+
+    @VisibleForTesting
+    public static ExpirationDateOverflowPolicy policy;
+
+    static {
+        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
+        try
+        {
+            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
+        }
+        catch (RuntimeException e)
+        {
+            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
+            policy = ExpirationDateOverflowPolicy.REJECT;
+        }
+    }
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
+                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
+                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
+                                                                          "is fixed. See CASSANDRA-14092 for more details.";
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
+                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
+                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
+                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
+
     private final Term timestamp;
     private final Term timeToLive;
 
@@ -77,10 +118,13 @@ public class Attributes
         return LongType.instance.compose(tval);
     }
 
-    public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+    public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
     {
         if (timeToLive == null)
-            return 0;
+        {
+            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
+            return metadata.getDefaultTimeToLive();
+        }
 
         ByteBuffer tval = timeToLive.bindAndGet(options);
         if (tval == null)
@@ -102,6 +146,8 @@ public class Attributes
         if (ttl > ExpiringCell.MAX_TTL)
             throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 
+        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+
         return ttl;
     }
 
@@ -135,4 +181,33 @@ public class Attributes
             return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[ttl]", true), Int32Type.instance);
         }
     }
+
+    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
+    {
+        if (ttl == 0)
+            return;
+
+        // Check for localExpirationTime overflow (CASSANDRA-14092)
+        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
+        if (ttl + nowInSecs < 0)
+        {
+            switch (policy)
+            {
+                case CAP:
+                    /**
+                     * Capping at this stage is basically not rejecting the request. The actual capping is done
+                     * by {@link org.apache.cassandra.db.BufferExpiringCell#computeLocalExpirationTime(int)},
+                     * which converts the negative TTL to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
+                     */
+                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES,
+                                     TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
+                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
+                    return;
+
+                default: //REJECT
+                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
+                                                                    isDefaultTTL? "default " : "", ttl));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 17edd6d..27dd57f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -22,6 +22,7 @@ import java.util.*;
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.CFMetaData.SpeculativeRetry;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -127,6 +128,12 @@ public class CFPropDefs extends PropertyDefinitions
         }
 
         validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
+        Integer defaultTimeToLive = getInt(KW_DEFAULT_TIME_TO_LIVE, 0);
+        if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+            throw new ConfigurationException(String.format("%s must be less than or equal to %d (got %s)",
+                                                           KW_DEFAULT_TIME_TO_LIVE,
+                                                           ExpiringCell.MAX_TTL,
+                                                           defaultTimeToLive));
 
         Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null);
         Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f84188a..8038c6c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -124,7 +124,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options);
+        return attrs.getTimeToLive(options, cfm);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
index e01d860..1b2c384 100644
--- a/src/java/org/apache/cassandra/db/AbstractNativeCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -575,6 +575,12 @@ public abstract class AbstractNativeCell extends AbstractCell implements CellNam
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     protected long internalSize()
     {
         return MemoryUtil.getInt(peer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCell.java b/src/java/org/apache/cassandra/db/BufferCell.java
index a7d632d..ee5fe41 100644
--- a/src/java/org/apache/cassandra/db/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCell.java
@@ -69,6 +69,12 @@ public class BufferCell extends AbstractCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public CellName name() {
         return name;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDeletedCell.java b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
index a38f322..3762e1f 100644
--- a/src/java/org/apache/cassandra/db/BufferDeletedCell.java
+++ b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
@@ -54,6 +54,12 @@ public class BufferDeletedCell extends BufferCell implements DeletedCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public boolean isLive()
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
index 25172c8..ea40676 100644
--- a/src/java/org/apache/cassandra/db/BufferExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -31,19 +31,23 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 public class BufferExpiringCell extends BufferCell implements ExpiringCell
 {
+    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
+
     private final int localExpirationTime;
     private final int timeToLive;
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
-        this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+        super(name, value, timestamp);
+        assert timeToLive > 0 : timeToLive;
+        this.timeToLive = timeToLive;
+        this.localExpirationTime = computeLocalExpirationTime(timeToLive);
     }
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
     {
         super(name, value, timestamp);
         assert timeToLive > 0 : timeToLive;
-        assert localExpirationTime > 0 : localExpirationTime;
         this.timeToLive = timeToLive;
         this.localExpirationTime = localExpirationTime;
     }
@@ -66,6 +70,12 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        return new BufferExpiringCell(name(), value(), newTimestamp, timeToLive, newLocalDeletionTime);
+    }
+
+    @Override
     public int cellDataSize()
     {
         return super.cellDataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
@@ -176,7 +186,9 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     /** @return Either a DeletedCell, or an ExpiringCell. */
     public static Cell create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
     {
-        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
+        // CASSANDRA-14092 may have written rows with negative localExpirationTime, so we don't turn them into tombstones yet
+        // to be able to recover them with scrub.
+        if (localExpirationTime < 0 || localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
             return new BufferExpiringCell(name, value, timestamp, timeToLive, localExpirationTime);
         // The column is now expired, we can safely return a simple tombstone. Note that
         // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
@@ -184,4 +196,20 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
         // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
         return new BufferDeletedCell(name, localExpirationTime - timeToLive, timestamp);
     }
+
+    /**
+     * This method computes the {@link #localExpirationTime}, maybe capping to the maximum representable value
+     * which is {@link #MAX_DELETION_TIME}.
+     *
+     * Please note that the {@link org.apache.cassandra.cql3.Attributes.ExpirationDateOverflowPolicy} is applied
+     * during {@link org.apache.cassandra.cql3.Attributes#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
+     * so if the request was not denied it means it's expiration date should be capped.
+     *
+     * See CASSANDRA-14092
+     */
+    private int computeLocalExpirationTime(int timeToLive)
+    {
+        int localExpirationTime =  (int) (System.currentTimeMillis() / 1000) + timeToLive;
+        return localExpirationTime >= 0? localExpirationTime : MAX_DELETION_TIME;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Cell.java b/src/java/org/apache/cassandra/db/Cell.java
index 7c3926a..274f369 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -38,6 +38,8 @@ public interface Cell extends OnDiskAtom
 
     public Cell withUpdatedTimestamp(long newTimestamp);
 
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
+
     @Override
     public CellName name();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6e82745..2989b9d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1516,12 +1516,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
     }
 
     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index 99cfe35..c10a15f 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -60,6 +60,7 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
     @VisibleForTesting
     public DeletionTime(long markedForDeleteAt, int localDeletionTime)
     {
+        assert localDeletionTime >= 0 : localDeletionTime;
         this.markedForDeleteAt = markedForDeleteAt;
         this.localDeletionTime = localDeletionTime;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 87819ba..6e3634a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -358,8 +358,15 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
+    @Deprecated
     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
+        return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+    }
+
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
+                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
+    {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -372,7 +379,7 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void execute(SSTableReader input) throws IOException
             {
-                scrubOne(cfs, input, skipCorrupted, checkData);
+                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
             }
         }, jobs);
     }
@@ -710,9 +717,9 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 
         CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
         metrics.beginCompaction(scrubInfo);
@@ -1352,7 +1359,7 @@ public class CompactionManager implements CompactionManagerMBean
         public void afterExecute(Runnable r, Throwable t)
         {
             DebuggableThreadPoolExecutor.maybeResetTraceSessionWrapper(r);
-    
+
             if (t == null)
                 t = DebuggableThreadPoolExecutor.extractThrowable(r);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 2df3665..6d4537c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
@@ -35,6 +36,7 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class Scrubber implements Closeable
 {
@@ -43,6 +45,7 @@ public class Scrubber implements Closeable
     private final File destination;
     private final boolean skipCorrupted;
     public final boolean validateColumns;
+    private final boolean reinsertOverflowedTTLRows;
 
     private final CompactionController controller;
     private final boolean isCommutative;
@@ -67,6 +70,7 @@ public class Scrubber implements Closeable
     long nextRowPositionFromIndex;
 
     private final OutputHandler outputHandler;
+    private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
 
     private static final Comparator<Row> rowComparator = new Comparator<Row>()
     {
@@ -79,10 +83,17 @@ public class Scrubber implements Closeable
 
     public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
+    {
+        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
+    }
+
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
     {
         this.cfs = cfs;
         this.sstable = sstable;
@@ -90,6 +101,7 @@ public class Scrubber implements Closeable
         this.skipCorrupted = skipCorrupted;
         this.isOffline = isOffline;
         this.validateColumns = checkData;
+        this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 
         List<SSTableReader> toScrub = Collections.singletonList(sstable);
 
@@ -131,6 +143,9 @@ public class Scrubber implements Closeable
 
         this.currentRowPositionFromIndex = 0;
         this.nextRowPositionFromIndex = 0;
+
+        if (reinsertOverflowedTTLRows)
+            outputHandler.output("Starting scrub with reinsert overflowed TTL option");
     }
 
     public void scrub()
@@ -311,6 +326,8 @@ public class Scrubber implements Closeable
         else
         {
             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+            if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
             if (badRows > 0)
                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
         }
@@ -322,7 +339,7 @@ public class Scrubber implements Closeable
         // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
         // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
         // to the outOfOrderRows that will be later written to a new SSTable.
-        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
+        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
                                                               cfs.metadata.comparator.onDiskAtomComparator());
         if (prevKey != null && prevKey.compareTo(key) > 0)
         {
@@ -342,6 +359,18 @@ public class Scrubber implements Closeable
         return true;
     }
 
+    /**
+     * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+     * is specified
+     */
+    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
+    {
+        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
+        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                    outputHandler,
+                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+    }
+
     private void updateIndexKey()
     {
         currentIndexKey = nextIndexKey;
@@ -516,6 +545,11 @@ public class Scrubber implements Closeable
         }
     }
 
+    public class NegativeLocalDeletionInfoMetrics
+    {
+        public volatile int fixedRows = 0;
+    }
+
     /**
      * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
      * cells order and collect the out of order cells to correct the problem.
@@ -601,4 +635,61 @@ public class Scrubber implements Closeable
             return cf;
         }
     }
+
+    /**
+     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
+     *
+     * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+     */
+    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+    {
+        /**
+         * The decorated iterator.
+         */
+        private final OnDiskAtomIterator iterator;
+
+        private final OutputHandler outputHandler;
+        private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+
+        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
+                                                    NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+        {
+            this.iterator = iterator;
+            this.outputHandler = outputHandler;
+            this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+        }
+
+        public ColumnFamily getColumnFamily()
+        {
+            return iterator.getColumnFamily();
+        }
+
+        public DecoratedKey getKey()
+        {
+            return iterator.getKey();
+        }
+
+        public void close() throws IOException
+        {
+            iterator.close();
+        }
+
+        @Override
+        protected OnDiskAtom computeNext()
+        {
+            if (!iterator.hasNext())
+                return endOfData();
+
+            OnDiskAtom next = iterator.next();
+
+            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
+            {
+                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
+                negativeLocalExpirationTimeMetrics.fixedRows++;
+                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
+            }
+
+            return next;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 557c3de..d718765 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -28,9 +28,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CounterId;
@@ -155,7 +157,20 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS) throws IOException
     {
-        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        int localExpirationTime = (int) (expirationTimestampMS / 1000);
+        try
+        {
+            // This will throw exception if policy is REJECT and now() + ttl is higher than MAX_DELETION_TIME
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+            // If exception was not thrown, this means the policy was CAP, so we check for overflow and cap if that's the case
+            if (localExpirationTime < 0)
+                localExpirationTime = BufferExpiringCell.MAX_DELETION_TIME;
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e);
+        }
+        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, localExpirationTime));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 27939f9..a7a8ca7 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2415,10 +2415,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d3a1725..90c0fb5 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -219,16 +219,16 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
-     * 
+     *
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
      * @param columnFamilyList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
-    
-    
-    
+
+
+
     /**
      * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
@@ -274,8 +274,11 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
@@ -381,11 +384,11 @@ public interface StorageServiceMBean extends NotificationEmitter
      * If level cannot be parsed, then the level will be defaulted to DEBUG<br>
      * <br>
      * The logback configuration should have < jmxConfigurator /> set
-     * 
+     *
      * @param classQualifier The logger's classQualifer
      * @param level The log level
-     * @throws Exception 
-     * 
+     * @throws Exception
+     *
      *  @see ch.qos.logback.classic.Level#toLevel(String)
      */
     public void setLoggingLevel(String classQualifier, String level) throws Exception;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d5d9f73..10e7185 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -24,6 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -314,7 +315,7 @@ public class ThriftValidation
             if (isCommutative)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 
-            validateTtl(cosc.column);
+            validateTtl(metadata, cosc.column);
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
             validateColumnData(metadata, key, null, cosc.column);
         }
@@ -349,7 +350,7 @@ public class ThriftValidation
         }
     }
 
-    private static void validateTtl(Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
+    private static void validateTtl(CFMetaData metadata, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
         if (column.isSetTtl())
         {
@@ -358,9 +359,11 @@ public class ThriftValidation
 
             if (column.ttl > ExpiringCell.MAX_TTL)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
         }
         else
         {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
             // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
             assert column.ttl == 0;
         }
@@ -434,7 +437,7 @@ public class ThriftValidation
      */
     public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        validateTtl(column);
+        validateTtl(metadata, column);
         if (!column.isSetValue())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
         if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 13c7acf..fcd4110 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -243,9 +243,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
     }
 
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
@@ -268,10 +268,10 @@ public class NodeProbe implements AutoCloseable
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         checkJobs(out, jobs);
-        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
@@ -562,7 +562,7 @@ public class NodeProbe implements AutoCloseable
 
     /**
      * Take a snapshot of all column family from different keyspaces.
-     * 
+     *
      * @param snapshotName
      *            the name of the snapshot.
      * @param columnfamilylist
@@ -1302,7 +1302,7 @@ public class NodeProbe implements AutoCloseable
         }
         catch (Exception e)
         {
-          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e); 
+          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d1afb6f..54d7fb7 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -551,20 +551,20 @@ public class NodeTool
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } 
+            }
             catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
                 errors.append("Note: " + ex.getMessage() + "%n");
                 showEffectiveOwnership = false;
-            } 
+            }
             catch (IllegalArgumentException ex)
             {
                 System.out.printf("%nError: " + ex.getMessage() + "%n");
                 return;
             }
 
-            
+
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
                 printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
@@ -1282,6 +1282,11 @@ public class NodeTool
                 description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
         private int jobs = 2;
 
+        @Option(title = "reinsert_overflowed_ttl",
+        name = {"r", "--reinsert-overflowed-ttl"},
+        description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+        private boolean reinsertOverflowedTTL = false;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1292,7 +1297,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, reinsertOverflowedTTL   , !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -2197,7 +2202,7 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
-            
+
             StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships = null;
@@ -2249,9 +2254,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-            
+
             System.out.printf("%n" + errors.toString());
-            
+
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -2726,7 +2731,7 @@ public class NodeTool
                 probe.truncateHints(endpoint);
         }
     }
-    
+
     @Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
     public static class SetLoggingLevel extends NodeToolCmd
     {
@@ -2741,7 +2746,7 @@ public class NodeTool
             probe.setLoggingLevel(classQualifier, level);
         }
     }
-    
+
     @Command(name = "getlogginglevels", description = "Get the runtime logging levels")
     public static class GetLoggingLevels extends NodeToolCmd
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index fdf6c8d..59d13d5 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -47,6 +47,12 @@ import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 public class StandaloneScrubber
 {
+    public static final String REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION = "Rewrites rows with overflowed expiration date affected by CASSANDRA-14092 with " +
+                                                                            "the maximum supported expiration date of 2038-01-19T03:14:06+00:00. " +
+                                                                            "The rows are rewritten with the original timestamp incremented by one millisecond " +
+                                                                            "to override/supersede any potential tombstone that may have been generated " +
+                                                                            "during compaction of the affected rows.";
+
     private static final String TOOL_NAME = "sstablescrub";
     private static final String VERBOSE_OPTION  = "verbose";
     private static final String DEBUG_OPTION  = "debug";
@@ -54,6 +60,7 @@ public class StandaloneScrubber
     private static final String MANIFEST_CHECK_OPTION  = "manifest-check";
     private static final String SKIP_CORRUPTED_OPTION = "skip-corrupted";
     private static final String NO_VALIDATE_OPTION = "no-validate";
+    private static final String REINSERT_OVERFLOWED_TTL_OPTION = "reinsert-overflowed-ttl";
 
     public static void main(String args[])
     {
@@ -110,7 +117,7 @@ public class StandaloneScrubber
                 {
                     try
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
                         try
                         {
                             scrubber.scrub();
@@ -192,6 +199,7 @@ public class StandaloneScrubber
         public boolean manifestCheckOnly;
         public boolean skipCorrupted;
         public boolean noValidate;
+        public boolean reinsertOverflowedTTL;
 
         private Options(String keyspaceName, String cfName)
         {
@@ -232,6 +240,7 @@ public class StandaloneScrubber
                 opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                 opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                 opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
+                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
 
                 return opts;
             }
@@ -258,6 +267,7 @@ public class StandaloneScrubber
             options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
             options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
             options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
+            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
             return options;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d7cc13b
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
new file mode 100644
index 0000000..0e3da66
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
new file mode 100644
index 0000000..8a6dcba
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
@@ -0,0 +1 @@
+4012184764
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
new file mode 100644
index 0000000..3ab96ee
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
new file mode 100644
index 0000000..9bde77e
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..38373b4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
new file mode 100644
index 0000000..bdd4549
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
new file mode 100644
index 0000000..f58914a
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
@@ -0,0 +1 @@
+3463582096
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
new file mode 100644
index 0000000..38a6e4c
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
new file mode 100644
index 0000000..8ee9116
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..04a7384
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
new file mode 100644
index 0000000..1fc8ba4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
new file mode 100644
index 0000000..cd091ad
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1524836732
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
new file mode 100644
index 0000000..5fb34e8
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
new file mode 100644
index 0000000..4d961fb
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..c814fef
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
new file mode 100644
index 0000000..92032a7
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
new file mode 100644
index 0000000..a45d821
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2189764235
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
new file mode 100644
index 0000000..8291383
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
new file mode 100644
index 0000000..68f76ae
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db


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


[07/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
new file mode 100644
index 0000000..ab4ef21
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -0,0 +1,410 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.BufferExpiringCell;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+public class TTLTest extends CQLTester
+{
+    public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+
+    public static int MAX_TTL = ExpiringCell.MAX_TTL;
+
+    public static final String SIMPLE_NOCLUSTERING = "table1";
+    public static final String SIMPLE_CLUSTERING = "table2";
+    public static final String COMPLEX_NOCLUSTERING = "table3";
+    public static final String COMPLEX_CLUSTERING = "table4";
+
+    @Test
+    public void testTTLPerRequestLimit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        // insert with low TTL should not be denied
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+        execute("TRUNCATE %s");
+
+        // insert with low TTL should not be denied
+        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+    }
+
+
+    @Test
+    public void testTTLDefaultLimit() throws Throwable
+    {
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live cannot be smaller than 0"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                        + (MAX_TTL + 1));
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                  + (MAX_TTL + 1) + ")"));
+        }
+
+        // table with default low TTL should not be denied
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+    }
+
+    @Test
+    public void testRejectExpirationDateOverflowPolicy() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+            execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        checkTTLIsCapped("i");
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyPerRequest() throws Throwable
+    {
+        // Test cap policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+
+        // simple column, clustering, flush
+        baseCapExpirationDateOverflowTest(true, true, true);
+        // simple column, clustering, noflush
+        baseCapExpirationDateOverflowTest(true, true, false);
+        // simple column, noclustering, flush
+        baseCapExpirationDateOverflowTest(true, false, true);
+        // simple column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(true, false, false);
+        // complex column, clustering, flush
+        baseCapExpirationDateOverflowTest(false, true, true);
+        // complex column, clustering, noflush
+        baseCapExpirationDateOverflowTest(false, true, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, true);
+        // complex column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(false, false, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, false);
+
+        // Return to previous policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+    {
+        createTable(true, true);
+        createTable(true, false);
+        createTable(false, true);
+        createTable(false, false);
+
+        baseTestRecoverOverflowedExpiration(false, false);
+        baseTestRecoverOverflowedExpiration(true, false);
+        baseTestRecoverOverflowedExpiration(true, true);
+    }
+
+    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+    {
+        // Create Table
+        if (simple)
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b int)");
+        }
+        else
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b set<text>)");
+        }
+
+        // Insert data with INSERT and UPDATE
+        if (simple)
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+            if (clustering)
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+            else
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+        }
+        else
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+            if (clustering)
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+            else
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+        }
+
+        // Maybe Flush
+        Keyspace ks = Keyspace.open(keyspace());
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+        // Verify data
+        verifyData(simple);
+
+        // Maybe major compact
+        if (flush)
+        {
+            // Major compact and check data is still present
+            ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+
+            // Verify data again
+            verifyData(simple);
+        }
+    }
+
+    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        // simple column, clustering
+        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+        // simple column, noclustering
+        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+        // complex column, clustering
+        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+        // complex column, noclustering
+        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+    }
+
+    private void verifyData(boolean simple) throws Throwable
+    {
+        if (simple)
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+        }
+        else
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        // Cannot retrieve TTL from collections
+        if (simple)
+            checkTTLIsCapped("b");
+    }
+
+    /**
+     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+     */
+    private void checkTTLIsCapped(String field) throws Throwable
+    {
+
+        // TTL is computed dynamically from row expiration time, so if it is
+        // equal or higher to the minimum max TTL we compute before the query
+        // we are fine.
+        int minMaxTTL = computeMaxTTL();
+        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+        for (UntypedResultSet.Row row : execute)
+        {
+            int ttl = row.getInt("ttl(" + field + ")");
+            assertTrue(ttl >= minMaxTTL);
+        }
+    }
+
+    /**
+     * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+     */
+    private int computeMaxTTL()
+    {
+        int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+    }
+
+    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        if (reinsertOverflowedTTL)
+        {
+            assert runScrub;
+        }
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
+
+        assertEquals(0, cfs.getLiveSSTableCount());
+
+        copySSTablesToTableDir(simple, clustering);
+
+        cfs.loadNewSSTables();
+
+        if (runScrub)
+        {
+            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+        }
+
+        if (reinsertOverflowedTTL)
+        {
+            if (simple)
+            {
+                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
+                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
+
+            }
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+
+            cfs.forceMajorCompaction();
+
+            if (simple)
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        else
+        {
+            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
+        }
+        cfs.truncateBlocking(); //cleanup for next tests
+    }
+
+    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
+    {
+        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
+        File sourceDir = getTableDir(simple, clustering);
+        for (File file : sourceDir.listFiles())
+        {
+            copyFile(file, destDir);
+        }
+    }
+
+    private void createTable(boolean simple, boolean clustering) throws Throwable
+    {
+        if (simple)
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
+        }
+        else
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
+        }
+    }
+
+    private static File getTableDir(boolean simple, boolean clustering)
+    {
+        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+    }
+
+    private static void copyFile(File src, File dest) throws IOException
+    {
+        byte[] buf = new byte[65536];
+        if (src.isFile())
+        {
+            File target = new File(dest, src.getName());
+            int rd;
+            FileInputStream is = new FileInputStream(src);
+            FileOutputStream os = new FileOutputStream(target);
+            while ((rd = is.read(buf)) >= 0)
+                os.write(buf, 0, rd);
+        }
+    }
+
+    public static String getTableName(boolean simple, boolean clustering)
+    {
+        if (simple)
+            return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+        else
+            return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+    }
+}


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


[03/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
new file mode 100644
index 0000000..ab4ef21
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@ -0,0 +1,410 @@
+package org.apache.cassandra.cql3.validation.operations;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import org.apache.cassandra.cql3.Attributes;
+import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.db.BufferExpiringCell;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.ExpiringCell;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.utils.FBUtilities;
+
+import org.junit.Test;
+
+public class TTLTest extends CQLTester
+{
+    public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+
+    public static int MAX_TTL = ExpiringCell.MAX_TTL;
+
+    public static final String SIMPLE_NOCLUSTERING = "table1";
+    public static final String SIMPLE_CLUSTERING = "table2";
+    public static final String COMPLEX_NOCLUSTERING = "table3";
+    public static final String COMPLEX_CLUSTERING = "table4";
+
+    @Test
+    public void testTTLPerRequestLimit() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        // insert with low TTL should not be denied
+        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+        execute("TRUNCATE %s");
+
+        // insert with low TTL should not be denied
+        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("ttl is too large."));
+        }
+
+        try
+        {
+            execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+            fail("Expect InvalidRequestException");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+        }
+    }
+
+
+    @Test
+    public void testTTLDefaultLimit() throws Throwable
+    {
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live cannot be smaller than 0"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                        + (MAX_TTL + 1));
+            fail("Expect Invalid schema");
+        }
+        catch (RuntimeException e)
+        {
+            assertTrue(e.getCause()
+                        .getCause()
+                        .getMessage()
+                        .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                  + (MAX_TTL + 1) + ")"));
+        }
+
+        // table with default low TTL should not be denied
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+    }
+
+    @Test
+    public void testRejectExpirationDateOverflowPolicy() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+        try
+        {
+            execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+        try
+        {
+            createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+            execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        }
+        catch (InvalidRequestException e)
+        {
+            assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+        }
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+    {
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+        checkTTLIsCapped("i");
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testCapExpirationDatePolicyPerRequest() throws Throwable
+    {
+        // Test cap policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+
+        // simple column, clustering, flush
+        baseCapExpirationDateOverflowTest(true, true, true);
+        // simple column, clustering, noflush
+        baseCapExpirationDateOverflowTest(true, true, false);
+        // simple column, noclustering, flush
+        baseCapExpirationDateOverflowTest(true, false, true);
+        // simple column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(true, false, false);
+        // complex column, clustering, flush
+        baseCapExpirationDateOverflowTest(false, true, true);
+        // complex column, clustering, noflush
+        baseCapExpirationDateOverflowTest(false, true, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, true);
+        // complex column, noclustering, noflush
+        baseCapExpirationDateOverflowTest(false, false, false);
+        // complex column, noclustering, flush
+        baseCapExpirationDateOverflowTest(false, false, false);
+
+        // Return to previous policy
+        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+    }
+
+    @Test
+    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+    {
+        createTable(true, true);
+        createTable(true, false);
+        createTable(false, true);
+        createTable(false, false);
+
+        baseTestRecoverOverflowedExpiration(false, false);
+        baseTestRecoverOverflowedExpiration(true, false);
+        baseTestRecoverOverflowedExpiration(true, true);
+    }
+
+    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+    {
+        // Create Table
+        if (simple)
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b int)");
+        }
+        else
+        {
+            if (clustering)
+                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+            else
+                createTable("create table %s (k int primary key, a int, b set<text>)");
+        }
+
+        // Insert data with INSERT and UPDATE
+        if (simple)
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+            if (clustering)
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+            else
+                execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+        }
+        else
+        {
+            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+            if (clustering)
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+            else
+                execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+        }
+
+        // Maybe Flush
+        Keyspace ks = Keyspace.open(keyspace());
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+        // Verify data
+        verifyData(simple);
+
+        // Maybe major compact
+        if (flush)
+        {
+            // Major compact and check data is still present
+            ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+
+            // Verify data again
+            verifyData(simple);
+        }
+    }
+
+    public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        // simple column, clustering
+        testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+        // simple column, noclustering
+        testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+        // complex column, clustering
+        testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+        // complex column, noclustering
+        testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+    }
+
+    private void verifyData(boolean simple) throws Throwable
+    {
+        if (simple)
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+        }
+        else
+        {
+            assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        // Cannot retrieve TTL from collections
+        if (simple)
+            checkTTLIsCapped("b");
+    }
+
+    /**
+     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+     */
+    private void checkTTLIsCapped(String field) throws Throwable
+    {
+
+        // TTL is computed dynamically from row expiration time, so if it is
+        // equal or higher to the minimum max TTL we compute before the query
+        // we are fine.
+        int minMaxTTL = computeMaxTTL();
+        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+        for (UntypedResultSet.Row row : execute)
+        {
+            int ttl = row.getInt("ttl(" + field + ")");
+            assertTrue(ttl >= minMaxTTL);
+        }
+    }
+
+    /**
+     * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+     */
+    private int computeMaxTTL()
+    {
+        int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+    }
+
+    public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+    {
+        if (reinsertOverflowedTTL)
+        {
+            assert runScrub;
+        }
+
+        Keyspace keyspace = Keyspace.open(KEYSPACE);
+        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
+
+        assertEquals(0, cfs.getLiveSSTableCount());
+
+        copySSTablesToTableDir(simple, clustering);
+
+        cfs.loadNewSSTables();
+
+        if (runScrub)
+        {
+            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+        }
+
+        if (reinsertOverflowedTTL)
+        {
+            if (simple)
+            {
+                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
+                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
+
+            }
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+
+            cfs.forceMajorCompaction();
+
+            if (simple)
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
+            else
+                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+        }
+        else
+        {
+            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
+        }
+        cfs.truncateBlocking(); //cleanup for next tests
+    }
+
+    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
+    {
+        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
+        File sourceDir = getTableDir(simple, clustering);
+        for (File file : sourceDir.listFiles())
+        {
+            copyFile(file, destDir);
+        }
+    }
+
+    private void createTable(boolean simple, boolean clustering) throws Throwable
+    {
+        if (simple)
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
+        }
+        else
+        {
+            if (clustering)
+                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
+            else
+                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
+        }
+    }
+
+    private static File getTableDir(boolean simple, boolean clustering)
+    {
+        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+    }
+
+    private static void copyFile(File src, File dest) throws IOException
+    {
+        byte[] buf = new byte[65536];
+        if (src.isFile())
+        {
+            File target = new File(dest, src.getName());
+            int rd;
+            FileInputStream is = new FileInputStream(src);
+            FileOutputStream os = new FileOutputStream(target);
+            while ((rd = is.read(buf)) >= 0)
+                os.write(buf, 0, rd);
+        }
+    }
+
+    public static String getTableName(boolean simple, boolean clustering)
+    {
+        if (simple)
+            return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+        else
+            return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+    }
+}


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


[14/29] cassandra git commit: Merge branch 'cassandra-2.1' into cassandra-2.2

Posted by pa...@apache.org.
Merge branch 'cassandra-2.1' into cassandra-2.2


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

Branch: refs/heads/cassandra-3.11
Commit: 1602e606348959aead18531cb8027afb15f276e7
Parents: aa831c9 b294943
Author: Paulo Motta <pa...@apache.org>
Authored: Sat Feb 10 14:53:20 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:54:56 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  21 +
 debian/rules                                    |   2 +-
 redhat/cassandra.spec                           |   2 +-
 .../org/apache/cassandra/cql3/Attributes.java   |  79 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |  11 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  14 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |   3 +
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |   8 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 .../apache/cassandra/tools/nodetool/Scrub.java  |   8 +-
 .../table1/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table1/lb-1-big-Data.db                     | Bin 0 -> 103 bytes
 .../table1/lb-1-big-Digest.adler32              |   1 +
 .../table1/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table1/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table1/lb-1-big-Statistics.db               | Bin 0 -> 4466 bytes
 .../table1/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table1/lb-1-big-TOC.txt                     |   8 +
 .../table2/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table2/lb-1-big-Data.db                     | Bin 0 -> 98 bytes
 .../table2/lb-1-big-Digest.adler32              |   1 +
 .../table2/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table2/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table2/lb-1-big-Statistics.db               | Bin 0 -> 4478 bytes
 .../table2/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table2/lb-1-big-TOC.txt                     |   8 +
 .../table3/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table3/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table3/lb-1-big-Digest.adler32              |   1 +
 .../table3/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table3/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table3/lb-1-big-Statistics.db               | Bin 0 -> 4482 bytes
 .../table3/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table3/lb-1-big-TOC.txt                     |   8 +
 .../table4/lb-1-big-CompressionInfo.db          | Bin 0 -> 43 bytes
 .../table4/lb-1-big-Data.db                     | Bin 0 -> 182 bytes
 .../table4/lb-1-big-Digest.adler32              |   1 +
 .../table4/lb-1-big-Filter.db                   | Bin 0 -> 16 bytes
 .../table4/lb-1-big-Index.db                    | Bin 0 -> 36 bytes
 .../table4/lb-1-big-Statistics.db               | Bin 0 -> 4494 bytes
 .../table4/lb-1-big-Summary.db                  | Bin 0 -> 84 bytes
 .../table4/lb-1-big-TOC.txt                     |   8 +
 .../cql3/validation/operations/TTLTest.java     | 405 +++++++++++++++++++
 .../unit/org/apache/cassandra/db/ScrubTest.java |   2 +-
 58 files changed, 847 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 0f6e61c,9332354..82da6ad
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,9 -1,5 +1,10 @@@
 -2.1.20
 +2.2.12
 + * Fix the inspectJvmOptions startup check (CASSANDRA-14112)
 + * Fix race that prevents submitting compaction for a table when executor is full (CASSANDRA-13801)
 + * Rely on the JVM to handle OutOfMemoryErrors (CASSANDRA-13006)
 + * Grab refs during scrub/index redistribution/cleanup (CASSANDRA-13873)
 +Merged from 2.1:
+  * Protect against overflow of local expiration time (CASSANDRA-14092)
   * More PEP8 compliance for cqlsh (CASSANDRA-14021)
   * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/NEWS.txt
----------------------------------------------------------------------
diff --cc NEWS.txt
index 5747941,fb6b4ee..4fe3508
--- a/NEWS.txt
+++ b/NEWS.txt
@@@ -18,15 -38,9 +38,16 @@@ using the provided 'sstableupgrade' too
  
  Upgrading
  ---------
 -   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 -
 -2.1.19
++    - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 +    - Cassandra is now relying on the JVM options to properly shutdown on OutOfMemoryError. By default it will
 +      rely on the OnOutOfMemoryError option as the ExitOnOutOfMemoryError and CrashOnOutOfMemoryError options
 +      are not supported by the older 1.7 and 1.8 JVMs. A warning will be logged at startup if none of those JVM
 +      options are used. See CASSANDRA-13006 for more details.
 +    - Cassandra is not logging anymore by default an Heap histogram on OutOfMemoryError. To enable that behavior
 +      set the 'cassandra.printHeapHistogramOnOutOfMemoryError' System property to 'true'. See CASSANDRA-13006
 +      for more details.
 +
 +2.2.11
  ======
  
  Upgrading

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/debian/rules
----------------------------------------------------------------------
diff --cc debian/rules
index 35f5a51,70db61c..ff1d64d
--- a/debian/rules
+++ b/debian/rules
@@@ -63,7 -64,7 +63,7 @@@ binary-indep: build instal
  	dh_testroot
  	dh_installchangelogs
  	dh_installinit -u'start 50 2 3 4 5 . stop 50 0 1 6 .'
- 	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html
 -	dh_installdocs README.asc CHANGES.txt NEWS.txt
++	dh_installdocs README.asc CHANGES.txt NEWS.txt doc/cql3/CQL.css doc/cql3/CQL.html CASSANDRA-14092.txt
  	dh_installexamples tools/*.yaml
  	dh_bash-completion
  	dh_compress

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/redhat/cassandra.spec
----------------------------------------------------------------------
diff --cc redhat/cassandra.spec
index 0d4b271,a3f09b0..07c3dc5
--- a/redhat/cassandra.spec
+++ b/redhat/cassandra.spec
@@@ -114,10 -113,10 +114,10 @@@ exit 
  
  %files
  %defattr(0644,root,root,0755)
--%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt
 -%attr(755,root,root) %{_bindir}/cassandra-cli
++%doc CHANGES.txt LICENSE.txt README.asc NEWS.txt NOTICE.txt CASSANDRA-14092.txt
  %attr(755,root,root) %{_bindir}/cassandra-stress
  %attr(755,root,root) %{_bindir}/cqlsh
 +%attr(755,root,root) %{_bindir}/cqlsh.py
  %attr(755,root,root) %{_bindir}/debug-cql
  %attr(755,root,root) %{_bindir}/nodetool
  %attr(755,root,root) %{_bindir}/sstablekeys

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/cql3/Attributes.java
index 7b38e9f,23571ca..84f423a
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@@ -18,17 -18,19 +18,23 @@@
  package org.apache.cassandra.cql3;
  
  import java.nio.ByteBuffer;
 +import java.util.Collections;
+ import java.util.concurrent.TimeUnit;
  
 +import com.google.common.collect.Iterables;
+ import com.google.common.annotations.VisibleForTesting;
+ import org.slf4j.Logger;
+ import org.slf4j.LoggerFactory;
  
+ import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.cql3.functions.Function;
  import org.apache.cassandra.db.ExpiringCell;
  import org.apache.cassandra.db.marshal.Int32Type;
  import org.apache.cassandra.db.marshal.LongType;
  import org.apache.cassandra.exceptions.InvalidRequestException;
  import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.ByteBufferUtil;
+ import org.apache.cassandra.utils.NoSpamLogger;
  
  /**
   * Utility class for the Parser to gather attributes for modification

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 7e36e11,2989b9d..45908de
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@@ -905,26 -879,13 +905,26 @@@ public class ColumnFamilyStore implemen
                                                  previousFlushFailure);
              logFlush();
              Flush flush = new Flush(false);
 -            ListenableFutureTask<?> flushTask = ListenableFutureTask.create(flush, null);
 -            flushExecutor.submit(flushTask);
 -            ListenableFutureTask<?> task = ListenableFutureTask.create(flush.postFlush, null);
 -            postFlushExecutor.submit(task);
 +            ListenableFutureTask<Void> flushTask = ListenableFutureTask.create(flush, null);
 +            flushExecutor.execute(flushTask);
 +            ListenableFutureTask<ReplayPosition> task = ListenableFutureTask.create(flush.postFlush);
 +            postFlushExecutor.execute(task);
  
              @SuppressWarnings("unchecked")
-             ListenableFuture<ReplayPosition> future = 
 -            ListenableFuture<?> future = Futures.allAsList(flushTask, task);
++            ListenableFuture<ReplayPosition> future =
 +                    // If either of the two tasks errors out, resulting future must also error out.
 +                    // Combine the two futures and only return post-flush result after both have completed.
 +                    // Note that flushTask will always yield null, but Futures.allAsList is
 +                    // order preserving, which is why the transform function returns the result
 +                    // from item 1 in it's input list (i.e. what was yielded by task).
 +                    Futures.transform(Futures.allAsList(flushTask, task),
 +                                      new Function<List<Object>, ReplayPosition>()
 +                                      {
 +                                          public ReplayPosition apply(List<Object> input)
 +                                          {
 +                                              return (ReplayPosition) input.get(1);
 +                                          }
 +                                      });
              return future;
          }
      }
@@@ -1610,29 -1516,48 +1610,28 @@@
          return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
      }
  
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
-         return scrub(disableSnapshot, skipCorrupted, false, checkData, jobs);
 -        // skip snapshot creation during scrub, SEE JIRA 5891
 -        if(!disableSnapshot)
 -            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
 -        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 -    }
 -
 -    public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException
 -    {
 -        return CompactionManager.instance.performSSTableRewrite(ColumnFamilyStore.this, excludeCurrentVersion, jobs);
 -    }
 -
 -    public void markObsolete(Collection<SSTableReader> sstables, OperationType compactionType)
 -    {
 -        assert !sstables.isEmpty();
 -        data.markObsolete(sstables, compactionType);
 -    }
 -
 -    void replaceFlushed(Memtable memtable, SSTableReader sstable)
 -    {
 -        compactionStrategyWrapper.replaceFlushed(memtable, sstable);
 -    }
 -
 -    public boolean isValid()
 -    {
 -        return valid;
++        return scrub(disableSnapshot, skipCorrupted, false, checkData, reinsertOverflowedTTLRows, jobs);
      }
  
-     @VisibleForTesting
-     public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, int jobs) throws ExecutionException, InterruptedException
 -    public long getMemtableColumnsCount()
++    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean alwaysFail, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
      {
 -        return metric.memtableColumnsCount.value();
 -    }
 +        // skip snapshot creation during scrub, SEE JIRA 5891
 +        if(!disableSnapshot)
 +            snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
  
 -    public long getMemtableDataSize()
 -    {
 -        return metric.memtableOnHeapSize.value();
 -    }
 +        try
 +        {
-             return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
++            return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
 +        }
 +        catch(Throwable t)
 +        {
 +            if (!rebuildOnFailedScrub(t))
 +                throw t;
  
 -    public int getMemtableSwitchCount()
 -    {
 -        return (int) metric.memtableSwitchCount.count();
 +            return alwaysFail ? CompactionManager.AllSSTableOpStatus.ABORTED : CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 +        }
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 3350b20,6e3634a..d90abe9
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -348,9 -358,16 +348,15 @@@ public class CompactionManager implemen
          }
      }
  
 -    @Deprecated
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
 +    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
 +    throws InterruptedException, ExecutionException
      {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
 -                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    throws InterruptedException, ExecutionException
+     {
 -        assert !cfs.isIndex();
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
          {
              @Override
@@@ -360,30 -377,11 +366,30 @@@
              }
  
              @Override
 -            public void execute(SSTableReader input) throws IOException
 +            public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
+                 scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
              }
 -        }, jobs);
 +        }, jobs, OperationType.SCRUB);
 +    }
 +
 +    public AllSSTableOpStatus performVerify(final ColumnFamilyStore cfs, final boolean extendedVerify) throws InterruptedException, ExecutionException
 +    {
 +        assert !cfs.isIndex();
 +        return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
 +        {
 +            @Override
 +            public Iterable<SSTableReader> filterSSTables(LifecycleTransaction input)
 +            {
 +                return input.originals();
 +            }
 +
 +            @Override
 +            public void execute(LifecycleTransaction input) throws IOException
 +            {
 +                verifyOne(cfs, input.onlyOne(), extendedVerify);
 +            }
 +        }, 0, OperationType.VERIFY);
      }
  
      public AllSSTableOpStatus performSSTableRewrite(final ColumnFamilyStore cfs, final boolean excludeCurrentVersion, int jobs) throws InterruptedException, ExecutionException
@@@ -730,14 -717,14 +736,14 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
      {
 -        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 +        CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
 -        metrics.beginCompaction(scrubInfo);
 -        try
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
          {
 +            scrubInfo = scrubber.getScrubInfo();
 +            metrics.beginCompaction(scrubInfo);
              scrubber.scrub();
          }
          finally

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index b6b20fb,6d4537c..affee11
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -22,33 -22,30 +22,36 @@@ import java.io.*
  import java.util.*;
  
  import com.google.common.annotations.VisibleForTesting;
 +import com.google.common.base.Predicate;
 +import com.google.common.base.Predicates;
  import com.google.common.base.Throwables;
  import com.google.common.collect.AbstractIterator;
 -import com.google.common.collect.Sets;
  
  import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
  import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+ import org.apache.cassandra.db.composites.CellNames;
  import org.apache.cassandra.io.sstable.*;
 +import org.apache.cassandra.io.sstable.format.SSTableReader;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.ByteBufferUtil;
  import org.apache.cassandra.utils.JVMStabilityInspector;
  import org.apache.cassandra.utils.OutputHandler;
+ import org.apache.cassandra.utils.memory.HeapAllocator;
 +import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.concurrent.Refs;
  
  public class Scrubber implements Closeable
  {
      private final ColumnFamilyStore cfs;
      private final SSTableReader sstable;
 +    private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
 -    public final boolean validateColumns;
+     private final boolean reinsertOverflowedTTLRows;
  
      private final CompactionController controller;
      private final boolean isCommutative;
@@@ -84,20 -81,27 +88,28 @@@
      };
      private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
 +    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
+     }
+ 
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
 -        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
++        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
 -    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
 +    @SuppressWarnings("resource")
-     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData) throws IOException
++    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
 -        this.sstable = sstable;
 +        this.transaction = transaction;
 +        this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.isOffline = isOffline;
 -        this.validateColumns = checkData;
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
  
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
@@@ -322,7 -339,7 +339,7 @@@
          // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
          // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
          // to the outOfOrderRows that will be later written to a new SSTable.
-         OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, checkData),
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
++        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
                                                                cfs.metadata.comparator.onDiskAtomComparator());
          if (prevKey != null && prevKey.compareTo(key) > 0)
          {
@@@ -342,6 -359,18 +359,18 @@@
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
++    private OnDiskAtomIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
++        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
+         return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                     outputHandler,
+                                                                                     negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 6896062,d718765..e416c7b
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@@ -31,8 -32,8 +32,9 @@@ import org.apache.cassandra.cql3.Attrib
  import org.apache.cassandra.db.*;
  import org.apache.cassandra.db.context.CounterContext;
  import org.apache.cassandra.dht.IPartitioner;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
 -import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 +import org.apache.cassandra.io.sstable.format.SSTableFormat;
 +import org.apache.cassandra.io.sstable.format.SSTableWriter;
  import org.apache.cassandra.service.ActiveRepairService;
  import org.apache.cassandra.utils.CounterId;
  import org.apache.cassandra.utils.Pair;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index 1ecedac,a7a8ca7..2c9ac4d
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2611,10 -2415,16 +2611,16 @@@ public class StorageService extends Not
  
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
+         return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+     }
+ 
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                      int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 761eed6,90c0fb5..f336bcc
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -265,15 -274,10 +265,18 @@@ public interface StorageServiceMBean ex
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++
 +    /**
 +     * Verify (checksums of) the given keyspace.
 +     * If columnFamilies array is empty, all CFs are verified.
 +     *
 +     * The entire sstable will be read to ensure each cell validates if extendedVerify is true
 +     */
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
      /**
       * Rewrite all sstables to the latest version.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d735676,10e7185..8bdf9dc
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -312,9 -313,9 +313,9 @@@ public class ThriftValidatio
          if (cosc.column != null)
          {
              if (isCommutative)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
              validateColumnData(metadata, key, null, cosc.column);
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 24c5874,fcd4110..17bef02
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -236,16 -243,11 +236,16 @@@ public class NodeProbe implements AutoC
          return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
      }
  
 +    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +    }
 +
      public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
@@@ -267,22 -268,13 +267,22 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+         if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
          {
              failed = true;
 -            out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
 +            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +        }
 +    }
 +
 +    public void verify(PrintStream out, boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    {
 +        if (verify(extendedVerify, keyspaceName, columnFamilies) != 0)
 +        {
 +            failed = true;
 +            out.println("Aborted verifying at least one table in keyspace "+keyspaceName+", check server logs for more information.");
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index a486a13,59d13d5..f5e84c5
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -119,10 -115,10 +126,10 @@@ public class StandaloneScrubbe
              {
                  for (SSTableReader sstable : sstables)
                  {
 -                    try
 +                    try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
 -                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
 -                        try
 +                        txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
                          {
                              scrubber.scrub();
                          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index dafe8d1,0000000..50224a0
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -1,76 -1,0 +1,82 @@@
 +/*
 + * 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.nodetool;
 +
 +import io.airlift.command.Arguments;
 +import io.airlift.command.Command;
 +import io.airlift.command.Option;
 +
 +import java.util.ArrayList;
 +import java.util.List;
 +
 +import org.apache.cassandra.tools.NodeProbe;
 +import org.apache.cassandra.tools.NodeTool.NodeToolCmd;
++import org.apache.cassandra.tools.StandaloneScrubber;
 +
 +@Command(name = "scrub", description = "Scrub (rebuild sstables for) one or more tables")
 +public class Scrub extends NodeToolCmd
 +{
 +    @Arguments(usage = "[<keyspace> <tables>...]", description = "The keyspace followed by one or many tables")
 +    private List<String> args = new ArrayList<>();
 +
 +    @Option(title = "disable_snapshot",
 +            name = {"-ns", "--no-snapshot"},
 +            description = "Scrubbed CFs will be snapshotted first, if disableSnapshot is false. (default false)")
 +    private boolean disableSnapshot = false;
 +
 +    @Option(title = "skip_corrupted",
 +            name = {"-s", "--skip-corrupted"},
 +            description = "Skip corrupted partitions even when scrubbing counter tables. (default false)")
 +    private boolean skipCorrupted = false;
 +
 +    @Option(title = "no_validate",
 +                   name = {"-n", "--no-validate"},
 +                   description = "Do not validate columns using column validator")
 +    private boolean noValidation = false;
 +
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
++    @Option(title = "reinsert_overflowed_ttl",
++    name = {"r", "--reinsert-overflowed-ttl"},
++    description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
++    private boolean reinsertOverflowedTTL = false;
++
 +    @Override
 +    public void execute(NodeProbe probe)
 +    {
 +        List<String> keyspaces = parseOptionalKeyspace(args, probe);
 +        String[] cfnames = parseOptionalColumnFamilies(args);
 +
 +        for (String keyspace : keyspaces)
 +        {
 +            try
 +            {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 +            } catch (IllegalArgumentException e)
 +            {
 +                throw e;
 +            } catch (Exception e)
 +            {
 +                throw new RuntimeException("Error occurred during scrubbing", e);
 +            }
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-CompressionInfo.db
index 0000000,0000000..d7cc13b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Data.db
index 0000000,0000000..51213c2
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
index 0000000,0000000..d5b12df
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2292388625

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Index.db
index 0000000,0000000..3ab96ee
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Statistics.db
index 0000000,0000000..e8cc7e0
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-CompressionInfo.db
index 0000000,0000000..38373b4
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Data.db
index 0000000,0000000..762a229
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
index 0000000,0000000..ae89849
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3829731931

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Index.db
index 0000000,0000000..38a6e4c
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Statistics.db
index 0000000,0000000..64dab43
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-CompressionInfo.db
index 0000000,0000000..04a7384
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Data.db
index 0000000,0000000..33145df
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
index 0000000,0000000..2a542cd
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++3574474340

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Index.db
index 0000000,0000000..5fb34e8
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Statistics.db
index 0000000,0000000..51203ae
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-CompressionInfo.db
index 0000000,0000000..c814fef
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Data.db
index 0000000,0000000..f40e71f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
index 0000000,0000000..e6675e4
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-Digest.adler32
@@@ -1,0 -1,0 +1,1 @@@
++2405377913

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Filter.db
index 0000000,0000000..f8e53be
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Index.db
index 0000000,0000000..8291383
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Statistics.db
index 0000000,0000000..2217c2d
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-Summary.db
index 0000000,0000000..1a3f81f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
index 0000000,0000000..26c7025
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/lb-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Statistics.db
++Summary.db
++Digest.adler32
++Data.db
++Index.db
++CompressionInfo.db
++Filter.db
++TOC.txt

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 0000000,ab4ef21..b1eaac1
mode 000000,100644..100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -1,0 -1,410 +1,405 @@@
+ package org.apache.cassandra.cql3.validation.operations;
+ 
+ import java.io.File;
+ import java.io.FileInputStream;
+ import java.io.FileOutputStream;
+ import java.io.IOException;
+ 
+ import static org.junit.Assert.assertEquals;
+ import static org.junit.Assert.assertTrue;
+ import static org.junit.Assert.fail;
+ 
+ import org.apache.cassandra.cql3.Attributes;
+ import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
+ import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
+ import org.apache.cassandra.db.ExpiringCell;
+ import org.apache.cassandra.db.Keyspace;
+ import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
+ import org.junit.Test;
+ 
+ public class TTLTest extends CQLTester
+ {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
+     public static int MAX_TTL = ExpiringCell.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
+ 
+     @Test
+     public void testTTLPerRequestLimit() throws Throwable
+     {
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         // insert with low TTL should not be denied
+         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+         execute("TRUNCATE %s");
+ 
+         // insert with low TTL should not be denied
+         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", MAX_TTL + 1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("ttl is too large."));
+         }
+ 
+         try
+         {
+             execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", -1);
+             fail("Expect InvalidRequestException");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("A TTL must be greater or equal to 0"));
+         }
+     }
+ 
+ 
+     @Test
+     public void testTTLDefaultLimit() throws Throwable
+     {
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=-1");
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live cannot be smaller than 0"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live="
+                         + (MAX_TTL + 1));
+             fail("Expect Invalid schema");
+         }
+         catch (RuntimeException e)
+         {
+             assertTrue(e.getCause()
 -                        .getCause()
+                         .getMessage()
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
+         }
+ 
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
+     public void testRejectExpirationDateOverflowPolicy() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
+     {
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         checkTTLIsCapped("i");
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testCapExpirationDatePolicyPerRequest() throws Throwable
+     {
+         // Test cap policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
+ 
+         // simple column, clustering, flush
+         baseCapExpirationDateOverflowTest(true, true, true);
+         // simple column, clustering, noflush
+         baseCapExpirationDateOverflowTest(true, true, false);
+         // simple column, noclustering, flush
+         baseCapExpirationDateOverflowTest(true, false, true);
+         // simple column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(true, false, false);
+         // complex column, clustering, flush
+         baseCapExpirationDateOverflowTest(false, true, true);
+         // complex column, clustering, noflush
+         baseCapExpirationDateOverflowTest(false, true, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, true);
+         // complex column, noclustering, noflush
+         baseCapExpirationDateOverflowTest(false, false, false);
+         // complex column, noclustering, flush
+         baseCapExpirationDateOverflowTest(false, false, false);
+ 
+         // Return to previous policy
+         Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
+     @Test
+     public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        createTable(true, true);
 -        createTable(true, false);
 -        createTable(false, true);
 -        createTable(false, false);
 -
+         baseTestRecoverOverflowedExpiration(false, false);
+         baseTestRecoverOverflowedExpiration(true, false);
+         baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
+     public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
+         if (simple)
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
+                 createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
+                 createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
+     }
+ 
+     /**
+      * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
+      * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
+         UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
+      * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
+         return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
++        createTable(simple, clustering);
++
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
 -        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(getTableName(simple, clustering));
++        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getLiveSSTableCount());
++        assertEquals(0, cfs.getSSTables().size());
+ 
 -        copySSTablesToTableDir(simple, clustering);
++        copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
+             cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -            {
 -                UntypedResultSet execute = execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering)));
 -                assertRows(execute, row(1, 1, 1), row(2, 2, 2));
 -
 -            }
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
+             else
 -                assertRows(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
++                assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
 -            assertEmpty(execute(String.format("SELECT * from %s.%s", KEYSPACE, getTableName(simple, clustering))));
++            assertEmpty(execute("SELECT * from %s"));
+         }
 -        cfs.truncateBlocking(); //cleanup for next tests
+     }
+ 
 -    private void copySSTablesToTableDir(boolean simple, boolean clustering) throws IOException
++    private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(KEYSPACE).getColumnFamilyStore(getTableName(simple, clustering)).directories.getCFDirectories().iterator().next();
 -        File sourceDir = getTableDir(simple, clustering);
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering) throws Throwable
++    private static File getTableDir(String table, boolean simple, boolean clustering)
++    {
++        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
++    }
++
++    private void createTable(boolean simple, boolean clustering)
+     {
+         if (simple)
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b int, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b int)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b int)");
+         }
+         else
+         {
+             if (clustering)
 -                execute(String.format("create table %s.%s (k int, a int, b set<text>, primary key(k, a))", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
+             else
 -                execute(String.format("create table %s.%s (k int primary key, a int, b set<text>)", KEYSPACE, getTableName(simple, clustering)));
++                createTable("create table %s (k int primary key, a int, b set<text>)");
+         }
+     }
+ 
+     private static File getTableDir(boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
+ }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/1602e606/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 4cca7ff,4efd082..9b1ede4
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -654,129 -565,4 +654,129 @@@ public class ScrubTes
          assertEquals("bar", iter.next().getString("c"));
          assertEquals("boo", iter.next().getString("c"));
      }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        //If the partitioner preserves the order then SecondaryIndex uses BytesType comparator,
 +        // otherwise it uses LocalByPartitionerType
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex_preserveOrder() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(BytesType.instance);
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        setKeyComparator(new LocalByPartionerType(StorageService.getPartitioner()));
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, true);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubKeysIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testFailScrubCompositeIndex() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX2, COL_COMPOSITES_INDEX, true, false);
 +    }
 +
 +    @Test /* CASSANDRA-5174 */
 +    public void testScrubTwice() throws IOException, ExecutionException, InterruptedException
 +    {
 +        testScrubIndex(CF_INDEX1, COL_KEYS_INDEX, false, true, true);
 +    }
 +
 +    /** The SecondaryIndex class is used for custom indexes so to avoid
 +     * making a public final field into a private field with getters
 +     * and setters, we resort to this hack in order to test it properly
 +     * since it can have two values which influence the scrubbing behavior.
 +     * @param comparator - the key comparator we want to test
 +     */
 +    private void setKeyComparator(AbstractType<?> comparator)
 +    {
 +        try
 +        {
 +            Field keyComparator = SecondaryIndex.class.getDeclaredField("keyComparator");
 +            keyComparator.setAccessible(true);
 +            int modifiers = keyComparator.getModifiers();
 +            Field modifierField = keyComparator.getClass().getDeclaredField("modifiers");
 +            modifiers = modifiers & ~Modifier.FINAL;
 +            modifierField.setAccessible(true);
 +            modifierField.setInt(keyComparator, modifiers);
 +
 +            keyComparator.set(null, comparator);
 +        }
 +        catch (Exception ex)
 +        {
 +            fail("Failed to change key comparator in secondary index : " + ex.getMessage());
 +            ex.printStackTrace();
 +        }
 +    }
 +
 +    private void testScrubIndex(String cfName, String colName, boolean composite, boolean ... scrubs)
 +            throws IOException, ExecutionException, InterruptedException
 +    {
 +        CompactionManager.instance.disableAutoCompaction();
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfName);
 +        cfs.clearUnsafe();
 +
 +        int numRows = 1000;
 +        long[] colValues = new long [numRows * 2]; // each row has two columns
 +        for (int i = 0; i < colValues.length; i+=2)
 +        {
 +            colValues[i] = (i % 4 == 0 ? 1L : 2L); // index column
 +            colValues[i+1] = 3L; //other column
 +        }
 +        fillIndexCF(cfs, composite, colValues);
 +
 +        // check index
 +        IndexExpression expr = new IndexExpression(ByteBufferUtil.bytes(colName), Operator.EQ, ByteBufferUtil.bytes(1L));
 +        List<Row> rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +
 +        // scrub index
 +        Set<ColumnFamilyStore> indexCfss = cfs.indexManager.getIndexesBackedByCfs();
 +        assertTrue(indexCfss.size() == 1);
 +        for(ColumnFamilyStore indexCfs : indexCfss)
 +        {
 +            for (int i = 0; i < scrubs.length; i++)
 +            {
 +                boolean failure = !scrubs[i];
 +                if (failure)
 +                { //make sure the next scrub fails
 +                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
 +                assertEquals(failure ?
 +                             CompactionManager.AllSSTableOpStatus.ABORTED :
 +                             CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
 +                                result);
 +            }
 +        }
 +
 +
 +        // check index is still working
 +        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 +        assertNotNull(rows);
 +        assertEquals(numRows / 2, rows.size());
 +    }
  }


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


[15/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
index 0000000,0000000..45113dc
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++CompressionInfo.db
++Data.db
++Summary.db
++Filter.db
++Statistics.db
++TOC.txt
++Digest.crc32
++Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-CompressionInfo.db
index 0000000,0000000..b4de068
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Data.db
index 0000000,0000000..e96f772
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
index 0000000,0000000..459804b
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++3064924389

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Index.db
index 0000000,0000000..807a27b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Statistics.db
index 0000000,0000000..1ee01e6
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
index 0000000,0000000..f445537
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Summary.db
++TOC.txt
++Filter.db
++Index.db
++Digest.crc32
++CompressionInfo.db
++Data.db
++Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-CompressionInfo.db
index 0000000,0000000..5d22c04
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Data.db
index 0000000,0000000..a22a7a3
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
index 0000000,0000000..db7a6c7
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++1803989939

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Index.db
index 0000000,0000000..6397b5e
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Statistics.db
index 0000000,0000000..4ee9294
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
index 0000000,0000000..f445537
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++Summary.db
++TOC.txt
++Filter.db
++Index.db
++Digest.crc32
++CompressionInfo.db
++Data.db
++Statistics.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
index 9f375d4,b1eaac1..fc70974
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/TTLTest.java
@@@ -5,20 -11,33 +11,33 @@@ import static org.junit.Assert.fail
  
  import org.apache.cassandra.cql3.Attributes;
  import org.apache.cassandra.cql3.CQLTester;
+ import org.apache.cassandra.cql3.UntypedResultSet;
 -import org.apache.cassandra.db.BufferExpiringCell;
+ import org.apache.cassandra.db.ColumnFamilyStore;
 -import org.apache.cassandra.db.ExpiringCell;
++import org.apache.cassandra.db.ExpirationDateOverflowHandling;
+ import org.apache.cassandra.db.Keyspace;
++import org.apache.cassandra.db.rows.AbstractCell;
  import org.apache.cassandra.exceptions.InvalidRequestException;
+ import org.apache.cassandra.utils.FBUtilities;
+ 
  import org.junit.Test;
  
  public class TTLTest extends CQLTester
  {
+     public static String NEGATIVE_LOCAL_EXPIRATION_TEST_DIR = "test/data/negative-local-expiration-test/%s";
+ 
 -    public static int MAX_TTL = ExpiringCell.MAX_TTL;
++    public static int MAX_TTL = Attributes.MAX_TTL;
+ 
+     public static final String SIMPLE_NOCLUSTERING = "table1";
+     public static final String SIMPLE_CLUSTERING = "table2";
+     public static final String COMPLEX_NOCLUSTERING = "table3";
+     public static final String COMPLEX_CLUSTERING = "table4";
  
      @Test
      public void testTTLPerRequestLimit() throws Throwable
      {
          createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
-         // insert
-         execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", Attributes.MAX_TTL); // max ttl
-         int ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > Attributes.MAX_TTL - 10);
+         // insert with low TTL should not be denied
 -        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10); // max ttl
++        execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL ?", 10);
  
          try
          {
@@@ -41,10 -60,8 +60,8 @@@
          }
          execute("TRUNCATE %s");
  
-         // update
-         execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", Attributes.MAX_TTL); // max ttl
-         ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > Attributes.MAX_TTL - 10);
+         // insert with low TTL should not be denied
 -        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5); // max ttl
++        execute("UPDATE %s USING TTL ? SET i = 1 WHERE k = 2", 5);
  
          try
          {
@@@ -91,14 -109,297 +108,288 @@@
          {
              assertTrue(e.getCause()
                          .getMessage()
-                         .contains("default_time_to_live must be less than or equal to " + Attributes.MAX_TTL + " (got "
-                                 + (Attributes.MAX_TTL + 1) + ")"));
+                         .contains("default_time_to_live must be less than or equal to " + MAX_TTL + " (got "
+                                   + (MAX_TTL + 1) + ")"));
          }
  
-         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + Attributes.MAX_TTL);
+         // table with default low TTL should not be denied
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + 5);
+         execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+     }
+ 
+     @Test
 -    public void testRejectExpirationDateOverflowPolicy() throws Throwable
++    public void testCapWarnExpirationOverflowPolicy() throws Throwable
++    {
++        // We don't test that the actual warn is logged here, only on dtest
++        testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.CAP);
++    }
++
++    @Test
++    public void testCapNoWarnExpirationOverflowPolicy() throws Throwable
++    {
++        testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.CAP_NOWARN);
++    }
++
++    @Test
++    public void testCapNoWarnExpirationOverflowPolicyDefaultTTL() throws Throwable
++    {
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.policy.CAP_NOWARN;
++        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
 +        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
-         int ttl = execute("SELECT ttl(i) FROM %s").one().getInt("ttl(i)");
-         assertTrue(ttl > 10000 - 10); // within 10 second
++        checkTTLIsCapped("i");
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.policy.REJECT;
++    }
++
++    @Test
++    public void testRejectExpirationOverflowPolicy() throws Throwable
+     {
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
++        //ExpirationDateOverflowHandling.expirationDateOverflowPolicy = ExpirationDateOverflowHandling.expirationDateOverflowPolicy.REJECT;
+         createTable("CREATE TABLE %s (k int PRIMARY KEY, i int)");
+         try
+         {
+             execute("INSERT INTO %s (k, i) VALUES (1, 1) USING TTL " + MAX_TTL);
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+         try
+         {
+             createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
+             execute("INSERT INTO %s (k, i) VALUES (1, 1)");
+         }
+         catch (InvalidRequestException e)
+         {
+             assertTrue(e.getMessage().contains("exceeds maximum supported expiration date"));
+         }
+     }
+ 
+     @Test
 -    public void testCapExpirationDatePolicyDefaultTTL() throws Throwable
++    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
+     {
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
 -        createTable("CREATE TABLE %s (k int PRIMARY KEY, i int) WITH default_time_to_live=" + MAX_TTL);
 -        execute("INSERT INTO %s (k, i) VALUES (1, 1)");
 -        checkTTLIsCapped("i");
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
++        baseTestRecoverOverflowedExpiration(false, false);
++        baseTestRecoverOverflowedExpiration(true, false);
++        baseTestRecoverOverflowedExpiration(true, true);
+     }
+ 
 -    @Test
 -    public void testCapExpirationDatePolicyPerRequest() throws Throwable
++    public void testCapExpirationDateOverflowPolicy(ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy policy) throws Throwable
+     {
 -        // Test cap policy
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.CAP;
++        ExpirationDateOverflowHandling.policy = policy;
+ 
+         // simple column, clustering, flush
 -        baseCapExpirationDateOverflowTest(true, true, true);
++        testCapExpirationDateOverflowPolicy(true, true, true);
+         // simple column, clustering, noflush
 -        baseCapExpirationDateOverflowTest(true, true, false);
++        testCapExpirationDateOverflowPolicy(true, true, false);
+         // simple column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(true, false, true);
++        testCapExpirationDateOverflowPolicy(true, false, true);
+         // simple column, noclustering, noflush
 -        baseCapExpirationDateOverflowTest(true, false, false);
++        testCapExpirationDateOverflowPolicy(true, false, false);
+         // complex column, clustering, flush
 -        baseCapExpirationDateOverflowTest(false, true, true);
++        testCapExpirationDateOverflowPolicy(false, true, true);
+         // complex column, clustering, noflush
 -        baseCapExpirationDateOverflowTest(false, true, false);
++        testCapExpirationDateOverflowPolicy(false, true, false);
+         // complex column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(false, false, true);
++        testCapExpirationDateOverflowPolicy(false, false, true);
+         // complex column, noclustering, noflush
 -        baseCapExpirationDateOverflowTest(false, false, false);
 -        // complex column, noclustering, flush
 -        baseCapExpirationDateOverflowTest(false, false, false);
++        testCapExpirationDateOverflowPolicy(false, false, false);
+ 
+         // Return to previous policy
 -        Attributes.policy = Attributes.ExpirationDateOverflowPolicy.REJECT;
 -    }
 -
 -    @Test
 -    public void testRecoverOverflowedExpirationWithScrub() throws Throwable
 -    {
 -        baseTestRecoverOverflowedExpiration(false, false);
 -        baseTestRecoverOverflowedExpiration(true, false);
 -        baseTestRecoverOverflowedExpiration(true, true);
++        ExpirationDateOverflowHandling.policy = ExpirationDateOverflowHandling.ExpirationDateOverflowPolicy.REJECT;
+     }
+ 
 -    public void baseCapExpirationDateOverflowTest(boolean simple, boolean clustering, boolean flush) throws Throwable
++    public void testCapExpirationDateOverflowPolicy(boolean simple, boolean clustering, boolean flush) throws Throwable
+     {
+         // Create Table
 -        if (simple)
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b int, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b int)");
 -        }
 -        else
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b set<text>)");
 -        }
++        createTable(simple, clustering);
+ 
+         // Insert data with INSERT and UPDATE
+         if (simple)
+         {
 -            execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, 2);
++            execute("INSERT INTO %s (k, a) VALUES (?, ?) USING TTL " + MAX_TTL, 2, 2);
+             if (clustering)
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET b = 1 WHERE k = 1 AND a = 1;");
+             else
+                 execute("UPDATE %s USING TTL " + MAX_TTL + " SET a = 1, b = 1 WHERE k = 1;");
+         }
+         else
+         {
+             execute("INSERT INTO %s (k, a, b) VALUES (?, ?, ?) USING TTL " + MAX_TTL, 2, 2, set("v21", "v22", "v23", "v24"));
+             if (clustering)
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET b = ? WHERE k = 1 AND a = 1;", set("v11", "v12", "v13", "v14"));
+             else
+                 execute("UPDATE  %s USING TTL " + MAX_TTL + " SET a = 1, b = ? WHERE k = 1;", set("v11", "v12", "v13", "v14"));
+         }
+ 
+         // Maybe Flush
+         Keyspace ks = Keyspace.open(keyspace());
+         if (flush)
+             FBUtilities.waitOnFutures(ks.flush());
+ 
+         // Verify data
+         verifyData(simple);
+ 
+         // Maybe major compact
+         if (flush)
+         {
+             // Major compact and check data is still present
+             ks.getColumnFamilyStore(currentTable()).forceMajorCompaction();
+ 
+             // Verify data again
+             verifyData(simple);
+         }
+     }
+ 
+     public void baseTestRecoverOverflowedExpiration(boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         // simple column, clustering
+         testRecoverOverflowedExpirationWithScrub(true, true, runScrub, reinsertOverflowedTTL);
+         // simple column, noclustering
+         testRecoverOverflowedExpirationWithScrub(true, false, runScrub, reinsertOverflowedTTL);
+         // complex column, clustering
+         testRecoverOverflowedExpirationWithScrub(false, true, runScrub, reinsertOverflowedTTL);
+         // complex column, noclustering
+         testRecoverOverflowedExpirationWithScrub(false, false, runScrub, reinsertOverflowedTTL);
+     }
+ 
++    private void createTable(boolean simple, boolean clustering)
++    {
++        if (simple)
++        {
++            if (clustering)
++                createTable("create table %s (k int, a int, b int, primary key(k, a))");
++            else
++                createTable("create table %s (k int primary key, a int, b int)");
++        }
++        else
++        {
++            if (clustering)
++                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
++            else
++                createTable("create table %s (k int primary key, a int, b set<text>)");
++        }
++    }
++
+     private void verifyData(boolean simple) throws Throwable
+     {
+         if (simple)
+         {
 -            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++            assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+         }
+         else
+         {
+             assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         // Cannot retrieve TTL from collections
+         if (simple)
+             checkTTLIsCapped("b");
      }
  
+     /**
 -     * Verify that the computed TTL is approximately equal to the maximum allowed ttl given the
 -     * {@link ExpiringCell#getLocalDeletionTime()} field limitation (CASSANDRA-14092)
++     * Verify that the computed TTL is equal to the maximum allowed ttl given the
++     * {@link AbstractCell#localDeletionTime()} field limitation (CASSANDRA-14092)
+      */
+     private void checkTTLIsCapped(String field) throws Throwable
+     {
+ 
+         // TTL is computed dynamically from row expiration time, so if it is
+         // equal or higher to the minimum max TTL we compute before the query
+         // we are fine.
+         int minMaxTTL = computeMaxTTL();
 -        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s");
++        UntypedResultSet execute = execute("SELECT ttl(" + field + ") FROM %s WHERE k = 1");
+         for (UntypedResultSet.Row row : execute)
+         {
+             int ttl = row.getInt("ttl(" + field + ")");
+             assertTrue(ttl >= minMaxTTL);
+         }
+     }
+ 
+     /**
+      * The max TTL is computed such that the TTL summed with the current time is equal to the maximum
 -     * allowed expiration time {@link BufferExpiringCell#getLocalDeletionTime()} (2038-01-19T03:14:06+00:00)
++     * allowed expiration time {@link org.apache.cassandra.db.rows.Cell#MAX_DELETION_TIME} (2038-01-19T03:14:06+00:00)
+      */
+     private int computeMaxTTL()
+     {
+         int nowInSecs = (int) (System.currentTimeMillis() / 1000);
 -        return BufferExpiringCell.MAX_DELETION_TIME - nowInSecs;
++        return AbstractCell.MAX_DELETION_TIME - nowInSecs;
+     }
+ 
+     public void testRecoverOverflowedExpirationWithScrub(boolean simple, boolean clustering, boolean runScrub, boolean reinsertOverflowedTTL) throws Throwable
+     {
+         if (reinsertOverflowedTTL)
+         {
+             assert runScrub;
+         }
+ 
+         createTable(simple, clustering);
+ 
+         Keyspace keyspace = Keyspace.open(KEYSPACE);
+         ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(currentTable());
+ 
 -        assertEquals(0, cfs.getSSTables().size());
++        assertEquals(0, cfs.getLiveSSTables().size());
+ 
+         copySSTablesToTableDir(currentTable(), simple, clustering);
+ 
+         cfs.loadNewSSTables();
+ 
+         if (runScrub)
+         {
 -            cfs.scrub(true, false, false, reinsertOverflowedTTL, 1);
++            cfs.scrub(true, false, true, reinsertOverflowedTTL, 1);
+         }
+ 
+         if (reinsertOverflowedTTL)
+         {
+             if (simple)
 -                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+             else
+                 assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+ 
+             cfs.forceMajorCompaction();
+ 
+             if (simple)
 -                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, 2));
++                assertRows(execute("SELECT * from %s"), row(1, 1, 1), row(2, 2, null));
+             else
+                 assertRows(execute("SELECT * from %s"), row(1, 1, set("v11", "v12", "v13", "v14")), row(2, 2, set("v21", "v22", "v23", "v24")));
+         }
+         else
+         {
+             assertEmpty(execute("SELECT * from %s"));
+         }
+     }
+ 
+     private void copySSTablesToTableDir(String table, boolean simple, boolean clustering) throws IOException
+     {
 -        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).directories.getCFDirectories().iterator().next();
++        File destDir = Keyspace.open(keyspace()).getColumnFamilyStore(table).getDirectories().getCFDirectories().iterator().next();
+         File sourceDir = getTableDir(table, simple, clustering);
+         for (File file : sourceDir.listFiles())
+         {
+             copyFile(file, destDir);
+         }
+     }
+ 
+     private static File getTableDir(String table, boolean simple, boolean clustering)
+     {
+         return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
+     }
+ 
 -    private void createTable(boolean simple, boolean clustering)
 -    {
 -        if (simple)
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b int, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b int)");
 -        }
 -        else
 -        {
 -            if (clustering)
 -                createTable("create table %s (k int, a int, b set<text>, primary key(k, a))");
 -            else
 -                createTable("create table %s (k int primary key, a int, b set<text>)");
 -        }
 -    }
 -
 -    private static File getTableDir(boolean simple, boolean clustering)
 -    {
 -        return new File(String.format(NEGATIVE_LOCAL_EXPIRATION_TEST_DIR, getTableName(simple, clustering)));
 -    }
 -
+     private static void copyFile(File src, File dest) throws IOException
+     {
+         byte[] buf = new byte[65536];
+         if (src.isFile())
+         {
+             File target = new File(dest, src.getName());
+             int rd;
+             FileInputStream is = new FileInputStream(src);
+             FileOutputStream os = new FileOutputStream(target);
+             while ((rd = is.read(buf)) >= 0)
+                 os.write(buf, 0, rd);
+         }
+     }
+ 
+     public static String getTableName(boolean simple, boolean clustering)
+     {
+         if (simple)
+             return clustering ? SIMPLE_CLUSTERING : SIMPLE_NOCLUSTERING;
+         else
+             return clustering ? COMPLEX_CLUSTERING : COMPLEX_NOCLUSTERING;
+     }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/unit/org/apache/cassandra/db/ScrubTest.java
----------------------------------------------------------------------
diff --cc test/unit/org/apache/cassandra/db/ScrubTest.java
index 08336a1,9b1ede4..fc2faea
--- a/test/unit/org/apache/cassandra/db/ScrubTest.java
+++ b/test/unit/org/apache/cassandra/db/ScrubTest.java
@@@ -113,14 -127,18 +113,14 @@@ public class ScrubTes
          ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(CF);
          cfs.clearUnsafe();
  
 -        List<Row> rows;
 -
          // insert data and verify we get it back w/ range query
          fillCF(cfs, 1);
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
  
--        CompactionManager.instance.performScrub(cfs, false, true, 2);
++        CompactionManager.instance.performScrub(cfs, false, true, false, 2);
  
          // check data is still there
 -        rows = cfs.getRangeSlice(Util.range("", ""), null, new IdentityQueryFilter(), 1000);
 -        assertEquals(1, rows.size());
 +        assertOrderedAll(cfs, 1);
      }
  
      @Test
@@@ -617,9 -763,9 +617,9 @@@
                  boolean failure = !scrubs[i];
                  if (failure)
                  { //make sure the next scrub fails
 -                    overrideWithGarbage(indexCfs.getSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
 +                    overrideWithGarbage(indexCfs.getLiveSSTables().iterator().next(), ByteBufferUtil.bytes(1L), ByteBufferUtil.bytes(2L));
                  }
-                 CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, 0);
 -                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, true, true, true, 0);
++                CompactionManager.AllSSTableOpStatus result = indexCfs.scrub(false, false, false, true, false,0);
                  assertEquals(failure ?
                               CompactionManager.AllSSTableOpStatus.ABORTED :
                               CompactionManager.AllSSTableOpStatus.SUCCESSFUL,
@@@ -629,118 -775,8 +629,118 @@@
  
  
          // check index is still working
 -        rows = cfs.search(Util.range("", ""), Arrays.asList(expr), new IdentityQueryFilter(), numRows);
 -        assertNotNull(rows);
 -        assertEquals(numRows / 2, rows.size());
 +        assertOrdered(Util.cmd(cfs).filterOn(colName, Operator.EQ, 1L).build(), numRows / 2);
 +    }
 +
 +    private static SSTableMultiWriter createTestWriter(Descriptor descriptor, long keyCount, CFMetaData metadata, LifecycleTransaction txn)
 +    {
 +        SerializationHeader header = new SerializationHeader(true, metadata, metadata.partitionColumns(), EncodingStats.NO_STATS);
 +        MetadataCollector collector = new MetadataCollector(metadata.comparator).sstableLevel(0);
 +        return new TestMultiWriter(new TestWriter(descriptor, keyCount, 0, metadata, collector, header, txn));
 +    }
 +
 +    private static class TestMultiWriter extends SimpleSSTableMultiWriter
 +    {
 +        TestMultiWriter(SSTableWriter writer)
 +        {
 +            super(writer);
 +        }
 +    }
 +
 +    /**
 +     * Test writer that allows to write out of order SSTable.
 +     */
 +    private static class TestWriter extends BigTableWriter
 +    {
 +        TestWriter(Descriptor descriptor, long keyCount, long repairedAt, CFMetaData metadata,
 +                   MetadataCollector collector, SerializationHeader header, LifecycleTransaction txn)
 +        {
 +            super(descriptor, keyCount, repairedAt, metadata, collector, header, txn);
 +        }
 +
 +        @Override
 +        protected long beforeAppend(DecoratedKey decoratedKey)
 +        {
 +            return dataFile.position();
 +        }
 +    }
 +
 +    /**
 +     * Tests with invalid sstables (containing duplicate entries in 2.0 and 3.0 storage format),
 +     * that were caused by upgrading from 2.x with duplicate range tombstones.
 +     *
 +     * See CASSANDRA-12144 for details.
 +     */
 +    @Test
 +    public void testFilterOutDuplicates() throws Exception
 +    {
 +        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".cf_with_duplicates_3_0 (a int, b int, c int, PRIMARY KEY (a, b))", KEYSPACE), ConsistencyLevel.ONE);
 +
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore("cf_with_duplicates_3_0");
 +
 +        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
 +                                           "Keyspace1",
 +                                           "cf_with_duplicates_3_0");
 +
 +        for (String filename : new String[]{ "mb-3-big-CompressionInfo.db",
 +                                             "mb-3-big-Digest.crc32",
 +                                             "mb-3-big-Index.db",
 +                                             "mb-3-big-Summary.db",
 +                                             "mb-3-big-Data.db",
 +                                             "mb-3-big-Filter.db",
 +                                             "mb-3-big-Statistics.db",
 +                                             "mb-3-big-TOC.txt" })
 +        {
 +            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
 +        }
 +
 +        cfs.loadNewSSTables();
 +
-         cfs.scrub(true, true, true, 1);
++        cfs.scrub(true, true, false, false, false, 1);
 +
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
 +        assertEquals(1, rs.size());
 +        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".cf_with_duplicates_3_0 WHERE a=1 AND b =2", KEYSPACE));
 +        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".cf_with_duplicates_3_0", KEYSPACE));
 +        assertEquals(0, rs.size());
 +    }
 +
 +    @Test
 +    public void testUpgradeSstablesWithDuplicates() throws Exception
 +    {
 +        DatabaseDescriptor.setPartitionerUnsafe(Murmur3Partitioner.instance);
 +        String cf = "cf_with_duplicates_2_0";
 +        QueryProcessor.process(String.format("CREATE TABLE \"%s\".%s (a int, b int, c int, PRIMARY KEY (a, b))", KEYSPACE, cf), ConsistencyLevel.ONE);
 +
 +        Keyspace keyspace = Keyspace.open(KEYSPACE);
 +        ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cf);
 +
 +        Path legacySSTableRoot = Paths.get(System.getProperty(INVALID_LEGACY_SSTABLE_ROOT_PROP),
 +                                           "Keyspace1",
 +                                           cf);
 +
 +        for (String filename : new String[]{ "lb-1-big-CompressionInfo.db",
 +                                             "lb-1-big-Data.db",
 +                                             "lb-1-big-Digest.adler32",
 +                                             "lb-1-big-Filter.db",
 +                                             "lb-1-big-Index.db",
 +                                             "lb-1-big-Statistics.db",
 +                                             "lb-1-big-Summary.db",
 +                                             "lb-1-big-TOC.txt" })
 +        {
 +            Files.copy(Paths.get(legacySSTableRoot.toString(), filename), cfs.getDirectories().getDirectoryForNewSSTables().toPath().resolve(filename));
 +        }
 +
 +        cfs.loadNewSSTables();
 +
 +        cfs.sstablesRewrite(true, 1);
 +
 +        UntypedResultSet rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE, cf));
 +        assertEquals(1, rs.size());
 +        QueryProcessor.executeInternal(String.format("DELETE FROM \"%s\".%s WHERE a=1 AND b =2", KEYSPACE, cf));
 +        rs = QueryProcessor.executeInternal(String.format("SELECT * FROM \"%s\".%s", KEYSPACE, cf));
 +        assertEquals(0, rs.size());
      }
  }


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


[21/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LegacyLayout.java
index 2117dd6,0000000..912d591
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@@ -1,2504 -1,0 +1,2509 @@@
 +/*
 + * 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;
 +
 +import java.io.DataInput;
 +import java.io.IOException;
 +import java.io.IOError;
 +import java.nio.ByteBuffer;
 +import java.security.MessageDigest;
 +import java.util.*;
 +
 +import org.apache.cassandra.cql3.SuperColumnCompatibility;
++import org.apache.cassandra.thrift.Column;
 +import org.apache.cassandra.utils.AbstractIterator;
 +import com.google.common.collect.Iterators;
 +import com.google.common.collect.Lists;
 +import com.google.common.collect.PeekingIterator;
 +
 +import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
 +import org.apache.cassandra.db.filter.ColumnFilter;
 +import org.apache.cassandra.db.filter.DataLimits;
 +import org.apache.cassandra.db.rows.*;
 +import org.apache.cassandra.db.partitions.*;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.*;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.net.MessagingService;
 +import org.apache.cassandra.utils.*;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 +
 +/**
 + * Functions to deal with the old format.
 + */
 +public abstract class LegacyLayout
 +{
 +    private static final Logger logger = LoggerFactory.getLogger(LegacyLayout.class);
 +
 +    public final static int MAX_CELL_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
 +
 +    public final static int STATIC_PREFIX = 0xFFFF;
 +
 +    public final static int DELETION_MASK        = 0x01;
 +    public final static int EXPIRATION_MASK      = 0x02;
 +    public final static int COUNTER_MASK         = 0x04;
 +    public final static int COUNTER_UPDATE_MASK  = 0x08;
 +    private final static int RANGE_TOMBSTONE_MASK = 0x10;
 +
 +    private LegacyLayout() {}
 +
 +    public static AbstractType<?> makeLegacyComparator(CFMetaData metadata)
 +    {
 +        ClusteringComparator comparator = metadata.comparator;
 +        if (!metadata.isCompound())
 +        {
 +            assert comparator.size() == 1;
 +            return comparator.subtype(0);
 +        }
 +
 +        boolean hasCollections = metadata.hasCollectionColumns() || metadata.hasDroppedCollectionColumns();
 +        List<AbstractType<?>> types = new ArrayList<>(comparator.size() + (metadata.isDense() ? 0 : 1) + (hasCollections ? 1 : 0));
 +
 +        types.addAll(comparator.subtypes());
 +
 +        if (!metadata.isDense())
 +        {
 +            types.add(UTF8Type.instance);
 +
 +            if (hasCollections)
 +            {
 +                Map<ByteBuffer, CollectionType> defined = new HashMap<>();
 +
 +                for (CFMetaData.DroppedColumn def : metadata.getDroppedColumns().values())
 +                    if (def.type instanceof CollectionType && def.type.isMultiCell())
 +                        defined.put(bytes(def.name), (CollectionType) def.type);
 +
 +                for (ColumnDefinition def : metadata.partitionColumns())
 +                    if (def.type instanceof CollectionType && def.type.isMultiCell())
 +                        defined.put(def.name.bytes, (CollectionType) def.type);
 +
 +                types.add(ColumnToCollectionType.getInstance(defined));
 +            }
 +        }
 +        return CompositeType.getInstance(types);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer cellname)
 +    throws UnknownColumnException
 +    {
 +        assert cellname != null;
 +        if (metadata.isSuper())
 +        {
 +            assert superColumnName != null;
 +            return decodeForSuperColumn(metadata, new Clustering(superColumnName), cellname);
 +        }
 +
 +        assert superColumnName == null;
 +        return decodeCellName(metadata, cellname);
 +    }
 +
 +    private static LegacyCellName decodeForSuperColumn(CFMetaData metadata, Clustering clustering, ByteBuffer subcol)
 +    {
 +        ColumnDefinition def = metadata.getColumnDefinition(subcol);
 +        if (def != null)
 +        {
 +            // it's a statically defined subcolumn
 +            return new LegacyCellName(clustering, def, null);
 +        }
 +
 +        def = metadata.compactValueColumn();
 +        assert def != null && def.type instanceof MapType;
 +        return new LegacyCellName(clustering, def, subcol);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException
 +    {
 +        return decodeCellName(metadata, cellname, false);
 +    }
 +
 +    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException
 +    {
 +        Clustering clustering = decodeClustering(metadata, cellname);
 +
 +        if (metadata.isSuper())
 +            return decodeForSuperColumn(metadata, clustering, CompositeType.extractComponent(cellname, 1));
 +
 +        if (metadata.isDense() || (metadata.isCompactTable() && readAllAsDynamic))
 +            return new LegacyCellName(clustering, metadata.compactValueColumn(), null);
 +
 +        ByteBuffer column = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size()) : cellname;
 +        if (column == null)
 +        {
 +            // Tables for composite 2ndary indexes used to be compound but dense, but we've transformed them into regular tables
 +            // (non compact ones) but with no regular column (i.e. we only care about the clustering). So we'll get here
 +            // in that case, and what we want to return is basically a row marker.
 +            if (metadata.partitionColumns().isEmpty())
 +                return new LegacyCellName(clustering, null, null);
 +
 +            // Otherwise, we shouldn't get there
 +            throw new IllegalArgumentException("No column name component found in cell name");
 +        }
 +
 +        // Row marker, this is ok
 +        if (!column.hasRemaining())
 +            return new LegacyCellName(clustering, null, null);
 +
 +        ColumnDefinition def = metadata.getColumnDefinition(column);
 +        if ((def == null) || def.isPrimaryKeyColumn())
 +        {
 +            // If it's a compact table, it means the column is in fact a "dynamic" one
 +            if (metadata.isCompactTable())
 +                return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null);
 +
 +            if (def == null)
 +                throw new UnknownColumnException(metadata, column);
 +            else
 +                throw new IllegalArgumentException("Cannot add primary key column to partition update");
 +        }
 +
 +        ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null;
 +
 +        // Note that because static compact columns are translated to static defs in the new world order, we need to force a static
 +        // clustering if the definition is static (as it might not be in this case).
 +        return new LegacyCellName(def.isStatic() ? Clustering.STATIC_CLUSTERING : clustering, def, collectionElement);
 +    }
 +
 +    public static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, boolean isStart)
 +    {
 +        if (!bound.hasRemaining())
 +            return isStart ? LegacyBound.BOTTOM : LegacyBound.TOP;
 +
 +        if (!metadata.isCompound())
 +        {
 +            // The non compound case is a lot easier, in that there is no EOC nor collection to worry about, so dealing
 +            // with that first.
 +            return new LegacyBound(isStart ? Slice.Bound.inclusiveStartOf(bound) : Slice.Bound.inclusiveEndOf(bound), false, null);
 +        }
 +
 +        int clusteringSize = metadata.comparator.size();
 +
 +        List<ByteBuffer> components = CompositeType.splitName(bound);
 +        byte eoc = CompositeType.lastEOC(bound);
 +
 +        // There can be  more components than the clustering size only in the case this is the bound of a collection
 +        // range tombstone. In which case, there is exactly one more component, and that component is the name of the
 +        // collection being selected/deleted.
 +        assert components.size() <= clusteringSize || (!metadata.isCompactTable() && components.size() == clusteringSize + 1);
 +
 +        ColumnDefinition collectionName = null;
 +        if (components.size() > clusteringSize)
 +            collectionName = metadata.getColumnDefinition(components.remove(clusteringSize));
 +
 +        boolean isInclusive;
 +        if (isStart)
 +        {
 +            isInclusive = eoc <= 0;
 +        }
 +        else
 +        {
 +            isInclusive = eoc >= 0;
 +
 +            // for an end bound, if we only have a prefix of all the components and the final EOC is zero,
 +            // then it should only match up to the prefix but no further, that is, it is an inclusive bound
 +            // of the exact prefix but an exclusive bound of anything beyond it, so adding an empty
 +            // composite value ensures this behavior, see CASSANDRA-12423 for more details
 +            if (eoc == 0 && components.size() < clusteringSize)
 +            {
 +                components.add(ByteBufferUtil.EMPTY_BYTE_BUFFER);
 +                isInclusive = false;
 +            }
 +        }
 +
 +        Slice.Bound.Kind boundKind = Slice.Bound.boundKind(isStart, isInclusive);
 +        Slice.Bound sb = Slice.Bound.create(boundKind, components.toArray(new ByteBuffer[components.size()]));
 +        return new LegacyBound(sb, metadata.isCompound() && CompositeType.isStaticName(bound), collectionName);
 +    }
 +
 +    public static ByteBuffer encodeBound(CFMetaData metadata, Slice.Bound bound, boolean isStart)
 +    {
 +        if (bound == Slice.Bound.BOTTOM || bound == Slice.Bound.TOP || metadata.comparator.size() == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        ClusteringPrefix clustering = bound.clustering();
 +
 +        if (!metadata.isCompound())
 +        {
 +            assert clustering.size() == 1;
 +            return clustering.get(0);
 +        }
 +
 +        CompositeType ctype = CompositeType.getInstance(metadata.comparator.subtypes());
 +        CompositeType.Builder builder = ctype.builder();
 +        for (int i = 0; i < clustering.size(); i++)
 +            builder.add(clustering.get(i));
 +
 +        if (isStart)
 +            return bound.isInclusive() ? builder.build() : builder.buildAsEndOfRange();
 +        else
 +            return bound.isInclusive() ? builder.buildAsEndOfRange() : builder.build();
 +    }
 +
 +    public static ByteBuffer encodeCellName(CFMetaData metadata, ClusteringPrefix clustering, ByteBuffer columnName, ByteBuffer collectionElement)
 +    {
 +        boolean isStatic = clustering == Clustering.STATIC_CLUSTERING;
 +
 +        if (!metadata.isCompound())
 +        {
 +            if (isStatic)
 +                return columnName;
 +
 +            assert clustering.size() == 1 : "Expected clustering size to be 1, but was " + clustering.size();
 +            return clustering.get(0);
 +        }
 +
 +        // We use comparator.size() rather than clustering.size() because of static clusterings
 +        int clusteringSize = metadata.comparator.size();
 +        int size = clusteringSize + (metadata.isDense() ? 0 : 1) + (collectionElement == null ? 0 : 1);
 +        if (metadata.isSuper())
 +            size = clusteringSize + 1;
 +        ByteBuffer[] values = new ByteBuffer[size];
 +        for (int i = 0; i < clusteringSize; i++)
 +        {
 +            if (isStatic)
 +            {
 +                values[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +                continue;
 +            }
 +
 +            ByteBuffer v = clustering.get(i);
 +            // we can have null (only for dense compound tables for backward compatibility reasons) but that
 +            // means we're done and should stop there as far as building the composite is concerned.
 +            if (v == null)
 +                return CompositeType.build(Arrays.copyOfRange(values, 0, i));
 +
 +            values[i] = v;
 +        }
 +
 +        if (metadata.isSuper())
 +        {
 +            // We need to set the "column" (in thrift terms) name, i.e. the value corresponding to the subcomparator.
 +            // What it is depends if this a cell for a declared "static" column or a "dynamic" column part of the
 +            // super-column internal map.
 +            assert columnName != null; // This should never be null for supercolumns, see decodeForSuperColumn() above
 +            values[clusteringSize] = columnName.equals(SuperColumnCompatibility.SUPER_COLUMN_MAP_COLUMN)
 +                                   ? collectionElement
 +                                   : columnName;
 +        }
 +        else
 +        {
 +            if (!metadata.isDense())
 +                values[clusteringSize] = columnName;
 +            if (collectionElement != null)
 +                values[clusteringSize + 1] = collectionElement;
 +        }
 +
 +        return CompositeType.build(isStatic, values);
 +    }
 +
 +    public static Clustering decodeClustering(CFMetaData metadata, ByteBuffer value)
 +    {
 +        int csize = metadata.comparator.size();
 +        if (csize == 0)
 +            return Clustering.EMPTY;
 +
 +        if (metadata.isCompound() && CompositeType.isStaticName(value))
 +            return Clustering.STATIC_CLUSTERING;
 +
 +        List<ByteBuffer> components = metadata.isCompound()
 +                                    ? CompositeType.splitName(value)
 +                                    : Collections.singletonList(value);
 +
 +        return new Clustering(components.subList(0, Math.min(csize, components.size())).toArray(new ByteBuffer[csize]));
 +    }
 +
 +    public static ByteBuffer encodeClustering(CFMetaData metadata, ClusteringPrefix clustering)
 +    {
 +        if (clustering.size() == 0)
 +            return ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +
 +        if (!metadata.isCompound())
 +        {
 +            assert clustering.size() == 1;
 +            return clustering.get(0);
 +        }
 +
 +        ByteBuffer[] values = new ByteBuffer[clustering.size()];
 +        for (int i = 0; i < clustering.size(); i++)
 +            values[i] = clustering.get(i);
 +        return CompositeType.build(values);
 +    }
 +
 +    /**
 +     * The maximum number of cells to include per partition when converting to the old format.
 +     * <p>
 +     * We already apply the limit during the actual query, but for queries that counts cells and not rows (thrift queries
 +     * and distinct queries as far as old nodes are concerned), we may still include a little bit more than requested
 +     * because {@link DataLimits} always include full rows. So if the limit ends in the middle of a queried row, the
 +     * full row will be part of our result. This would confuse old nodes however so we make sure to truncate it to
 +     * what's expected before writting it on the wire.
 +     *
 +     * @param command the read commmand for which to determine the maximum cells per partition. This can be {@code null}
 +     * in which case {@code Integer.MAX_VALUE} is returned.
 +     * @return the maximum number of cells per partition that should be enforced according to the read command if
 +     * post-query limitation are in order (see above). This will be {@code Integer.MAX_VALUE} if no such limits are
 +     * necessary.
 +     */
 +    private static int maxCellsPerPartition(ReadCommand command)
 +    {
 +        if (command == null)
 +            return Integer.MAX_VALUE;
 +
 +        DataLimits limits = command.limits();
 +
 +        // There is 2 types of DISTINCT queries: those that includes only the partition key, and those that include static columns.
 +        // On old nodes, the latter expects the first row in term of CQL count, which is what we already have and there is no additional
 +        // limit to apply. The former however expect only one cell per partition and rely on it (See CASSANDRA-10762).
 +        if (limits.isDistinct())
 +            return command.columnFilter().fetchedColumns().statics.isEmpty() ? 1 : Integer.MAX_VALUE;
 +
 +        switch (limits.kind())
 +        {
 +            case THRIFT_LIMIT:
 +            case SUPER_COLUMN_COUNTING_LIMIT:
 +                return limits.perPartitionCount();
 +            default:
 +                return Integer.MAX_VALUE;
 +        }
 +    }
 +
 +    // For serializing to old wire format
 +    public static LegacyUnfilteredPartition fromUnfilteredRowIterator(ReadCommand command, UnfilteredRowIterator iterator)
 +    {
 +        // we need to extract the range tombstone so materialize the partition. Since this is
 +        // used for the on-wire format, this is not worst than it used to be.
 +        final ImmutableBTreePartition partition = ImmutableBTreePartition.create(iterator);
 +        DeletionInfo info = partition.deletionInfo();
 +        Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> pair = fromRowIterator(partition.metadata(), partition.iterator(), partition.staticRow());
 +
 +        LegacyLayout.LegacyRangeTombstoneList rtl = pair.left;
 +
 +        // Processing the cell iterator results in the LegacyRangeTombstoneList being populated, so we do this
 +        // before we use the LegacyRangeTombstoneList at all
 +        List<LegacyLayout.LegacyCell> cells = Lists.newArrayList(pair.right);
 +
 +        int maxCellsPerPartition = maxCellsPerPartition(command);
 +        if (cells.size() > maxCellsPerPartition)
 +            cells = cells.subList(0, maxCellsPerPartition);
 +
 +        // The LegacyRangeTombstoneList already has range tombstones for the single-row deletions and complex
 +        // deletions.  Go through our normal range tombstones and add then to the LegacyRTL so that the range
 +        // tombstones all get merged and sorted properly.
 +        if (info.hasRanges())
 +        {
 +            Iterator<RangeTombstone> rangeTombstoneIterator = info.rangeIterator(false);
 +            while (rangeTombstoneIterator.hasNext())
 +            {
 +                RangeTombstone rt = rangeTombstoneIterator.next();
 +                Slice slice = rt.deletedSlice();
 +                LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(slice.start(), false, null);
 +                LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(slice.end(), false, null);
 +                rtl.add(start, end, rt.deletionTime().markedForDeleteAt(), rt.deletionTime().localDeletionTime());
 +            }
 +        }
 +
 +        return new LegacyUnfilteredPartition(info.getPartitionDeletion(), rtl, cells);
 +    }
 +
 +    public static void serializeAsLegacyPartition(ReadCommand command, UnfilteredRowIterator partition, DataOutputPlus out, int version) throws IOException
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        out.writeBoolean(true);
 +
 +        LegacyLayout.LegacyUnfilteredPartition legacyPartition = LegacyLayout.fromUnfilteredRowIterator(command, partition);
 +
 +        UUIDSerializer.serializer.serialize(partition.metadata().cfId, out, version);
 +        DeletionTime.serializer.serialize(legacyPartition.partitionDeletion, out);
 +
 +        legacyPartition.rangeTombstones.serialize(out, partition.metadata());
 +
 +        // begin cell serialization
 +        out.writeInt(legacyPartition.cells.size());
 +        for (LegacyLayout.LegacyCell cell : legacyPartition.cells)
 +        {
 +            ByteBufferUtil.writeWithShortLength(cell.name.encode(partition.metadata()), out);
 +            out.writeByte(cell.serializationFlags());
 +            if (cell.isExpiring())
 +            {
 +                out.writeInt(cell.ttl);
 +                out.writeInt(cell.localDeletionTime);
 +            }
 +            else if (cell.isTombstone())
 +            {
 +                out.writeLong(cell.timestamp);
 +                out.writeInt(TypeSizes.sizeof(cell.localDeletionTime));
 +                out.writeInt(cell.localDeletionTime);
 +                continue;
 +            }
 +            else if (cell.isCounterUpdate())
 +            {
 +                out.writeLong(cell.timestamp);
 +                long count = CounterContext.instance().getLocalCount(cell.value);
 +                ByteBufferUtil.writeWithLength(ByteBufferUtil.bytes(count), out);
 +                continue;
 +            }
 +            else if (cell.isCounter())
 +            {
 +                out.writeLong(Long.MIN_VALUE);  // timestampOfLastDelete (not used, and MIN_VALUE is the default)
 +            }
 +
 +            out.writeLong(cell.timestamp);
 +            ByteBufferUtil.writeWithLength(cell.value, out);
 +        }
 +    }
 +
 +    // For the old wire format
 +    // Note: this can return null if an empty partition is serialized!
 +    public static UnfilteredRowIterator deserializeLegacyPartition(DataInputPlus in, int version, SerializationHelper.Flag flag, ByteBuffer key) throws IOException
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        // This is only used in mutation, and mutation have never allowed "null" column families
 +        boolean present = in.readBoolean();
 +        if (!present)
 +            return null;
 +
 +        CFMetaData metadata = CFMetaData.serializer.deserialize(in, version);
 +        LegacyDeletionInfo info = LegacyDeletionInfo.deserialize(metadata, in);
 +        int size = in.readInt();
 +        Iterator<LegacyCell> cells = deserializeCells(metadata, in, flag, size);
 +        SerializationHelper helper = new SerializationHelper(metadata, version, flag);
 +        return onWireCellstoUnfilteredRowIterator(metadata, metadata.partitioner.decorateKey(key), info, cells, false, helper);
 +    }
 +
 +    // For the old wire format
 +    public static long serializedSizeAsLegacyPartition(ReadCommand command, UnfilteredRowIterator partition, int version)
 +    {
 +        assert version < MessagingService.VERSION_30;
 +
 +        if (partition.isEmpty())
 +            return TypeSizes.sizeof(false);
 +
 +        long size = TypeSizes.sizeof(true);
 +
 +        LegacyLayout.LegacyUnfilteredPartition legacyPartition = LegacyLayout.fromUnfilteredRowIterator(command, partition);
 +
 +        size += UUIDSerializer.serializer.serializedSize(partition.metadata().cfId, version);
 +        size += DeletionTime.serializer.serializedSize(legacyPartition.partitionDeletion);
 +        size += legacyPartition.rangeTombstones.serializedSize(partition.metadata());
 +
 +        // begin cell serialization
 +        size += TypeSizes.sizeof(legacyPartition.cells.size());
 +        for (LegacyLayout.LegacyCell cell : legacyPartition.cells)
 +        {
 +            size += ByteBufferUtil.serializedSizeWithShortLength(cell.name.encode(partition.metadata()));
 +            size += 1;  // serialization flags
 +            if (cell.isExpiring())
 +            {
 +                size += TypeSizes.sizeof(cell.ttl);
 +                size += TypeSizes.sizeof(cell.localDeletionTime);
 +            }
 +            else if (cell.isTombstone())
 +            {
 +                size += TypeSizes.sizeof(cell.timestamp);
 +                // localDeletionTime replaces cell.value as the body
 +                size += TypeSizes.sizeof(TypeSizes.sizeof(cell.localDeletionTime));
 +                size += TypeSizes.sizeof(cell.localDeletionTime);
 +                continue;
 +            }
 +            else if (cell.isCounterUpdate())
 +            {
 +                size += TypeSizes.sizeof(cell.timestamp);
 +                long count = CounterContext.instance().getLocalCount(cell.value);
 +                size += ByteBufferUtil.serializedSizeWithLength(ByteBufferUtil.bytes(count));
 +                continue;
 +            }
 +            else if (cell.isCounter())
 +            {
 +                size += TypeSizes.sizeof(Long.MIN_VALUE);  // timestampOfLastDelete
 +            }
 +
 +            size += TypeSizes.sizeof(cell.timestamp);
 +            size += ByteBufferUtil.serializedSizeWithLength(cell.value);
 +        }
 +
 +        return size;
 +    }
 +
 +    // For thrift sake
 +    public static UnfilteredRowIterator toUnfilteredRowIterator(CFMetaData metadata,
 +                                                                DecoratedKey key,
 +                                                                LegacyDeletionInfo delInfo,
 +                                                                Iterator<LegacyCell> cells)
 +    {
 +        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
 +        return toUnfilteredRowIterator(metadata, key, delInfo, cells, false, helper);
 +    }
 +
 +    // For deserializing old wire format
 +    public static UnfilteredRowIterator onWireCellstoUnfilteredRowIterator(CFMetaData metadata,
 +                                                                           DecoratedKey key,
 +                                                                           LegacyDeletionInfo delInfo,
 +                                                                           Iterator<LegacyCell> cells,
 +                                                                           boolean reversed,
 +                                                                           SerializationHelper helper)
 +    {
 +
 +        // If the table is a static compact, the "column_metadata" are now internally encoded as
 +        // static. This has already been recognized by decodeCellName, but it means the cells
 +        // provided are not in the expected order (the "static" cells are not necessarily at the front).
 +        // So sort them to make sure toUnfilteredRowIterator works as expected.
 +        // Further, if the query is reversed, then the on-wire format still has cells in non-reversed
 +        // order, but we need to have them reverse in the final UnfilteredRowIterator. So reverse them.
 +        if (metadata.isStaticCompactTable() || reversed)
 +        {
 +            List<LegacyCell> l = new ArrayList<>();
 +            Iterators.addAll(l, cells);
 +            Collections.sort(l, legacyCellComparator(metadata, reversed));
 +            cells = l.iterator();
 +        }
 +
 +        return toUnfilteredRowIterator(metadata, key, delInfo, cells, reversed, helper);
 +    }
 +
 +    private static UnfilteredRowIterator toUnfilteredRowIterator(CFMetaData metadata,
 +                                                                 DecoratedKey key,
 +                                                                 LegacyDeletionInfo delInfo,
 +                                                                 Iterator<LegacyCell> cells,
 +                                                                 boolean reversed,
 +                                                                 SerializationHelper helper)
 +    {
 +        // A reducer that basically does nothing, we know the 2 merged iterators can't have conflicting atoms (since we merge cells with range tombstones).
 +        MergeIterator.Reducer<LegacyAtom, LegacyAtom> reducer = new MergeIterator.Reducer<LegacyAtom, LegacyAtom>()
 +        {
 +            private LegacyAtom atom;
 +
 +            public void reduce(int idx, LegacyAtom current)
 +            {
 +                // We're merging cell with range tombstones, so we should always only have a single atom to reduce.
 +                assert atom == null;
 +                atom = current;
 +            }
 +
 +            protected LegacyAtom getReduced()
 +            {
 +                return atom;
 +            }
 +
 +            protected void onKeyChange()
 +            {
 +                atom = null;
 +            }
 +        };
 +        List<Iterator<LegacyAtom>> iterators = Arrays.asList(asLegacyAtomIterator(cells), asLegacyAtomIterator(delInfo.inRowRangeTombstones()));
 +        PeekingIterator<LegacyAtom> atoms = Iterators.peekingIterator(MergeIterator.get(iterators, legacyAtomComparator(metadata), reducer));
 +
 +        // Check if we have some static
 +        Row staticRow = atoms.hasNext() && atoms.peek().isStatic()
 +                      ? getNextRow(CellGrouper.staticGrouper(metadata, helper), atoms)
 +                      : Rows.EMPTY_STATIC_ROW;
 +
 +        Iterator<Row> rows = convertToRows(new CellGrouper(metadata, helper), atoms);
 +        Iterator<RangeTombstone> ranges = delInfo.deletionInfo.rangeIterator(reversed);
 +        return new RowAndDeletionMergeIterator(metadata,
 +                                               key,
 +                                               delInfo.deletionInfo.getPartitionDeletion(),
 +                                               ColumnFilter.all(metadata),
 +                                               staticRow,
 +                                               reversed,
 +                                               EncodingStats.NO_STATS,
 +                                               rows,
 +                                               ranges,
 +                                               true);
 +    }
 +
 +    public static Row extractStaticColumns(CFMetaData metadata, DataInputPlus in, Columns statics) throws IOException
 +    {
 +        assert !statics.isEmpty();
 +        assert metadata.isCompactTable();
 +
 +        if (metadata.isSuper())
 +            // TODO: there is in practice nothing to do here, but we need to handle the column_metadata for super columns somewhere else
 +            throw new UnsupportedOperationException();
 +
 +        Set<ByteBuffer> columnsToFetch = new HashSet<>(statics.size());
 +        for (ColumnDefinition column : statics)
 +            columnsToFetch.add(column.name.bytes);
 +
 +        Row.Builder builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
 +        builder.newRow(Clustering.STATIC_CLUSTERING);
 +
 +        boolean foundOne = false;
 +        LegacyAtom atom;
 +        while ((atom = readLegacyAtomSkippingUnknownColumn(metadata,in)) != null)
 +        {
 +            if (atom.isCell())
 +            {
 +                LegacyCell cell = atom.asCell();
 +                if (!columnsToFetch.contains(cell.name.encode(metadata)))
 +                    continue;
 +
 +                foundOne = true;
 +                builder.addCell(new BufferCell(cell.name.column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, null));
 +            }
 +            else
 +            {
 +                LegacyRangeTombstone tombstone = atom.asRangeTombstone();
 +                // TODO: we need to track tombstones and potentially ignore cells that are
 +                // shadowed (or even better, replace them by tombstones).
 +                throw new UnsupportedOperationException();
 +            }
 +        }
 +
 +        return foundOne ? builder.build() : Rows.EMPTY_STATIC_ROW;
 +    }
 +
 +    private static LegacyAtom readLegacyAtomSkippingUnknownColumn(CFMetaData metadata, DataInputPlus in)
 +    throws IOException
 +    {
 +        while (true)
 +        {
 +            try
 +            {
 +                return readLegacyAtom(metadata, in, false);
 +            }
 +            catch (UnknownColumnException e)
 +            {
 +                // Simply skip, as the method name implies.
 +            }
 +        }
 +
 +    }
 +
 +    private static Row getNextRow(CellGrouper grouper, PeekingIterator<? extends LegacyAtom> cells)
 +    {
 +        if (!cells.hasNext())
 +            return null;
 +
 +        grouper.reset();
 +        while (cells.hasNext() && grouper.addAtom(cells.peek()))
 +        {
 +            // We've added the cell already in the grouper, so just skip it
 +            cells.next();
 +        }
 +        return grouper.getRow();
 +    }
 +
 +    @SuppressWarnings("unchecked")
 +    private static Iterator<LegacyAtom> asLegacyAtomIterator(Iterator<? extends LegacyAtom> iter)
 +    {
 +        return (Iterator<LegacyAtom>)iter;
 +    }
 +
 +    private static Iterator<Row> convertToRows(final CellGrouper grouper, final PeekingIterator<LegacyAtom> atoms)
 +    {
 +        return new AbstractIterator<Row>()
 +        {
 +            protected Row computeNext()
 +            {
 +                if (!atoms.hasNext())
 +                    return endOfData();
 +
 +                return getNextRow(grouper, atoms);
 +            }
 +        };
 +    }
 +
 +    public static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRowIterator(final RowIterator iterator)
 +    {
 +        return fromRowIterator(iterator.metadata(), iterator, iterator.staticRow());
 +    }
 +
 +    private static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRowIterator(final CFMetaData metadata, final Iterator<Row> iterator, final Row staticRow)
 +    {
 +        LegacyRangeTombstoneList deletions = new LegacyRangeTombstoneList(new LegacyBoundComparator(metadata.comparator), 10);
 +        Iterator<LegacyCell> cells = new AbstractIterator<LegacyCell>()
 +        {
 +            private Iterator<LegacyCell> currentRow = initializeRow();
 +
 +            private Iterator<LegacyCell> initializeRow()
 +            {
 +                if (staticRow == null || staticRow.isEmpty())
 +                    return Collections.<LegacyLayout.LegacyCell>emptyIterator();
 +
 +                Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> row = fromRow(metadata, staticRow);
 +                deletions.addAll(row.left);
 +                return row.right;
 +            }
 +
 +            protected LegacyCell computeNext()
 +            {
 +                while (true)
 +                {
 +                    if (currentRow.hasNext())
 +                        return currentRow.next();
 +
 +                    if (!iterator.hasNext())
 +                        return endOfData();
 +
 +                    Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> row = fromRow(metadata, iterator.next());
 +                    deletions.addAll(row.left);
 +                    currentRow = row.right;
 +                }
 +            }
 +        };
 +
 +        return Pair.create(deletions, cells);
 +    }
 +
 +    private static Pair<LegacyRangeTombstoneList, Iterator<LegacyCell>> fromRow(final CFMetaData metadata, final Row row)
 +    {
 +        // convert any complex deletions or row deletion into normal range tombstones so that we can build and send a proper RangeTombstoneList
 +        // to legacy nodes
 +        LegacyRangeTombstoneList deletions = new LegacyRangeTombstoneList(new LegacyBoundComparator(metadata.comparator), 10);
 +
 +        if (!row.deletion().isLive())
 +        {
 +            Clustering clustering = row.clustering();
 +            Slice.Bound startBound = Slice.Bound.inclusiveStartOf(clustering);
 +            Slice.Bound endBound = Slice.Bound.inclusiveEndOf(clustering);
 +
 +            LegacyBound start = new LegacyLayout.LegacyBound(startBound, false, null);
 +            LegacyBound end = new LegacyLayout.LegacyBound(endBound, false, null);
 +
 +            deletions.add(start, end, row.deletion().time().markedForDeleteAt(), row.deletion().time().localDeletionTime());
 +        }
 +
 +        for (ColumnData cd : row)
 +        {
 +            ColumnDefinition col = cd.column();
 +            if (col.isSimple())
 +                continue;
 +
 +            DeletionTime delTime = ((ComplexColumnData)cd).complexDeletion();
 +            if (!delTime.isLive())
 +            {
 +                Clustering clustering = row.clustering();
 +
 +                Slice.Bound startBound = Slice.Bound.inclusiveStartOf(clustering);
 +                Slice.Bound endBound = Slice.Bound.inclusiveEndOf(clustering);
 +
 +                LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(startBound, col.isStatic(), col);
 +                LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(endBound, col.isStatic(), col);
 +
 +                deletions.add(start, end, delTime.markedForDeleteAt(), delTime.localDeletionTime());
 +            }
 +        }
 +
 +        Iterator<LegacyCell> cells = new AbstractIterator<LegacyCell>()
 +        {
 +            private final Iterator<Cell> cells = row.cellsInLegacyOrder(metadata, false).iterator();
 +            // we don't have (and shouldn't have) row markers for compact tables.
 +            private boolean hasReturnedRowMarker = metadata.isCompactTable();
 +
 +            protected LegacyCell computeNext()
 +            {
 +                if (!hasReturnedRowMarker)
 +                {
 +                    hasReturnedRowMarker = true;
 +
 +                    // don't include a row marker if there's no timestamp on the primary key; this is the 3.0+ equivalent
 +                    // of a row marker
 +                    if (!row.primaryKeyLivenessInfo().isEmpty())
 +                    {
 +                        LegacyCellName cellName = new LegacyCellName(row.clustering(), null, null);
 +                        LivenessInfo info = row.primaryKeyLivenessInfo();
 +                        return new LegacyCell(info.isExpiring() ? LegacyCell.Kind.EXPIRING : LegacyCell.Kind.REGULAR, cellName, ByteBufferUtil.EMPTY_BYTE_BUFFER, info.timestamp(), info.localExpirationTime(), info.ttl());
 +                    }
 +                }
 +
 +                if (!cells.hasNext())
 +                    return endOfData();
 +
 +                return makeLegacyCell(row.clustering(), cells.next());
 +            }
 +        };
 +        return Pair.create(deletions, cells);
 +    }
 +
 +    private static LegacyCell makeLegacyCell(Clustering clustering, Cell cell)
 +    {
 +        LegacyCell.Kind kind;
 +        if (cell.isCounterCell())
 +            kind = LegacyCell.Kind.COUNTER;
 +        else if (cell.isTombstone())
 +            kind = LegacyCell.Kind.DELETED;
 +        else if (cell.isExpiring())
 +            kind = LegacyCell.Kind.EXPIRING;
 +        else
 +            kind = LegacyCell.Kind.REGULAR;
 +
 +        CellPath path = cell.path();
 +        assert path == null || path.size() == 1;
 +        LegacyCellName name = new LegacyCellName(clustering, cell.column(), path == null ? null : path.get(0));
 +        return new LegacyCell(kind, name, cell.value(), cell.timestamp(), cell.localDeletionTime(), cell.ttl());
 +    }
 +
 +    public static RowIterator toRowIterator(final CFMetaData metadata,
 +                                            final DecoratedKey key,
 +                                            final Iterator<LegacyCell> cells,
 +                                            final int nowInSec)
 +    {
 +        SerializationHelper helper = new SerializationHelper(metadata, 0, SerializationHelper.Flag.LOCAL);
 +        return UnfilteredRowIterators.filter(toUnfilteredRowIterator(metadata, key, LegacyDeletionInfo.live(), cells, false, helper), nowInSec);
 +    }
 +
 +    public static Comparator<LegacyCell> legacyCellComparator(CFMetaData metadata)
 +    {
 +        return legacyCellComparator(metadata, false);
 +    }
 +
 +    public static Comparator<LegacyCell> legacyCellComparator(final CFMetaData metadata, final boolean reversed)
 +    {
 +        final Comparator<LegacyCellName> cellNameComparator = legacyCellNameComparator(metadata, reversed);
 +        return new Comparator<LegacyCell>()
 +        {
 +            public int compare(LegacyCell cell1, LegacyCell cell2)
 +            {
 +                LegacyCellName c1 = cell1.name;
 +                LegacyCellName c2 = cell2.name;
 +
 +                int c = cellNameComparator.compare(c1, c2);
 +                if (c != 0)
 +                    return c;
 +
 +                // The actual sorting when the cellname is equal doesn't matter, we just want to make
 +                // sure the cells are not considered equal.
 +                if (cell1.timestamp != cell2.timestamp)
 +                    return cell1.timestamp < cell2.timestamp ? -1 : 1;
 +
 +                if (cell1.localDeletionTime != cell2.localDeletionTime)
 +                    return cell1.localDeletionTime < cell2.localDeletionTime ? -1 : 1;
 +
 +                return cell1.value.compareTo(cell2.value);
 +            }
 +        };
 +    }
 +
 +    // Note that this doesn't exactly compare cells as they were pre-3.0 because within a row they sort columns like
 +    // in 3.0, that is, with simple columns before complex columns. In other words, this comparator makes sure cells
 +    // are in the proper order to convert them to actual 3.0 rows.
 +    public static Comparator<LegacyCellName> legacyCellNameComparator(final CFMetaData metadata, final boolean reversed)
 +    {
 +        return new Comparator<LegacyCellName>()
 +        {
 +            public int compare(LegacyCellName c1, LegacyCellName c2)
 +            {
 +                // Compare clustering first
 +                if (c1.clustering == Clustering.STATIC_CLUSTERING)
 +                {
 +                    if (c2.clustering != Clustering.STATIC_CLUSTERING)
 +                        return -1;
 +                }
 +                else if (c2.clustering == Clustering.STATIC_CLUSTERING)
 +                {
 +                    return 1;
 +                }
 +                else
 +                {
 +                    int c = metadata.comparator.compare(c1.clustering, c2.clustering);
 +                    if (c != 0)
 +                        return reversed ? -c : c;
 +                }
 +
 +                // Note that when reversed, we only care about the clustering being reversed, so it's ok
 +                // not to take reversed into account below.
 +
 +                // Then check the column name
 +                if (c1.column != c2.column)
 +                {
 +                    // A null for the column means it's a row marker
 +                    if (c1.column == null)
 +                        return -1;
 +                    if (c2.column == null)
 +                        return 1;
 +
 +                    assert c1.column.isRegular() || c1.column.isStatic();
 +                    assert c2.column.isRegular() || c2.column.isStatic();
 +                    int cmp = c1.column.compareTo(c2.column);
 +                    if (cmp != 0)
 +                        return cmp;
 +                }
 +
 +                assert (c1.collectionElement == null) == (c2.collectionElement == null);
 +
 +                if (c1.collectionElement != null)
 +                {
 +                    AbstractType<?> colCmp = ((CollectionType)c1.column.type).nameComparator();
 +                    return colCmp.compare(c1.collectionElement, c2.collectionElement);
 +                }
 +                return 0;
 +            }
 +        };
 +    }
 +
 +    private static boolean equalValues(ClusteringPrefix c1, ClusteringPrefix c2, ClusteringComparator comparator)
 +    {
 +        assert c1.size() == c2.size();
 +        for (int i = 0; i < c1.size(); i++)
 +        {
 +            if (comparator.compareComponent(i, c1.get(i), c2.get(i)) != 0)
 +                return false;
 +        }
 +        return true;
 +    }
 +
 +    private static Comparator<LegacyAtom> legacyAtomComparator(CFMetaData metadata)
 +    {
 +        return (o1, o2) ->
 +        {
 +            // First we want to compare by clustering, but we have to be careful with range tombstone, because
 +            // we can have collection deletion and we want those to sort properly just before the column they
 +            // delete, not before the whole row.
 +            // We also want to special case static so they sort before any non-static. Note in particular that
 +            // this special casing is important in the case of one of the Atom being Slice.Bound.BOTTOM: we want
 +            // it to sort after the static as we deal with static first in toUnfilteredAtomIterator and having
 +            // Slice.Bound.BOTTOM first would mess that up (note that static deletion is handled through a specific
 +            // static tombstone, see LegacyDeletionInfo.add()).
 +            if (o1.isStatic() != o2.isStatic())
 +                return o1.isStatic() ? -1 : 1;
 +
 +            ClusteringPrefix c1 = o1.clustering();
 +            ClusteringPrefix c2 = o2.clustering();
 +
 +            int clusteringComparison;
 +            if (c1.size() != c2.size() || (o1.isCell() == o2.isCell()) || !equalValues(c1, c2, metadata.comparator))
 +            {
 +                clusteringComparison = metadata.comparator.compare(c1, c2);
 +            }
 +            else
 +            {
 +                // one is a cell and one is a range tombstone, and both have the same prefix size (that is, the
 +                // range tombstone is either a row deletion or a collection deletion).
 +                LegacyRangeTombstone rt = o1.isCell() ? o2.asRangeTombstone() : o1.asRangeTombstone();
 +                clusteringComparison = rt.isCollectionTombstone()
 +                                       ? 0
 +                                       : metadata.comparator.compare(c1, c2);
 +            }
 +
 +            // Note that if both are range tombstones and have the same clustering, then they are equal.
 +            if (clusteringComparison != 0)
 +                return clusteringComparison;
 +
 +            if (o1.isCell())
 +            {
 +                LegacyCell cell1 = o1.asCell();
 +                if (o2.isCell())
 +                {
 +                    LegacyCell cell2 = o2.asCell();
 +                    // Check for row marker cells
 +                    if (cell1.name.column == null)
 +                        return cell2.name.column == null ? 0 : -1;
 +                    return cell2.name.column == null ? 1 : cell1.name.column.compareTo(cell2.name.column);
 +                }
 +
 +                LegacyRangeTombstone rt2 = o2.asRangeTombstone();
 +                assert rt2.isCollectionTombstone(); // otherwise, we shouldn't have got a clustering equality
 +                if (cell1.name.column == null)
 +                    return -1;
 +                int cmp = cell1.name.column.compareTo(rt2.start.collectionName);
 +                // If both are for the same column, then the RT should come first
 +                return cmp == 0 ? 1 : cmp;
 +            }
 +            else
 +            {
 +                assert o2.isCell();
 +                LegacyCell cell2 = o2.asCell();
 +
 +                LegacyRangeTombstone rt1 = o1.asRangeTombstone();
 +                assert rt1.isCollectionTombstone(); // otherwise, we shouldn't have got a clustering equality
 +
 +                if (cell2.name.column == null)
 +                    return 1;
 +
 +                int cmp = rt1.start.collectionName.compareTo(cell2.name.column);
 +                // If both are for the same column, then the RT should come first
 +                return cmp == 0 ? -1 : cmp;
 +            }
 +        };
 +    }
 +
 +    public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, boolean readAllAsDynamic)
 +    throws IOException, UnknownColumnException
 +    {
 +        ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
 +        if (!cellname.hasRemaining())
 +            return null; // END_OF_ROW
 +
 +        try
 +        {
 +            int b = in.readUnsignedByte();
 +            return (b & RANGE_TOMBSTONE_MASK) != 0
 +                   ? readLegacyRangeTombstoneBody(metadata, in, cellname)
 +                   : readLegacyCellBody(metadata, in, cellname, b, SerializationHelper.Flag.LOCAL, readAllAsDynamic);
 +        }
 +        catch (UnknownColumnException e)
 +        {
 +            // We legitimately can get here in 2 cases:
 +            // 1) for system tables, because we've unceremoniously removed columns (without registering them as dropped)
 +            // 2) for dropped columns.
 +            // In any other case, there is a mismatch between the schema and the data, and we complain loudly in
 +            // that case. Note that if we are in a legit case of an unknown column, we want to simply skip that cell,
 +            // but we don't do this here and re-throw the exception because the calling code sometimes has to know
 +            // about this happening. This does mean code calling this method should handle this case properly.
 +            if (!metadata.ksName.equals(SystemKeyspace.NAME) && metadata.getDroppedColumnDefinition(e.columnName) == null)
 +                throw new IllegalStateException(String.format("Got cell for unknown column %s in sstable of %s.%s: " +
 +                                                              "This suggest a problem with the schema which doesn't list " +
 +                                                              "this column. Even if that column was dropped, it should have " +
 +                                                              "been listed as such", metadata.ksName, metadata.cfName, UTF8Type.instance.compose(e.columnName)), e);
 +
 +            throw e;
 +        }
 +    }
 +
 +    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException
 +    {
 +        ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
 +        int b = in.readUnsignedByte();
 +        return readLegacyCellBody(metadata, in, cellname, b, flag, false);
 +    }
 +
 +    public static LegacyCell readLegacyCellBody(CFMetaData metadata, DataInput in, ByteBuffer cellname, int mask, SerializationHelper.Flag flag, boolean readAllAsDynamic)
 +    throws IOException, UnknownColumnException
 +    {
 +        // Note that we want to call decodeCellName only after we've deserialized other parts, since it can throw
 +        // and we want to throw only after having deserialized the full cell.
 +        if ((mask & COUNTER_MASK) != 0)
 +        {
 +            in.readLong(); // timestampOfLastDelete: this has been unused for a long time so we ignore it
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            if (flag == SerializationHelper.Flag.FROM_REMOTE || (flag == SerializationHelper.Flag.LOCAL && CounterContext.instance().shouldClearLocal(value)))
 +                value = CounterContext.instance().clearAllLocal(value);
 +            return new LegacyCell(LegacyCell.Kind.COUNTER, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +        else if ((mask & EXPIRATION_MASK) != 0)
 +        {
 +            int ttl = in.readInt();
 +            int expiration = in.readInt();
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            return new LegacyCell(LegacyCell.Kind.EXPIRING, decodeCellName(metadata, cellname, readAllAsDynamic), value, ts, expiration, ttl);
 +        }
 +        else
 +        {
 +            long ts = in.readLong();
 +            ByteBuffer value = ByteBufferUtil.readWithLength(in);
 +            LegacyCellName name = decodeCellName(metadata, cellname, readAllAsDynamic);
 +            return (mask & COUNTER_UPDATE_MASK) != 0
 +                ? new LegacyCell(LegacyCell.Kind.COUNTER, name, CounterContext.instance().createUpdate(ByteBufferUtil.toLong(value)), ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
 +                : ((mask & DELETION_MASK) == 0
 +                        ? new LegacyCell(LegacyCell.Kind.REGULAR, name, value, ts, Cell.NO_DELETION_TIME, Cell.NO_TTL)
 +                        : new LegacyCell(LegacyCell.Kind.DELETED, name, ByteBufferUtil.EMPTY_BYTE_BUFFER, ts, ByteBufferUtil.toInt(value), Cell.NO_TTL));
 +        }
 +    }
 +
 +    public static LegacyRangeTombstone readLegacyRangeTombstoneBody(CFMetaData metadata, DataInputPlus in, ByteBuffer boundname) throws IOException
 +    {
 +        LegacyBound min = decodeBound(metadata, boundname, true);
 +        LegacyBound max = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
 +        DeletionTime dt = DeletionTime.serializer.deserialize(in);
 +        return new LegacyRangeTombstone(min, max, dt);
 +    }
 +
 +    public static Iterator<LegacyCell> deserializeCells(final CFMetaData metadata,
 +                                                        final DataInput in,
 +                                                        final SerializationHelper.Flag flag,
 +                                                        final int size)
 +    {
 +        return new AbstractIterator<LegacyCell>()
 +        {
 +            private int i = 0;
 +
 +            protected LegacyCell computeNext()
 +            {
 +                if (i >= size)
 +                    return endOfData();
 +
 +                ++i;
 +                try
 +                {
 +                    return readLegacyCell(metadata, in, flag);
 +                }
 +                catch (UnknownColumnException e)
 +                {
 +                    // We can get there if we read a cell for a dropped column, and if that is the case,
 +                    // then simply ignore the cell is fine. But also not that we ignore if it's the
 +                    // system keyspace because for those table we actually remove columns without registering
 +                    // them in the dropped columns
 +                    if (metadata.ksName.equals(SystemKeyspace.NAME) || metadata.getDroppedColumnDefinition(e.columnName) != null)
 +                        return computeNext();
 +                    else
 +                        throw new IOError(e);
 +                }
 +                catch (IOException e)
 +                {
 +                    throw new IOError(e);
 +                }
 +            }
 +        };
 +    }
 +
 +    public static class CellGrouper
 +    {
 +        /**
 +         * The fake TTL used for expired rows that have been compacted.
 +         */
 +        private static final int FAKE_TTL = 1;
 +
 +        public final CFMetaData metadata;
 +        private final boolean isStatic;
 +        private final SerializationHelper helper;
 +        private final Row.Builder builder;
 +        private Clustering clustering;
 +
 +        private LegacyRangeTombstone rowDeletion;
 +        private LegacyRangeTombstone collectionDeletion;
 +
 +        public CellGrouper(CFMetaData metadata, SerializationHelper helper)
 +        {
 +            this(metadata, helper, false);
 +        }
 +
 +        private CellGrouper(CFMetaData metadata, SerializationHelper helper, boolean isStatic)
 +        {
 +            this.metadata = metadata;
 +            this.isStatic = isStatic;
 +            this.helper = helper;
 +            // We cannot use a sorted builder because we don't have exactly the same ordering in 3.0 and pre-3.0. More precisely, within a row, we
 +            // store all simple columns before the complex ones in 3.0, which we use to sort everything sorted by the column name before. Note however
 +            // that the unsorted builder won't have to reconcile cells, so the exact value we pass for nowInSec doesn't matter.
 +            this.builder = BTreeRow.unsortedBuilder(FBUtilities.nowInSeconds());
 +        }
 +
 +        public static CellGrouper staticGrouper(CFMetaData metadata, SerializationHelper helper)
 +        {
 +            return new CellGrouper(metadata, helper, true);
 +        }
 +
 +        public void reset()
 +        {
 +            this.clustering = null;
 +            this.rowDeletion = null;
 +            this.collectionDeletion = null;
 +        }
 +
 +        public boolean addAtom(LegacyAtom atom)
 +        {
 +            return atom.isCell()
 +                 ? addCell(atom.asCell())
 +                 : addRangeTombstone(atom.asRangeTombstone());
 +        }
 +
 +        public boolean addCell(LegacyCell cell)
 +        {
 +            if (clustering == null)
 +            {
 +                clustering = cell.name.clustering;
 +                assert !isStatic || clustering == Clustering.STATIC_CLUSTERING;
 +                builder.newRow(clustering);
 +            }
 +            else if (!clustering.equals(cell.name.clustering))
 +            {
 +                return false;
 +            }
 +
 +            // Ignore shadowed cells
 +            if (rowDeletion != null && rowDeletion.deletionTime.deletes(cell.timestamp))
 +                return true;
 +
 +            ColumnDefinition column = cell.name.column;
 +            if (column == null)
 +            {
 +                // It's the row marker
 +                assert !cell.value.hasRemaining();
 +                // In 2.1, the row marker expired cell might have been converted into a deleted one by compaction.
 +                // If we do not set the primary key liveness info for this row and it does not contains any regular columns
 +                // the row will be empty. To avoid that, we reuse the localDeletionTime but use a fake TTL.
 +                // The only time in 2.x that we actually delete a row marker is in 2i tables, so in that case we do
 +                // want to actually propagate the row deletion. (CASSANDRA-13320)
 +                if (!cell.isTombstone())
 +                    builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, cell.ttl, cell.localDeletionTime));
 +                else if (metadata.isIndex())
 +                    builder.addRowDeletion(Row.Deletion.regular(new DeletionTime(cell.timestamp, cell.localDeletionTime)));
 +                else
 +                    builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, FAKE_TTL, cell.localDeletionTime));
 +            }
 +            else
 +            {
 +                if (collectionDeletion != null && collectionDeletion.start.collectionName.name.equals(column.name) && collectionDeletion.deletionTime.deletes(cell.timestamp))
 +                    return true;
 +
 +                if (helper.includes(column))
 +                {
 +                    CellPath path = null;
 +                    if (column.isComplex())
 +                    {
 +                        // Recalling startOfComplexColumn for every cell is a big inefficient, but it's ok in practice
 +                        // and it's simpler. And since 1) this only matter for super column selection in thrift in
 +                        // practice and 2) is only used during upgrade, it's probably worth keeping things simple.
 +                        helper.startOfComplexColumn(column);
 +                        path = cell.name.collectionElement == null ? null : CellPath.create(cell.name.collectionElement);
 +                        if (!helper.includes(path))
 +                            return true;
 +                    }
 +                    Cell c = new BufferCell(column, cell.timestamp, cell.ttl, cell.localDeletionTime, cell.value, path);
 +                    if (!helper.isDropped(c, column.isComplex()))
 +                        builder.addCell(c);
 +                    if (column.isComplex())
 +                    {
 +                        helper.endOfComplexColumn();
 +                    }
 +                }
 +            }
 +            return true;
 +        }
 +
 +        private boolean addRangeTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (tombstone.isRowDeletion(metadata))
 +                return addRowTombstone(tombstone);
 +            else if (tombstone.isCollectionTombstone())
 +                return addCollectionTombstone(tombstone);
 +            else
 +                return addGenericRangeTombstone(tombstone);
 +        }
 +
 +        private boolean addRowTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (clustering != null)
 +            {
 +                // If we're already in the row, there might be a chance that there were two range tombstones
 +                // written, as 2.x storage format does not guarantee just one range tombstone, unlike 3.x.
 +                // We have to make sure that clustering matches, which would mean that tombstone is for the
 +                // same row.
 +                if (rowDeletion != null && clustering.equals(tombstone.start.getAsClustering(metadata)))
 +                {
 +                    // If the tombstone superceeds the previous delete, we discard the previous one
 +                    if (tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
 +                    {
 +                        builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
 +                        rowDeletion = tombstone;
 +                    }
 +                    return true;
 +                }
 +
 +                // If we're already within a row and there was no delete written before that one, it can't be the same one
 +                return false;
 +            }
 +
 +            clustering = tombstone.start.getAsClustering(metadata);
 +            builder.newRow(clustering);
 +            builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
 +            rowDeletion = tombstone;
 +
 +            return true;
 +        }
 +
 +        private boolean addCollectionTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            if (!helper.includes(tombstone.start.collectionName))
 +                return false; // see CASSANDRA-13109
 +
 +            if (clustering == null)
 +            {
 +                clustering = tombstone.start.getAsClustering(metadata);
 +                builder.newRow(clustering);
 +            }
 +            else if (!clustering.equals(tombstone.start.getAsClustering(metadata)))
 +            {
 +                return false;
 +            }
 +
 +            builder.addComplexDeletion(tombstone.start.collectionName, tombstone.deletionTime);
 +            if (rowDeletion == null || tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
 +                collectionDeletion = tombstone;
 +
 +            return true;
 +        }
 +
 +        private boolean addGenericRangeTombstone(LegacyRangeTombstone tombstone)
 +        {
 +            /*
 +             * We can see a non-collection, non-row deletion in two scenarios:
 +             *
 +             * 1. Most commonly, the tombstone's start bound is bigger than current row's clustering, which means that
 +             *    the current row is over, and we should move on to the next row or RT;
 +             *
 +             * 2. Less commonly, the tombstone's start bound is smaller than current row's clustering, which means that
 +             *    we've crossed an index boundary and are seeing a non-closed RT from the previous block, repeated;
 +             *    we should ignore it and stay in the current row.
 +             *
 +             *  In either case, clustering should be non-null, or we shouldn't have gotten to this method at all
 +             *  However, to be absolutely SURE we're in case two above, we check here.
 +             */
 +            return clustering != null && metadata.comparator.compare(clustering, tombstone.start.bound.clustering()) > 0;
 +        }
 +
 +        public Row getRow()
 +        {
 +            return builder.build();
 +        }
 +    }
 +
 +    public static class LegacyUnfilteredPartition
 +    {
 +        public final DeletionTime partitionDeletion;
 +        public final LegacyRangeTombstoneList rangeTombstones;
 +        public final List<LegacyCell> cells;
 +
 +        private LegacyUnfilteredPartition(DeletionTime partitionDeletion, LegacyRangeTombstoneList rangeTombstones, List<LegacyCell> cells)
 +        {
 +            this.partitionDeletion = partitionDeletion;
 +            this.rangeTombstones = rangeTombstones;
 +            this.cells = cells;
 +        }
 +
 +        public void digest(CFMetaData metadata, MessageDigest digest)
 +        {
 +            for (LegacyCell cell : cells)
 +            {
 +                digest.update(cell.name.encode(metadata).duplicate());
 +
 +                if (cell.isCounter())
 +                    CounterContext.instance().updateDigest(digest, cell.value);
 +                else
 +                    digest.update(cell.value.duplicate());
 +
 +                FBUtilities.updateWithLong(digest, cell.timestamp);
 +                FBUtilities.updateWithByte(digest, cell.serializationFlags());
 +
 +                if (cell.isExpiring())
 +                    FBUtilities.updateWithInt(digest, cell.ttl);
 +
 +                if (cell.isCounter())
 +                {
 +                    // Counters used to have the timestampOfLastDelete field, which we stopped using long ago and has been hard-coded
 +                    // to Long.MIN_VALUE but was still taken into account in 2.2 counter digests (to maintain backward compatibility
 +                    // in the first place).
 +                    FBUtilities.updateWithLong(digest, Long.MIN_VALUE);
 +                }
 +            }
 +
 +            if (partitionDeletion.markedForDeleteAt() != Long.MIN_VALUE)
 +                digest.update(ByteBufferUtil.bytes(partitionDeletion.markedForDeleteAt()));
 +
 +            if (!rangeTombstones.isEmpty())
 +                rangeTombstones.updateDigest(digest);
 +        }
 +    }
 +
 +    public static class LegacyCellName
 +    {
 +        public final Clustering clustering;
 +        public final ColumnDefinition column;
 +        public final ByteBuffer collectionElement;
 +
 +        private LegacyCellName(Clustering clustering, ColumnDefinition column, ByteBuffer collectionElement)
 +        {
 +            this.clustering = clustering;
 +            this.column = column;
 +            this.collectionElement = collectionElement;
 +        }
 +
 +        public ByteBuffer encode(CFMetaData metadata)
 +        {
 +            return encodeCellName(metadata, clustering, column == null ? ByteBufferUtil.EMPTY_BYTE_BUFFER : column.name.bytes, collectionElement);
 +        }
 +
 +        public ByteBuffer superColumnSubName()
 +        {
 +            assert collectionElement != null;
 +            return collectionElement;
 +        }
 +
 +        public ByteBuffer superColumnName()
 +        {
 +            return clustering.get(0);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            for (int i = 0; i < clustering.size(); i++)
 +                sb.append(i > 0 ? ":" : "").append(clustering.get(i) == null ? "null" : ByteBufferUtil.bytesToHex(clustering.get(i)));
 +            return String.format("Cellname(clustering=%s, column=%s, collElt=%s)", sb.toString(), column == null ? "null" : column.name, collectionElement == null ? "null" : ByteBufferUtil.bytesToHex(collectionElement));
 +        }
 +    }
 +
 +    public static class LegacyBound
 +    {
 +        public static final LegacyBound BOTTOM = new LegacyBound(Slice.Bound.BOTTOM, false, null);
 +        public static final LegacyBound TOP = new LegacyBound(Slice.Bound.TOP, false, null);
 +
 +        public final Slice.Bound bound;
 +        public final boolean isStatic;
 +        public final ColumnDefinition collectionName;
 +
 +        public LegacyBound(Slice.Bound bound, boolean isStatic, ColumnDefinition collectionName)
 +        {
 +            this.bound = bound;
 +            this.isStatic = isStatic;
 +            this.collectionName = collectionName;
 +        }
 +
 +        public Clustering getAsClustering(CFMetaData metadata)
 +        {
 +            if (isStatic)
 +                return Clustering.STATIC_CLUSTERING;
 +
 +            assert bound.size() == metadata.comparator.size();
 +            ByteBuffer[] values = new ByteBuffer[bound.size()];
 +            for (int i = 0; i < bound.size(); i++)
 +                values[i] = bound.get(i);
 +            return new Clustering(values);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            StringBuilder sb = new StringBuilder();
 +            sb.append(bound.kind()).append('(');
 +            for (int i = 0; i < bound.size(); i++)
 +                sb.append(i > 0 ? ":" : "").append(bound.get(i) == null ? "null" : ByteBufferUtil.bytesToHex(bound.get(i)));
 +            sb.append(')');
 +            return String.format("Bound(%s, collection=%s)", sb.toString(), collectionName == null ? "null" : collectionName.name);
 +        }
 +    }
 +
 +    public interface LegacyAtom
 +    {
 +        public boolean isCell();
 +
 +        public ClusteringPrefix clustering();
 +        public boolean isStatic();
 +
 +        public LegacyCell asCell();
 +        public LegacyRangeTombstone asRangeTombstone();
 +    }
 +
 +    /**
 +     * A legacy cell.
 +     * <p>
 +     * This is used as a temporary object to facilitate dealing with the legacy format, this
 +     * is not meant to be optimal.
 +     */
 +    public static class LegacyCell implements LegacyAtom
 +    {
 +        private final static int DELETION_MASK        = 0x01;
 +        private final static int EXPIRATION_MASK      = 0x02;
 +        private final static int COUNTER_MASK         = 0x04;
 +        private final static int COUNTER_UPDATE_MASK  = 0x08;
 +        private final static int RANGE_TOMBSTONE_MASK = 0x10;
 +
 +        public enum Kind { REGULAR, EXPIRING, DELETED, COUNTER }
 +
 +        public final Kind kind;
 +
 +        public final LegacyCellName name;
 +        public final ByteBuffer value;
 +
 +        public final long timestamp;
 +        public final int localDeletionTime;
 +        public final int ttl;
 +
 +        private LegacyCell(Kind kind, LegacyCellName name, ByteBuffer value, long timestamp, int localDeletionTime, int ttl)
 +        {
 +            this.kind = kind;
 +            this.name = name;
 +            this.value = value;
 +            this.timestamp = timestamp;
 +            this.localDeletionTime = localDeletionTime;
 +            this.ttl = ttl;
 +        }
 +
 +        public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp)
 +        throws UnknownColumnException
 +        {
 +            return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +
 +        public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp, int ttl, int nowInSec)
 +        throws UnknownColumnException
 +        {
-             return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, nowInSec + ttl, ttl);
++            /*
++             * CASSANDRA-14092: Max expiration date capping is maybe performed here, expiration overflow policy application
++             * is done at {@link org.apache.cassandra.thrift.ThriftValidation#validateTtl(CFMetaData, Column)}
++             */
++            return new LegacyCell(Kind.EXPIRING, decodeCellName(metadata, superColumnName, name), value, timestamp, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), ttl);
 +        }
 +
 +        public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
 +        throws UnknownColumnException
 +        {
 +            return new LegacyCell(Kind.DELETED, decodeCellName(metadata, superColumnName, name), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, nowInSec, LivenessInfo.NO_TTL);
 +        }
 +
 +        public static LegacyCell counterUpdate(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long value)
 +        throws UnknownColumnException
 +        {
 +            // See UpdateParameters.addCounter() for more details on this
 +            ByteBuffer counterValue = CounterContext.instance().createUpdate(value);
 +            return counter(decodeCellName(metadata, superColumnName, name), counterValue);
 +        }
 +
 +        public static LegacyCell counter(LegacyCellName name, ByteBuffer value)
 +        {
 +            return new LegacyCell(Kind.COUNTER, name, value, FBUtilities.timestampMicros(), Cell.NO_DELETION_TIME, Cell.NO_TTL);
 +        }
 +
 +        public byte serializationFlags()
 +        {
 +            if (isExpiring())
 +                return EXPIRATION_MASK;
 +            if (isTombstone())
 +                return DELETION_MASK;
 +            if (isCounterUpdate())
 +                return COUNTER_UPDATE_MASK;
 +            if (isCounter())
 +                return COUNTER_MASK;
 +            return 0;
 +        }
 +
 +        public boolean isCounterUpdate()
 +        {
 +            // See UpdateParameters.addCounter() for more details on this
 +            return isCounter() && CounterContext.instance().isUpdate(value);
 +        }
 +
 +        public ClusteringPrefix clustering()
 +        {
 +            return name.clustering;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return name.clustering == Clustering.STATIC_CLUSTERING;
 +        }
 +
 +        public boolean isCell()
 +        {
 +            return true;
 +        }
 +
 +        public LegacyCell asCell()
 +        {
 +            return this;
 +        }
 +
 +        public LegacyRangeTombstone asRangeTombstone()
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        public boolean isCounter()
 +        {
 +            return kind == Kind.COUNTER;
 +        }
 +
 +        public boolean isExpiring()
 +        {
 +            return kind == Kind.EXPIRING;
 +        }
 +
 +        public boolean isTombstone()
 +        {
 +            return kind == Kind.DELETED;
 +        }
 +
 +        public boolean isLive(int nowInSec)
 +        {
 +            if (isTombstone())
 +                return false;
 +
 +            return !isExpiring() || nowInSec < localDeletionTime;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("LegacyCell(%s, name=%s, v=%s, ts=%s, ldt=%s, ttl=%s)", kind, name, ByteBufferUtil.bytesToHex(value), timestamp, localDeletionTime, ttl);
 +        }
 +    }
 +
 +    /**
 +     * A legacy range tombstone.
 +     * <p>
 +     * This is used as a temporary object to facilitate dealing with the legacy format, this
 +     * is not meant to be optimal.
 +     */
 +    public static class LegacyRangeTombstone implements LegacyAtom
 +    {
 +        public final LegacyBound start;
 +        public final LegacyBound stop;
 +        public final DeletionTime deletionTime;
 +
 +        public LegacyRangeTombstone(LegacyBound start, LegacyBound stop, DeletionTime deletionTime)
 +        {
 +            // Because of the way RangeTombstoneList work, we can have a tombstone where only one of
 +            // the bound has a collectionName. That happens if we have a big tombstone A (spanning one
 +            // or multiple rows) and a collection tombstone B. In that case, RangeTombstoneList will
 +            // split this into 3 RTs: the first one from the beginning of A to the beginning of B,
 +            // then B, then a third one from the end of B to the end of A. To make this simpler, if
 +            // we detect that case we transform the 1st and 3rd tombstone so they don't end in the middle
 +            // of a row (which is still correct).
 +            if ((start.collectionName == null) != (stop.collectionName == null))
 +            {
 +                if (start.collectionName == null)
 +                    stop = new LegacyBound(stop.bound, stop.isStatic, null);
 +                else
 +                    start = new LegacyBound(start.bound, start.isStatic, null);
 +            }
 +            else if (!Objects.equals(start.collectionName, stop.collectionName))
 +            {
 +                // We're in the similar but slightly more complex case where on top of the big tombstone
 +                // A, we have 2 (or more) collection tombstones B and C within A. So we also end up with
 +                // a tombstone that goes between the end of B and the start of C.
 +                start = new LegacyBound(start.bound, start.isStatic, null);
 +                stop = new LegacyBound(stop.bound, stop.isStatic, null);
 +            }
 +
 +            this.start = start;
 +            this.stop = stop;
 +            this.deletionTime = deletionTime;
 +        }
 +
 +        public ClusteringPrefix clustering()
 +        {
 +            return start.bound;
 +        }
 +
 +        public LegacyRangeTombstone withNewStart(LegacyBound newStart)
 +        {
 +            return new LegacyRangeTombstone(newStart, stop, deletionTime);
 +        }
 +
 +        public LegacyRangeTombstone withNewEnd(LegacyBound newStop)
 +        {
 +            return new LegacyRangeTombstone(start, newStop, deletionTime);
 +        }
 +
 +        public boolean isCell()
 +        {
 +            return false;
 +        }
 +
 +        public boolean isStatic()
 +        {
 +            return start.isStatic || stop.isStatic;
 +        }
 +
 +        public LegacyCell asCell()
 +        {
 +            throw new UnsupportedOperationException();
 +        }
 +
 +        public LegacyRangeTombstone asRangeTombstone()
 +        {
 +            return this;
 +        }
 +
 +        public boolean isCollectionTombstone()
 +        {
 +            return start.collectionName != null;
 +        }
 +
 +        public boolean isRowDeletion(CFMetaData metadata)
 +        {
 +            if (start.collectionName != null
 +                || stop.collectionName != null
 +                || start.bound.size() != metadata.comparator.size()
 +                || stop.bound.size() != metadata.comparator.size())
 +                return false;
 +
 +            for (int i = 0; i < start.bound.size(); i++)
 +                if (!Objects.equals(start.bound.get(i), stop.bound.get(i)))
 +                    return false;
 +            return true;
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("RT(%s-%s, %s)", start, stop, deletionTime);
 +        }
 +    }
 +
 +    public static class LegacyDeletionInfo
 +    {
 +        public final MutableDeletionInfo deletionInfo;
 +        public final List<LegacyRangeTombstone> inRowTombstones = new ArrayList<>();
 +
 +        private LegacyDeletionInfo(MutableDeletionInfo deletionInfo)
 +        {
 +            this.deletionInfo = deletionInfo;
 +        }
 +
 +        public static LegacyDeletionInfo live()
 +        {
 +            return new LegacyDeletionInfo(MutableDeletionInfo.live());
 +        }
 +
 +        public void add(DeletionTime topLevel)
 +        {
 +            deletionInfo.add(topLevel);
 +        }
 +
 +        private static Slice.Bound staticBound(CFMetaData metadata, boolean isStart)
 +        {
 +            // In pre-3.0 nodes, static row started by a clustering with all empty values so we
 +            // preserve that here. Note that in practice, it doesn't really matter since the rest
 +            // of the code will ignore the bound for RT that have their static flag set.
 +            ByteBuffer[] values = new ByteBuffer[metadata.comparator.size()];
 +            for (int i = 0; i < values.length; i++)
 +                values[i] = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +            return isStart
 +                 ? Slice.Bound.inclusiveStartOf(values)
 +                 : Slice.Bound.inclusiveEndOf(values);
 +        }
 +
 +        public void add(CFMetaData metadata, LegacyRangeTombstone tombstone)
 +        {
 +            if (metadata.hasStaticColumns())
 +            {
 +                /*
 +                 * For table having static columns we have to deal with the following cases:
 +                 *  1. the end of the tombstone is static (in which case either the start is static or is BOTTOM, which is the same
 +                 *     for our consideration). This mean that either the range only delete the static row, or that it's a collection
 +                 *     tombstone of a static collection. In both case, we just add the tombstone to the inRowTombstones.
 +                 *  2. only the start is static. There is then 2 subcase: either the start is inclusive, and that mean we include the
 +                 *     static row and more (so we add an inRowTombstone for the static and deal with the rest normally). Or the start
 +                 *     is exclusive, and that means we explicitely exclude the static (in which case we can just add the tombstone
 +                 *     as if it started at BOTTOM).
 +                 *  3. none of the bound are static but the start is BOTTOM. This means we intended to delete the static row so we
 +                 *     need to add it to the inRowTombstones (and otherwise handle the range normally).
 +                 */
 +                if (tombstone.stop.isStatic)
 +                {
 +                    // If the start is BOTTOM, we replace it by the beginning of the starting row so as to not confuse the
 +                    // RangeTombstone.isRowDeletion() method
 +                    if (tombstone.start == LegacyBound.BOTTOM)
 +                        tombstone = tombstone.withNewStart(new LegacyBound(staticBound(metadata, true), true, null));
 +                    inRowTombstones.add(tombstone);
 +                    return;
 +                }
 +
 +                if (tombstone.start.isStatic)
 +                {
 +                    if (tombstone.start.bound.isInclusive())
 +                        inRowTombstones.add(tombstone.withNewEnd(new LegacyBound(staticBound(metadata, false), true, null)));
 +
 +                    tombstone = tombstone.withNewStart(LegacyBound.BOTTOM);
 +                }
 +                else if (tombstone.start == LegacyBound.BOTTOM)
 +                {
 +                    inRowTombstones.add(new LegacyRangeTombstone(new LegacyBound(staticBound(metadata, true), true, null),
 +                                                                 new LegacyBound(staticBound(metadata, false), true, null),
 +                                                                 tombstone.deletionTime));
 +                }
 +            }
 +
 +            if (tombstone.isCollectionTombstone() || tombstone.isRowDeletion(metadata))
 +                inRowTombstones.add(tombstone);
 +            else
 +                add(metadata, new RangeTombstone(Slice.make(tombstone.start.bound, tombstone.stop.bound), tombstone.deletionTime));
 +        }
 +
 +        public void add(CFMetaData metadata, RangeTombstone tombstone)
 +        {
 +            deletionInfo.add(tombstone, metadata.comparator);
 +        }
 +
 +        public Iterator<LegacyRangeTombstone> inRowRangeTombstones()
 +        {
 +            return inRowTombstones.iterator();
 +        }
 +
 +        public static LegacyDeletionInfo deserialize(CFMetaData metadata, DataInputPlus in) throws IOException
 +        {
 +            DeletionTime topLevel = DeletionTime.serializer.deserialize(in);
 +
 +            int rangeCount = in.readInt();
 +            if (rangeCount == 0)
 +                return new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
 +
 +            LegacyDeletionInfo delInfo = new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
 +            for (int i = 0; i < rangeCount; i++)
 +            {
 +                LegacyBound start = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
 +                LegacyBound end = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
 +                int delTime =  in.readInt();
 +                long markedAt = in.readLong();
 +
 +                delInfo.add(metadata, new LegacyRangeTombstone(start, end, new DeletionTime(markedAt, delTime)));
 +            }
 +            return delInfo;
 +        }
 +    }
 +
 +    /**
 +     * A helper class for LegacyRangeTombstoneList.  This replaces the Comparator<Composite> that RTL used before 3.0.
 +     */
 +    private static class LegacyBoundComparator implements Comparator<LegacyBound>
 +    {
 +        ClusteringComparator clusteringComparator;
 +
 +        public LegacyBoundComparator(ClusteringComparator clusteringComparator)
 +        {
 +            this.clusteringComparator = clusteringComparator;
 +        }
 +
 +        public int compare(LegacyBound a, LegacyBound b)
 +        {
 +            // In the legacy sorting, BOTTOM comes before anything else
 +            if (a == LegacyBound.BOTTOM)
 +                return b == LegacyBound.BOTTOM ? 0 : -1;
 +            if (b == LegacyBound.BOTTOM)
 +                return 1;
 +
 +            // Excluding BOTTOM, statics are always before anything else.
 +            if (a.isStatic != b.isStatic)
 +                return a.isStatic ? -1 : 1;
 +
 +            // We have to be careful with bound comparison because of collections. Namely, if the 2 bounds represent the
 +            // same prefix, then we should take the collectionName into account before taking the bounds kind
 +            // (ClusteringPrefix.Kind). This means we can't really call ClusteringComparator.compare() directly.
 +            // For instance, if
 +            //    a is (bound=INCL_START_BOUND('x'), collectionName='d')
 +            //    b is (bound=INCL_END_BOUND('x'),   collectionName='c')
 +            // Ten b < a since the element 'c' of collection 'x' comes before element 'd', but calling
 +            // clusteringComparator.compare(a.bound, b.bound) returns -1.
 +            // See CASSANDRA-13125 for details.
 +            int sa = a.bound.size();
 +            int sb = b.bound.size();
 +            for (int i = 0; i < Math.min(sa, sb); i++)
 +            {
 +                int cmp = clusteringComparator.compareComponent(i, a.bound.get(i), b.bound.get(i));
 +                if (cmp != 0)
 +                    return cmp;
 +            }
 +
 +            if (sa != sb)
 +                return sa < sb ? a.bound.kind().comparedToClustering : -b.bound.kind().comparedToClustering;
 +
 +            // Both bound represent the same prefix, compare the collection names
 +            // If one has a collection name and the other doesn't, the other comes before as it points to the beginning of the row.
 +            if ((a.collectionName == null) != (b.collectionName == null))
 +                return a.collectionName == null ? -1 : 1;
 +
 +            // If they both have a collection, compare that first
 +            if (a.collectionName != null)
 +            {
 +                int cmp = UTF8Type.instance.compare(a.collectionName.name.bytes, b.collectionName.name.bytes);
 +                if (cmp != 0)
 +                    return cmp;
 +            }
 +
 +            // Lastly, if everything so far is equal, compare their clustering kind
 +            return ClusteringPrefix.Kind.compare(a.bound.kind(), b.bound.kind());
 +        }
 +    }
 +
 +    /**
 +     * Almost an entire copy of RangeTombstoneList from C* 2.1.  The main difference is that LegacyBoundComparator
 +     * is used in place of Comparator<Composite> (because Composite doesn't exist any more).
 +     *
 +     * This class is needed to allow us to convert single-row deletions and complex deletions into range tombstones
 +     * and properly merge them into the normal set of range tombstones.
 +     */
 +    public static class LegacyRangeTombstoneList
 +    {
 +        private final LegacyBoundComparator comparator;
 +
 +        // Note: we don't want to use a List for the markedAts and delTimes to avoid boxin

<TRUNCATED>

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


[24/29] cassandra git commit: Merge branch 'cassandra-2.2' into cassandra-3.0

Posted by pa...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/LivenessInfo.java
index 89e0578,0000000..f6c9b62
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@@ -1,369 -1,0 +1,375 @@@
 +/*
 + * 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;
 +
 +import java.util.Objects;
 +import java.security.MessageDigest;
 +
 +import org.apache.cassandra.config.CFMetaData;
++import org.apache.cassandra.db.rows.Cell;
 +import org.apache.cassandra.serializers.MarshalException;
 +import org.apache.cassandra.utils.FBUtilities;
 +
 +/**
 + * Stores the information relating to the liveness of the primary key columns of a row.
 + * <p>
 + * A {@code LivenessInfo} can first be empty. If it isn't, it contains at least a timestamp,
 + * which is the timestamp for the row primary key columns. On top of that, the info can be
 + * ttl'ed, in which case the {@code LivenessInfo} also has both a ttl and a local expiration time.
 + * <p>
 + * Please note that if a liveness info is ttl'ed, that expiration is <b>only</b> an expiration
 + * of the liveness info itself (so, of the timestamp), and once the info expires it becomes
 + * {@code EMPTY}. But if a row has a liveness info which expires, the rest of the row data is
 + * unaffected (of course, the rest of said row data might be ttl'ed on its own but this is
 + * separate).
 + */
 +public class LivenessInfo
 +{
 +    public static final long NO_TIMESTAMP = Long.MIN_VALUE;
-     public static final int NO_TTL = 0;
++    public static final int NO_TTL = Cell.NO_TTL;
 +    /**
 +     * Used as flag for representing an expired liveness.
 +     *
 +     * TTL per request is at most 20 yrs, so this shouldn't conflict
 +     * (See {@link org.apache.cassandra.cql3.Attributes#MAX_TTL})
 +     */
 +    public static final int EXPIRED_LIVENESS_TTL = Integer.MAX_VALUE;
-     public static final int NO_EXPIRATION_TIME = Integer.MAX_VALUE;
++    public static final int NO_EXPIRATION_TIME = Cell.NO_DELETION_TIME;
 +
 +    public static final LivenessInfo EMPTY = new LivenessInfo(NO_TIMESTAMP);
 +
 +    protected final long timestamp;
 +
 +    protected LivenessInfo(long timestamp)
 +    {
 +        this.timestamp = timestamp;
 +    }
 +
 +    public static LivenessInfo create(CFMetaData metadata, long timestamp, int nowInSec)
 +    {
 +        int defaultTTL = metadata.params.defaultTimeToLive;
 +        if (defaultTTL != NO_TTL)
 +            return expiring(timestamp, defaultTTL, nowInSec);
 +
 +        return new LivenessInfo(timestamp);
 +    }
 +
 +    public static LivenessInfo expiring(long timestamp, int ttl, int nowInSec)
 +    {
 +        assert ttl != EXPIRED_LIVENESS_TTL;
-         return new ExpiringLivenessInfo(timestamp, ttl, nowInSec + ttl);
++        return new ExpiringLivenessInfo(timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl));
 +    }
 +
 +    public static LivenessInfo create(CFMetaData metadata, long timestamp, int ttl, int nowInSec)
 +    {
 +        return ttl == NO_TTL
 +             ? create(metadata, timestamp, nowInSec)
 +             : expiring(timestamp, ttl, nowInSec);
 +    }
 +
 +    // Note that this ctor ignores the default table ttl and takes the expiration time, not the current time.
 +    // Use when you know that's what you want.
 +    public static LivenessInfo create(long timestamp, int ttl, int localExpirationTime)
 +    {
 +        if (ttl == EXPIRED_LIVENESS_TTL)
 +            return new ExpiredLivenessInfo(timestamp, ttl, localExpirationTime);
 +        return ttl == NO_TTL ? new LivenessInfo(timestamp) : new ExpiringLivenessInfo(timestamp, ttl, localExpirationTime);
 +    }
 +
 +    /**
 +     * Whether this liveness info is empty (has no timestamp).
 +     *
 +     * @return whether this liveness info is empty or not.
 +     */
 +    public boolean isEmpty()
 +    {
 +        return timestamp == NO_TIMESTAMP;
 +    }
 +
 +    /**
 +     * The timestamp for this liveness info.
 +     *
 +     * @return the liveness info timestamp (or {@link #NO_TIMESTAMP} if the info is empty).
 +     */
 +    public long timestamp()
 +    {
 +        return timestamp;
 +    }
 +
 +    /**
 +     * Whether the info has a ttl.
 +     */
 +    public boolean isExpiring()
 +    {
 +        return false;
 +    }
 +
 +    /**
 +     * The ttl (if any) on the row primary key columns or {@link #NO_TTL} if it is not
 +     * expiring.
 +     *
 +     * Please note that this value is the TTL that was set originally and is thus not
 +     * changing.
 +     */
 +    public int ttl()
 +    {
 +        return NO_TTL;
 +    }
 +
 +    /**
 +     * The expiration time (in seconds) if the info is expiring ({@link #NO_EXPIRATION_TIME} otherwise).
 +     *
 +     */
 +    public int localExpirationTime()
 +    {
 +        return NO_EXPIRATION_TIME;
 +    }
 +
 +    /**
 +     * Whether that info is still live.
 +     *
 +     * A {@code LivenessInfo} is live if it is either not expiring, or if its expiration time if after
 +     * {@code nowInSec}.
 +     *
 +     * @param nowInSec the current time in seconds.
 +     * @return whether this liveness info is live or not.
 +     */
 +    public boolean isLive(int nowInSec)
 +    {
 +        return !isEmpty();
 +    }
 +
 +    /**
 +     * Adds this liveness information to the provided digest.
 +     *
 +     * @param digest the digest to add this liveness information to.
 +     */
 +    public void digest(MessageDigest digest)
 +    {
 +        FBUtilities.updateWithLong(digest, timestamp());
 +    }
 +
 +    /**
 +     * Validate the data contained by this liveness information.
 +     *
 +     * @throws MarshalException if some of the data is corrupted.
 +     */
 +    public void validate()
 +    {
 +    }
 +
 +    /**
 +     * The size of the (useful) data this liveness information contains.
 +     *
 +     * @return the size of the data this liveness information contains.
 +     */
 +    public int dataSize()
 +    {
 +        return TypeSizes.sizeof(timestamp());
 +    }
 +
 +    /**
 +     * Whether this liveness information supersedes another one (that is
 +     * whether is has a greater timestamp than the other or not).
 +     *
 +     * </br>
 +     *
 +     * If timestamps are the same and none of them are expired livenessInfo,
 +     * livenessInfo with greater TTL supersedes another. It also means, if timestamps are the same,
 +     * ttl superseders no-ttl. This is the same rule as {@link Conflicts#resolveRegular}
 +     *
 +     * If timestamps are the same and one of them is expired livenessInfo. Expired livenessInfo
 +     * supersedes, ie. tombstone supersedes.
 +     *
 +     * If timestamps are the same and both of them are expired livenessInfo(Ideally it shouldn't happen),
 +     * greater localDeletionTime wins.
 +     *
 +     * @param other
 +     *            the {@code LivenessInfo} to compare this info to.
 +     *
 +     * @return whether this {@code LivenessInfo} supersedes {@code other}.
 +     */
 +    public boolean supersedes(LivenessInfo other)
 +    {
 +        if (timestamp != other.timestamp)
 +            return timestamp > other.timestamp;
 +        if (isExpired() ^ other.isExpired())
 +            return isExpired();
 +        if (isExpiring() == other.isExpiring())
 +            return localExpirationTime() > other.localExpirationTime();
 +        return isExpiring();
 +    }
 +
 +    protected boolean isExpired()
 +    {
 +        return false;
 +    }
 +
 +    /**
 +     * Returns a copy of this liveness info updated with the provided timestamp.
 +     *
 +     * @param newTimestamp the timestamp for the returned info.
 +     * @return if this liveness info has a timestamp, a copy of it with {@code newTimestamp}
 +     * as timestamp. If it has no timestamp however, this liveness info is returned
 +     * unchanged.
 +     */
 +    public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +    {
 +        return new LivenessInfo(newTimestamp);
 +    }
 +
++    public LivenessInfo withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return LivenessInfo.create(newTimestamp, ttl(), newLocalDeletionTime);
++    }
++
 +    @Override
 +    public String toString()
 +    {
 +        return String.format("[ts=%d]", timestamp);
 +    }
 +
 +    @Override
 +    public boolean equals(Object other)
 +    {
 +        if(!(other instanceof LivenessInfo))
 +            return false;
 +
 +        LivenessInfo that = (LivenessInfo)other;
 +        return this.timestamp() == that.timestamp()
 +            && this.ttl() == that.ttl()
 +            && this.localExpirationTime() == that.localExpirationTime();
 +    }
 +
 +    @Override
 +    public int hashCode()
 +    {
 +        return Objects.hash(timestamp(), ttl(), localExpirationTime());
 +    }
 +
 +    /**
 +     * Effectively acts as a PK tombstone. This is used for Materialized Views to shadow
 +     * updated entries while co-existing with row tombstones.
 +     *
 +     * See {@link org.apache.cassandra.db.view.ViewUpdateGenerator#deleteOldEntryInternal}.
 +     */
 +    private static class ExpiredLivenessInfo extends ExpiringLivenessInfo
 +    {
 +        private ExpiredLivenessInfo(long timestamp, int ttl, int localExpirationTime)
 +        {
 +            super(timestamp, ttl, localExpirationTime);
 +            assert ttl == EXPIRED_LIVENESS_TTL;
 +            assert timestamp != NO_TIMESTAMP;
 +        }
 +
 +        @Override
 +        public boolean isExpired()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean isLive(int nowInSec)
 +        {
 +            // used as tombstone to shadow entire PK
 +            return false;
 +        }
 +
 +        @Override
 +        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +        {
 +            return new ExpiredLivenessInfo(newTimestamp, ttl(), localExpirationTime());
 +        }
 +    }
 +
 +    private static class ExpiringLivenessInfo extends LivenessInfo
 +    {
 +        private final int ttl;
 +        private final int localExpirationTime;
 +
 +        private ExpiringLivenessInfo(long timestamp, int ttl, int localExpirationTime)
 +        {
 +            super(timestamp);
 +            assert ttl != NO_TTL && localExpirationTime != NO_EXPIRATION_TIME;
 +            this.ttl = ttl;
 +            this.localExpirationTime = localExpirationTime;
 +        }
 +
 +        @Override
 +        public int ttl()
 +        {
 +            return ttl;
 +        }
 +
 +        @Override
 +        public int localExpirationTime()
 +        {
 +            return localExpirationTime;
 +        }
 +
 +        @Override
 +        public boolean isExpiring()
 +        {
 +            return true;
 +        }
 +
 +        @Override
 +        public boolean isLive(int nowInSec)
 +        {
 +            return nowInSec < localExpirationTime;
 +        }
 +
 +        @Override
 +        public void digest(MessageDigest digest)
 +        {
 +            super.digest(digest);
 +            FBUtilities.updateWithInt(digest, localExpirationTime);
 +            FBUtilities.updateWithInt(digest, ttl);
 +        }
 +
 +        @Override
 +        public void validate()
 +        {
 +            if (ttl < 0)
 +                throw new MarshalException("A TTL should not be negative");
 +            if (localExpirationTime < 0)
 +                throw new MarshalException("A local expiration time should not be negative");
 +        }
 +
 +        @Override
 +        public int dataSize()
 +        {
 +            return super.dataSize()
 +                 + TypeSizes.sizeof(ttl)
 +                 + TypeSizes.sizeof(localExpirationTime);
 +
 +        }
 +
 +        @Override
 +        public LivenessInfo withUpdatedTimestamp(long newTimestamp)
 +        {
 +            return new ExpiringLivenessInfo(newTimestamp, ttl, localExpirationTime);
 +        }
 +
 +        @Override
 +        public String toString()
 +        {
 +            return String.format("[ts=%d ttl=%d, let=%d]", timestamp, ttl, localExpirationTime);
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index eaf6dab,d90abe9..1d54667
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@@ -363,7 -348,13 +363,15 @@@ public class CompactionManager implemen
          }
      }
  
--    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs)
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
++                                           int jobs)
+     throws InterruptedException, ExecutionException
+     {
+         return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+     }
+ 
 -    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, final boolean reinsertOverflowedTTLRows, int jobs)
++    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
++                                           final boolean reinsertOverflowedTTL, int jobs)
      throws InterruptedException, ExecutionException
      {
          return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
@@@ -377,7 -368,7 +385,7 @@@
              @Override
              public void execute(LifecycleTransaction input) throws IOException
              {
-                 scrubOne(cfs, input, skipCorrupted, checkData);
 -                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
++                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTL);
              }
          }, jobs, OperationType.SCRUB);
      }
@@@ -745,11 -736,11 +753,11 @@@
          }
      }
  
-     private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData) throws IOException
 -    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
++    private void scrubOne(ColumnFamilyStore cfs, LifecycleTransaction modifier, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL) throws IOException
      {
          CompactionInfo.Holder scrubInfo = null;
  
-         try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData))
 -        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTLRows))
++        try (Scrubber scrubber = new Scrubber(cfs, modifier, skipCorrupted, checkData, reinsertOverflowedTTL))
          {
              scrubInfo = scrubber.getScrubInfo();
              metrics.beginCompaction(scrubInfo);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/compaction/Scrubber.java
index c8e0c53,affee11..bc11504
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@@ -35,8 -37,12 +35,9 @@@ import org.apache.cassandra.io.sstable.
  import org.apache.cassandra.io.util.FileUtils;
  import org.apache.cassandra.io.util.RandomAccessReader;
  import org.apache.cassandra.service.ActiveRepairService;
 -import org.apache.cassandra.utils.ByteBufferUtil;
 -import org.apache.cassandra.utils.JVMStabilityInspector;
 -import org.apache.cassandra.utils.OutputHandler;
 -import org.apache.cassandra.utils.memory.HeapAllocator;
 -import org.apache.cassandra.utils.UUIDGen;
 +import org.apache.cassandra.utils.*;
  import org.apache.cassandra.utils.concurrent.Refs;
++import org.apache.cassandra.utils.memory.HeapAllocator;
  
  public class Scrubber implements Closeable
  {
@@@ -45,7 -51,9 +46,8 @@@
      private final LifecycleTransaction transaction;
      private final File destination;
      private final boolean skipCorrupted;
+     private final boolean reinsertOverflowedTTLRows;
  
 -    private final CompactionController controller;
      private final boolean isCommutative;
      private final boolean isIndex;
      private final boolean checkData;
@@@ -65,38 -76,41 +67,47 @@@
      long currentRowPositionFromIndex;
      long nextRowPositionFromIndex;
  
 -    private final OutputHandler outputHandler;
+     private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
+ 
 -    private static final Comparator<Row> rowComparator = new Comparator<Row>()
 +    private final OutputHandler outputHandler;
 +
 +    private static final Comparator<Partition> partitionComparator = new Comparator<Partition>()
      {
 -         public int compare(Row r1, Row r2)
 +         public int compare(Partition r1, Partition r2)
           {
 -             return r1.key.compareTo(r2.key);
 +             return r1.partitionKey().compareTo(r2.partitionKey());
           }
      };
 -    private final SortedSet<Row> outOfOrderRows = new TreeSet<>(rowComparator);
 +    private final SortedSet<Partition> outOfOrder = new TreeSet<>(partitionComparator);
  
      public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData) throws IOException
      {
-         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData);
 -        this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, false);
++        this(cfs, transaction, skipCorrupted, checkData, false);
+     }
+ 
+     public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
+     {
+         this(cfs, transaction, skipCorrupted, new OutputHandler.LogOutput(), checkData, reinsertOverflowedTTLRows);
      }
  
      @SuppressWarnings("resource")
 -    public Scrubber(ColumnFamilyStore cfs, LifecycleTransaction transaction, boolean skipCorrupted, OutputHandler outputHandler, boolean checkData,
 +    public Scrubber(ColumnFamilyStore cfs,
 +                    LifecycleTransaction transaction,
 +                    boolean skipCorrupted,
 +                    OutputHandler outputHandler,
-                     boolean checkData) throws IOException
++                    boolean checkData,
+                     boolean reinsertOverflowedTTLRows) throws IOException
      {
          this.cfs = cfs;
          this.transaction = transaction;
          this.sstable = transaction.onlyOne();
          this.outputHandler = outputHandler;
          this.skipCorrupted = skipCorrupted;
 -        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata);
+         this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 -
 +        this.rowIndexEntrySerializer = sstable.descriptor.version.getSSTableFormat().getIndexSerializer(sstable.metadata,
 +                                                                                                        sstable.descriptor.version,
 +                                                                                                        sstable.header);
- 
          List<SSTableReader> toScrub = Collections.singletonList(sstable);
  
          // Calculate the expected compacted filesize
@@@ -134,19 -150,15 +145,22 @@@
  
          this.currentRowPositionFromIndex = 0;
          this.nextRowPositionFromIndex = 0;
+ 
+         if (reinsertOverflowedTTLRows)
+             outputHandler.output("Starting scrub with reinsert overflowed TTL option");
      }
  
 +    private UnfilteredRowIterator withValidation(UnfilteredRowIterator iter, String filename)
 +    {
 +        return checkData ? UnfilteredRowIterators.withValidation(iter, filename) : iter;
 +    }
 +
      public void scrub()
      {
 +        List<SSTableReader> finished = new ArrayList<>();
 +        boolean completed = false;
          outputHandler.output(String.format("Scrubbing %s (%s bytes)", sstable, dataFile.length()));
 -        try (SSTableRewriter writer = new SSTableRewriter(cfs, transaction, sstable.maxDataAge, transaction.isOffline());
 +        try (SSTableRewriter writer = SSTableRewriter.construct(cfs, transaction, false, sstable.maxDataAge, transaction.isOffline());
               Refs<SSTableReader> refs = Refs.ref(Collections.singleton(sstable)))
          {
              nextIndexKey = indexAvailable() ? ByteBufferUtil.readWithShortLength(indexFile) : null;
@@@ -285,57 -311,66 +299,71 @@@
          }
          finally
          {
 -            controller.close();
 -            if (transaction.isOffline() && newSstable != null)
 -                newSstable.selfRef().release();
 +            if (transaction.isOffline())
 +                finished.forEach(sstable -> sstable.selfRef().release());
          }
  
 -        if (newSstable == null)
 -        {
 -            if (badRows > 0)
 -                outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
 -            else
 -                outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
 -        }
 -        else
 +        if (completed)
          {
+             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+             if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                 outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
              if (badRows > 0)
-                 outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
-             else
-                 outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
+                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
          }
 +        else
 +        {
-             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
 +            if (badRows > 0)
-                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
++                outputHandler.warn("No valid rows found while scrubbing " + sstable + "; it is marked for deletion now. If you want to attempt manual recovery, you can find a copy in the pre-scrub snapshot");
++            else
++                outputHandler.output("Scrub of " + sstable + " complete; looks like all " + emptyRows + " rows were tombstoned");
 +        }
      }
  
      @SuppressWarnings("resource")
      private boolean tryAppend(DecoratedKey prevKey, DecoratedKey key, SSTableRewriter writer)
      {
 -        // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
 -        // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
 -        // to the outOfOrderRows that will be later written to a new SSTable.
 -        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key),
 -                                                              cfs.metadata.comparator.onDiskAtomComparator());
 -        if (prevKey != null && prevKey.compareTo(key) > 0)
 -        {
 -            saveOutOfOrderRow(prevKey, key, atoms);
 -            return false;
 -        }
 +        // OrderCheckerIterator will check, at iteration time, that the rows are in the proper order. If it detects
 +        // that one row is out of order, it will stop returning them. The remaining rows will be sorted and added
 +        // to the outOfOrder set that will be later written to a new SSTable.
-         OrderCheckerIterator sstableIterator = new OrderCheckerIterator(new RowMergingSSTableIterator(sstable, dataFile, key),
++        OrderCheckerIterator sstableIterator = new OrderCheckerIterator(getIterator(key),
 +                                                                        cfs.metadata.comparator);
  
 -        AbstractCompactedRow compactedRow = new LazilyCompactedRow(controller, Collections.singletonList(atoms));
 -        if (writer.tryAppend(compactedRow) == null)
 -            emptyRows++;
 -        else
 -            goodRows++;
 +        try (UnfilteredRowIterator iterator = withValidation(sstableIterator, dataFile.getPath()))
 +        {
 +            if (prevKey != null && prevKey.compareTo(key) > 0)
 +            {
 +                saveOutOfOrderRow(prevKey, key, iterator);
 +                return false;
 +            }
  
 -        if (atoms.hasOutOfOrderCells())
 -            saveOutOfOrderRow(key, atoms);
 +            if (writer.tryAppend(iterator) == null)
 +                emptyRows++;
 +            else
 +                goodRows++;
 +        }
 +
 +        if (sstableIterator.hasRowsOutOfOrder())
 +        {
 +            outputHandler.warn(String.format("Out of order rows found in partition: %s", key));
 +            outOfOrder.add(sstableIterator.getRowsOutOfOrder());
 +        }
  
          return true;
      }
  
+     /**
+      * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+      * is specified
+      */
 -    private OnDiskAtomIterator getIterator(DecoratedKey key)
++    private UnfilteredRowIterator getIterator(DecoratedKey key)
+     {
 -        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, checkData);
 -        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
++        RowMergingSSTableIterator rowMergingIterator = new RowMergingSSTableIterator(sstable, dataFile, key);
++        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(rowMergingIterator,
+                                                                                     outputHandler,
 -                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
++                                                                                    negativeLocalDeletionInfoMetrics) : rowMergingIterator;
+     }
+ 
      private void updateIndexKey()
      {
          currentIndexKey = nextIndexKey;
@@@ -477,49 -555,12 +505,55 @@@
          }
      }
  
+     public class NegativeLocalDeletionInfoMetrics
+     {
+         public volatile int fixedRows = 0;
+     }
+ 
      /**
 +     * During 2.x migration, under some circumstances rows might have gotten duplicated.
 +     * Merging iterator merges rows with same clustering.
 +     *
 +     * For more details, refer to CASSANDRA-12144.
 +     */
 +    private static class RowMergingSSTableIterator extends SSTableIdentityIterator
 +    {
 +        RowMergingSSTableIterator(SSTableReader sstable, RandomAccessReader file, DecoratedKey key)
 +        {
 +            super(sstable, file, key);
 +        }
 +
 +        @Override
 +        protected Unfiltered doCompute()
 +        {
 +            if (!iterator.hasNext())
 +                return endOfData();
 +
 +            Unfiltered next = iterator.next();
 +            if (!next.isRow())
 +                return next;
 +
 +            while (iterator.hasNext())
 +            {
 +                Unfiltered peek = iterator.peek();
 +                // If there was a duplicate row, merge it.
 +                if (next.clustering().equals(peek.clustering()) && peek.isRow())
 +                {
 +                    iterator.next(); // Make sure that the peeked item was consumed.
 +                    next = Rows.merge((Row) next, (Row) peek, FBUtilities.nowInSeconds());
 +                }
 +                else
 +                {
 +                    break;
 +                }
 +            }
 +
 +            return next;
 +        }
++
 +    }
 +
 +    /**
       * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
       * cells order and collect the out of order cells to correct the problem.
       */
@@@ -571,53 -608,98 +605,199 @@@
          }
  
          @Override
 -        protected OnDiskAtom computeNext()
 +        public boolean isEmpty()
          {
 -            if (!iterator.hasNext())
 -                return endOfData();
 +            return iterator.isEmpty();
 +        }
  
 -            OnDiskAtom next = iterator.next();
 +        public void close()
 +        {
 +            iterator.close();
 +        }
  
 -            // If we detect that some cells are out of order we will store and sort the remaining once to insert them
 -            // in a separate SSTable.
 -            if (previous != null && comparator.compare(next, previous) < 0)
 -            {
 -                outOfOrderCells = collectOutOfOrderCells(next, iterator);
 -                return endOfData();
 -            }
 -            previous = next;
 -            return next;
 +        public DeletionTime partitionLevelDeletion()
 +        {
 +            return iterator.partitionLevelDeletion();
          }
  
 -        public boolean hasOutOfOrderCells()
 +        public EncodingStats stats()
          {
 -            return outOfOrderCells != null;
 +            return iterator.stats();
          }
  
 -        public ColumnFamily getOutOfOrderCells()
 +        public boolean hasRowsOutOfOrder()
          {
 -            return outOfOrderCells;
 +            return rowsOutOfOrder != null;
          }
  
 -        private static ColumnFamily collectOutOfOrderCells(OnDiskAtom atom, OnDiskAtomIterator iterator)
 +        public Partition getRowsOutOfOrder()
          {
 -            ColumnFamily cf = iterator.getColumnFamily().cloneMeShallow(ArrayBackedSortedColumns.factory, false);
 -            cf.addAtom(atom);
 -            while (iterator.hasNext())
 -                cf.addAtom(iterator.next());
 -            return cf;
 +            return rowsOutOfOrder;
 +        }
 +
 +        protected Unfiltered computeNext()
 +        {
 +            if (!iterator.hasNext())
 +                return endOfData();
 +
 +            Unfiltered next = iterator.next();
 +
 +            // If we detect that some rows are out of order we will store and sort the remaining ones to insert them
 +            // in a separate SSTable.
 +            if (previous != null && comparator.compare(next, previous) < 0)
 +            {
 +                rowsOutOfOrder = ImmutableBTreePartition.create(UnfilteredRowIterators.concat(next, iterator), false);
 +                return endOfData();
 +            }
 +            previous = next;
 +            return next;
          }
+     }
+ 
+     /**
 -     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
++     * This iterator converts negative {@link AbstractCell#localDeletionTime()} into {@link AbstractCell#MAX_DELETION_TIME}
+      *
+      * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+      */
 -    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
++    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<Unfiltered> implements UnfilteredRowIterator
+     {
+         /**
+          * The decorated iterator.
+          */
 -        private final OnDiskAtomIterator iterator;
++        private final UnfilteredRowIterator iterator;
+ 
+         private final OutputHandler outputHandler;
+         private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+ 
 -        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
++        public FixNegativeLocalDeletionTimeIterator(UnfilteredRowIterator iterator, OutputHandler outputHandler,
+                                                     NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+         {
+             this.iterator = iterator;
+             this.outputHandler = outputHandler;
+             this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+         }
+ 
 -        public ColumnFamily getColumnFamily()
++        public CFMetaData metadata()
+         {
 -            return iterator.getColumnFamily();
++            return iterator.metadata();
+         }
+ 
 -        public DecoratedKey getKey()
++        public boolean isReverseOrder()
+         {
 -            return iterator.getKey();
++            return iterator.isReverseOrder();
+         }
+ 
 -        public void close() throws IOException
++        public PartitionColumns columns()
+         {
 -            iterator.close();
++            return iterator.columns();
++        }
++
++        public DecoratedKey partitionKey()
++        {
++            return iterator.partitionKey();
++        }
++
++        public Row staticRow()
++        {
++            return iterator.staticRow();
+         }
+ 
+         @Override
 -        protected OnDiskAtom computeNext()
++        public boolean isEmpty()
++        {
++            return iterator.isEmpty();
++        }
++
++        public void close()
++        {
++            iterator.close();
++        }
++
++        public DeletionTime partitionLevelDeletion()
++        {
++            return iterator.partitionLevelDeletion();
++        }
++
++        public EncodingStats stats()
++        {
++            return iterator.stats();
++        }
++
++        protected Unfiltered computeNext()
+         {
+             if (!iterator.hasNext())
+                 return endOfData();
+ 
 -            OnDiskAtom next = iterator.next();
++            Unfiltered next = iterator.next();
++            if (!next.isRow())
++                return next;
+ 
 -            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
++            if (hasNegativeLocalExpirationTime((Row) next))
+             {
 -                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
++                outputHandler.debug(String.format("Found row with negative local expiration time: %s", next.toString(metadata(), false)));
+                 negativeLocalExpirationTimeMetrics.fixedRows++;
 -                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
++                return fixNegativeLocalExpirationTime((Row) next);
+             }
+ 
+             return next;
+         }
++
++        private boolean hasNegativeLocalExpirationTime(Row next)
++        {
++            Row row = next;
++            if (row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0)
++            {
++                return true;
++            }
++
++            for (ColumnData cd : row)
++            {
++                if (cd.column().isSimple())
++                {
++                    Cell cell = (Cell)cd;
++                    if (cell.isExpiring() && cell.localDeletionTime() < 0)
++                        return true;
++                }
++                else
++                {
++                    ComplexColumnData complexData = (ComplexColumnData)cd;
++                    for (Cell cell : complexData)
++                    {
++                        if (cell.isExpiring() && cell.localDeletionTime() < 0)
++                            return true;
++                    }
++                }
++            }
++
++            return false;
++        }
 +
++        private Unfiltered fixNegativeLocalExpirationTime(Row row)
++        {
++            Row.Builder builder = HeapAllocator.instance.cloningBTreeRowBuilder();
++            builder.newRow(row.clustering());
++            builder.addPrimaryKeyLivenessInfo(row.primaryKeyLivenessInfo().isExpiring() && row.primaryKeyLivenessInfo().localExpirationTime() < 0 ?
++                                              row.primaryKeyLivenessInfo().withUpdatedTimestampAndLocalDeletionTime(row.primaryKeyLivenessInfo().timestamp() + 1, AbstractCell.MAX_DELETION_TIME)
++                                              :row.primaryKeyLivenessInfo());
++            builder.addRowDeletion(row.deletion());
++            for (ColumnData cd : row)
++            {
++                if (cd.column().isSimple())
++                {
++                    Cell cell = (Cell)cd;
++                    builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
++                }
++                else
++                {
++                    ComplexColumnData complexData = (ComplexColumnData)cd;
++                    builder.addComplexDeletion(complexData.column(), complexData.complexDeletion());
++                    for (Cell cell : complexData)
++                    {
++                        builder.addCell(cell.isExpiring() && cell.localDeletionTime() < 0 ? cell.withUpdatedTimestampAndLocalDeletionTime(cell.timestamp() + 1, AbstractCell.MAX_DELETION_TIME) : cell);
++                    }
++                }
++            }
++            return builder.build();
++        }
      }
  }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/BufferCell.java
index 82ae02c,0000000..df2619c
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@@ -1,365 -1,0 +1,370 @@@
 +/*
 + * 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.io.IOException;
 +import java.nio.ByteBuffer;
 +
 +import org.apache.cassandra.config.*;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.db.context.CounterContext;
 +import org.apache.cassandra.db.marshal.ByteType;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.utils.ByteBufferUtil;
 +import org.apache.cassandra.utils.ObjectSizes;
 +import org.apache.cassandra.utils.FBUtilities;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +
 +public class BufferCell extends AbstractCell
 +{
 +    private static final long EMPTY_SIZE = ObjectSizes.measure(new BufferCell(ColumnDefinition.regularDef("", "", "", ByteType.instance), 0L, 0, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER, null));
 +
 +    private final long timestamp;
 +    private final int ttl;
 +    private final int localDeletionTime;
 +
 +    private final ByteBuffer value;
 +    private final CellPath path;
 +
 +    public BufferCell(ColumnDefinition column, long timestamp, int ttl, int localDeletionTime, ByteBuffer value, CellPath path)
 +    {
 +        super(column);
 +        assert column.isComplex() == (path != null);
 +        this.timestamp = timestamp;
 +        this.ttl = ttl;
 +        this.localDeletionTime = localDeletionTime;
 +        this.value = value;
 +        this.path = path;
 +    }
 +
 +    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value)
 +    {
 +        return live(metadata, column, timestamp, value, null);
 +    }
 +
 +    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value, CellPath path)
 +    {
 +        if (metadata.params.defaultTimeToLive != NO_TTL)
 +            return expiring(column, timestamp, metadata.params.defaultTimeToLive, FBUtilities.nowInSeconds(), value, path);
 +
 +        return new BufferCell(column, timestamp, NO_TTL, NO_DELETION_TIME, value, path);
 +    }
 +
 +    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value)
 +    {
 +        return expiring(column, timestamp, ttl, nowInSec, value, null);
 +    }
 +
 +    public static BufferCell expiring(ColumnDefinition column, long timestamp, int ttl, int nowInSec, ByteBuffer value, CellPath path)
 +    {
 +        assert ttl != NO_TTL;
-         return new BufferCell(column, timestamp, ttl, nowInSec + ttl, value, path);
++        return new BufferCell(column, timestamp, ttl, ExpirationDateOverflowHandling.computeLocalExpirationTime(nowInSec, ttl), value, path);
 +    }
 +
 +    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec)
 +    {
 +        return tombstone(column, timestamp, nowInSec, null);
 +    }
 +
 +    public static BufferCell tombstone(ColumnDefinition column, long timestamp, int nowInSec, CellPath path)
 +    {
 +        return new BufferCell(column, timestamp, NO_TTL, nowInSec, ByteBufferUtil.EMPTY_BYTE_BUFFER, path);
 +    }
 +
 +    public boolean isCounterCell()
 +    {
 +        return !isTombstone() && column.isCounterColumn();
 +    }
 +
 +    public boolean isLive(int nowInSec)
 +    {
 +        return localDeletionTime == NO_DELETION_TIME || (ttl != NO_TTL && nowInSec < localDeletionTime);
 +    }
 +
 +    public boolean isTombstone()
 +    {
 +        return localDeletionTime != NO_DELETION_TIME && ttl == NO_TTL;
 +    }
 +
 +    public boolean isExpiring()
 +    {
 +        return ttl != NO_TTL;
 +    }
 +
 +    public long timestamp()
 +    {
 +        return timestamp;
 +    }
 +
 +    public int ttl()
 +    {
 +        return ttl;
 +    }
 +
 +    public int localDeletionTime()
 +    {
 +        return localDeletionTime;
 +    }
 +
 +    public ByteBuffer value()
 +    {
 +        return value;
 +    }
 +
 +    public CellPath path()
 +    {
 +        return path;
 +    }
 +
 +    public Cell withUpdatedColumn(ColumnDefinition newColumn)
 +    {
 +        return new BufferCell(newColumn, timestamp, ttl, localDeletionTime, value, path);
 +    }
 +
 +    public Cell withUpdatedValue(ByteBuffer newValue)
 +    {
 +        return new BufferCell(column, timestamp, ttl, localDeletionTime, newValue, path);
 +    }
 +
++    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
++    {
++        return new BufferCell(column, newTimestamp, ttl, newLocalDeletionTime, value, path);
++    }
++
 +    public Cell copy(AbstractAllocator allocator)
 +    {
 +        if (!value.hasRemaining())
 +            return this;
 +
 +        return new BufferCell(column, timestamp, ttl, localDeletionTime, allocator.clone(value), path == null ? null : path.copy(allocator));
 +    }
 +
 +    public Cell markCounterLocalToBeCleared()
 +    {
 +        if (!isCounterCell())
 +            return this;
 +
 +        ByteBuffer marked = CounterContext.instance().markLocalToBeCleared(value());
 +        return marked == value() ? this : new BufferCell(column, timestamp, ttl, localDeletionTime, marked, path);
 +    }
 +
 +    public Cell purge(DeletionPurger purger, int nowInSec)
 +    {
 +        if (!isLive(nowInSec))
 +        {
 +            if (purger.shouldPurge(timestamp, localDeletionTime))
 +                return null;
 +
 +            // We slightly hijack purging to convert expired but not purgeable columns to tombstones. The reason we do that is
 +            // that once a column has expired it is equivalent to a tombstone but actually using a tombstone is more compact since
 +            // we don't keep the column value. The reason we do it here is that 1) it's somewhat related to dealing with tombstones
 +            // so hopefully not too surprising and 2) we want to this and purging at the same places, so it's simpler/more efficient
 +            // to do both here.
 +            if (isExpiring())
 +            {
 +                // Note that as long as the expiring column and the tombstone put together live longer than GC grace seconds,
 +                // we'll fulfil our responsibility to repair. See discussion at
 +                // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
 +                return BufferCell.tombstone(column, timestamp, localDeletionTime - ttl, path).purge(purger, nowInSec);
 +            }
 +        }
 +        return this;
 +    }
 +
 +    public Cell updateAllTimestamp(long newTimestamp)
 +    {
 +        return new BufferCell(column, isTombstone() ? newTimestamp - 1 : newTimestamp, ttl, localDeletionTime, value, path);
 +    }
 +
 +    public int dataSize()
 +    {
 +        return TypeSizes.sizeof(timestamp)
 +             + TypeSizes.sizeof(ttl)
 +             + TypeSizes.sizeof(localDeletionTime)
 +             + value.remaining()
 +             + (path == null ? 0 : path.dataSize());
 +    }
 +
 +    public long unsharedHeapSizeExcludingData()
 +    {
 +        return EMPTY_SIZE + ObjectSizes.sizeOnHeapExcludingData(value) + (path == null ? 0 : path.unsharedHeapSizeExcludingData());
 +    }
 +
 +    /**
 +     * The serialization format for cell is:
 +     *     [ flags ][ timestamp ][ deletion time ][    ttl    ][ path size ][ path ][ value size ][ value ]
 +     *     [   1b  ][ 8b (vint) ][   4b (vint)   ][ 4b (vint) ][ 4b (vint) ][  arb ][  4b (vint) ][  arb  ]
 +     *
 +     * where not all field are always present (in fact, only the [ flags ] are guaranteed to be present). The fields have the following
 +     * meaning:
 +     *   - [ flags ] is the cell flags. It is a byte for which each bit represents a flag whose meaning is explained below (*_MASK constants)
 +     *   - [ timestamp ] is the cell timestamp. Present unless the cell has the USE_TIMESTAMP_MASK.
 +     *   - [ deletion time]: the local deletion time for the cell. Present if either the cell is deleted (IS_DELETED_MASK)
 +     *       or it is expiring (IS_EXPIRING_MASK) but doesn't have the USE_ROW_TTL_MASK.
 +     *   - [ ttl ]: the ttl for the cell. Present if the row is expiring (IS_EXPIRING_MASK) but doesn't have the
 +     *       USE_ROW_TTL_MASK.
 +     *   - [ value size ] is the size of the [ value ] field. It's present unless either the cell has the HAS_EMPTY_VALUE_MASK, or the value
 +     *       for columns of this type have a fixed length.
 +     *   - [ path size ] is the size of the [ path ] field. Present iff this is the cell of a complex column.
 +     *   - [ value ]: the cell value, unless it has the HAS_EMPTY_VALUE_MASK.
 +     *   - [ path ]: the cell path if the column this is a cell of is complex.
 +     */
 +    static class Serializer implements Cell.Serializer
 +    {
 +        private final static int IS_DELETED_MASK             = 0x01; // Whether the cell is a tombstone or not.
 +        private final static int IS_EXPIRING_MASK            = 0x02; // Whether the cell is expiring.
 +        private final static int HAS_EMPTY_VALUE_MASK        = 0x04; // Wether the cell has an empty value. This will be the case for tombstone in particular.
 +        private final static int USE_ROW_TIMESTAMP_MASK      = 0x08; // Wether the cell has the same timestamp than the row this is a cell of.
 +        private final static int USE_ROW_TTL_MASK            = 0x10; // Wether the cell has the same ttl than the row this is a cell of.
 +
 +        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException
 +        {
 +            assert cell != null;
 +            boolean hasValue = cell.value().hasRemaining();
 +            boolean isDeleted = cell.isTombstone();
 +            boolean isExpiring = cell.isExpiring();
 +            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
 +            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
 +            int flags = 0;
 +            if (!hasValue)
 +                flags |= HAS_EMPTY_VALUE_MASK;
 +
 +            if (isDeleted)
 +                flags |= IS_DELETED_MASK;
 +            else if (isExpiring)
 +                flags |= IS_EXPIRING_MASK;
 +
 +            if (useRowTimestamp)
 +                flags |= USE_ROW_TIMESTAMP_MASK;
 +            if (useRowTTL)
 +                flags |= USE_ROW_TTL_MASK;
 +
 +            out.writeByte((byte)flags);
 +
 +            if (!useRowTimestamp)
 +                header.writeTimestamp(cell.timestamp(), out);
 +
 +            if ((isDeleted || isExpiring) && !useRowTTL)
 +                header.writeLocalDeletionTime(cell.localDeletionTime(), out);
 +            if (isExpiring && !useRowTTL)
 +                header.writeTTL(cell.ttl(), out);
 +
 +            if (column.isComplex())
 +                column.cellPathSerializer().serialize(cell.path(), out);
 +
 +            if (hasValue)
 +                header.getType(column).writeValue(cell.value(), out);
 +        }
 +
 +        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException
 +        {
 +            int flags = in.readUnsignedByte();
 +            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
 +            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
 +            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
 +            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
 +            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
 +
 +            long timestamp = useRowTimestamp ? rowLiveness.timestamp() : header.readTimestamp(in);
 +
 +            int localDeletionTime = useRowTTL
 +                                  ? rowLiveness.localExpirationTime()
 +                                  : (isDeleted || isExpiring ? header.readLocalDeletionTime(in) : NO_DELETION_TIME);
 +
 +            int ttl = useRowTTL ? rowLiveness.ttl() : (isExpiring ? header.readTTL(in) : NO_TTL);
 +
 +            CellPath path = column.isComplex()
 +                          ? column.cellPathSerializer().deserialize(in)
 +                          : null;
 +
 +            boolean isCounter = localDeletionTime == NO_DELETION_TIME && column.type.isCounter();
 +
 +            ByteBuffer value = ByteBufferUtil.EMPTY_BYTE_BUFFER;
 +            if (hasValue)
 +            {
 +                if (helper.canSkipValue(column) || (path != null && helper.canSkipValue(path)))
 +                {
 +                    header.getType(column).skipValue(in);
 +                }
 +                else
 +                {
 +                    value = header.getType(column).readValue(in, DatabaseDescriptor.getMaxValueSize());
 +                    if (isCounter)
 +                        value = helper.maybeClearCounterValue(value);
 +                }
 +            }
 +
 +            return new BufferCell(column, timestamp, ttl, localDeletionTime, value, path);
 +        }
 +
 +        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header)
 +        {
 +            long size = 1; // flags
 +            boolean hasValue = cell.value().hasRemaining();
 +            boolean isDeleted = cell.isTombstone();
 +            boolean isExpiring = cell.isExpiring();
 +            boolean useRowTimestamp = !rowLiveness.isEmpty() && cell.timestamp() == rowLiveness.timestamp();
 +            boolean useRowTTL = isExpiring && rowLiveness.isExpiring() && cell.ttl() == rowLiveness.ttl() && cell.localDeletionTime() == rowLiveness.localExpirationTime();
 +
 +            if (!useRowTimestamp)
 +                size += header.timestampSerializedSize(cell.timestamp());
 +
 +            if ((isDeleted || isExpiring) && !useRowTTL)
 +                size += header.localDeletionTimeSerializedSize(cell.localDeletionTime());
 +            if (isExpiring && !useRowTTL)
 +                size += header.ttlSerializedSize(cell.ttl());
 +
 +            if (column.isComplex())
 +                size += column.cellPathSerializer().serializedSize(cell.path());
 +
 +            if (hasValue)
 +                size += header.getType(column).writtenLength(cell.value());
 +
 +            return size;
 +        }
 +
 +        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
 +        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException
 +        {
 +            int flags = in.readUnsignedByte();
 +            boolean hasValue = (flags & HAS_EMPTY_VALUE_MASK) == 0;
 +            boolean isDeleted = (flags & IS_DELETED_MASK) != 0;
 +            boolean isExpiring = (flags & IS_EXPIRING_MASK) != 0;
 +            boolean useRowTimestamp = (flags & USE_ROW_TIMESTAMP_MASK) != 0;
 +            boolean useRowTTL = (flags & USE_ROW_TTL_MASK) != 0;
 +
 +            if (!useRowTimestamp)
 +                header.skipTimestamp(in);
 +
 +            if (!useRowTTL && (isDeleted || isExpiring))
 +                header.skipLocalDeletionTime(in);
 +
 +            if (!useRowTTL && isExpiring)
 +                header.skipTTL(in);
 +
 +            if (column.isComplex())
 +                column.cellPathSerializer().skip(in);
 +
 +            if (hasValue)
 +                header.getType(column).skipValue(in);
 +
 +            return true;
 +        }
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/db/rows/Cell.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/db/rows/Cell.java
index d10cc74,0000000..c69e11f
mode 100644,000000..100644
--- a/src/java/org/apache/cassandra/db/rows/Cell.java
+++ b/src/java/org/apache/cassandra/db/rows/Cell.java
@@@ -1,157 -1,0 +1,166 @@@
 +/*
 + * 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.io.IOException;
 +import java.nio.ByteBuffer;
 +import java.util.Comparator;
 +
++import com.google.common.annotations.VisibleForTesting;
++import org.slf4j.Logger;
++import org.slf4j.LoggerFactory;
++
++import org.apache.cassandra.config.CFMetaData;
 +import org.apache.cassandra.config.ColumnDefinition;
++import org.apache.cassandra.cql3.Attributes;
 +import org.apache.cassandra.db.*;
 +import org.apache.cassandra.io.util.DataOutputPlus;
 +import org.apache.cassandra.io.util.DataInputPlus;
 +import org.apache.cassandra.utils.memory.AbstractAllocator;
 +
 +/**
 + * A cell is our atomic unit for a single value of a single column.
 + * <p>
 + * A cell always holds at least a timestamp that gives us how the cell reconcile. We then
 + * have 3 main types of cells:
 + *   1) live regular cells: those will also have a value and, if for a complex column, a path.
 + *   2) expiring cells: on top of regular cells, those have a ttl and a local deletion time (when they are expired).
 + *   3) tombstone cells: those won't have value, but they have a local deletion time (when the tombstone was created).
 + */
 +public abstract class Cell extends ColumnData
 +{
 +    public static final int NO_TTL = 0;
 +    public static final int NO_DELETION_TIME = Integer.MAX_VALUE;
++    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
 +
 +    public final static Comparator<Cell> comparator = (c1, c2) ->
 +    {
 +        int cmp = c1.column().compareTo(c2.column());
 +        if (cmp != 0)
 +            return cmp;
 +
 +        Comparator<CellPath> pathComparator = c1.column().cellPathComparator();
 +        return pathComparator == null ? 0 : pathComparator.compare(c1.path(), c2.path());
 +    };
 +
 +    public static final Serializer serializer = new BufferCell.Serializer();
 +
 +    protected Cell(ColumnDefinition column)
 +    {
 +        super(column);
 +    }
 +
 +    /**
 +     * Whether the cell is a counter cell or not.
 +     *
 +     * @return whether the cell is a counter cell or not.
 +     */
 +    public abstract boolean isCounterCell();
 +
 +    /**
 +     * The cell value.
 +     *
 +     * @return the cell value.
 +     */
 +    public abstract ByteBuffer value();
 +
 +    /**
 +     * The cell timestamp.
 +     * <p>
 +     * @return the cell timestamp.
 +     */
 +    public abstract long timestamp();
 +
 +    /**
 +     * The cell ttl.
 +     *
 +     * @return the cell ttl, or {@code NO_TTL} if the cell isn't an expiring one.
 +     */
 +    public abstract int ttl();
 +
 +    /**
 +     * The cell local deletion time.
 +     *
 +     * @return the cell local deletion time, or {@code NO_DELETION_TIME} if the cell is neither
 +     * a tombstone nor an expiring one.
 +     */
 +    public abstract int localDeletionTime();
 +
 +    /**
 +     * Whether the cell is a tombstone or not.
 +     *
 +     * @return whether the cell is a tombstone or not.
 +     */
 +    public abstract boolean isTombstone();
 +
 +    /**
 +     * Whether the cell is an expiring one or not.
 +     * <p>
 +     * Note that this only correspond to whether the cell liveness info
 +     * have a TTL or not, but doesn't tells whether the cell is already expired
 +     * or not. You should use {@link #isLive} for that latter information.
 +     *
 +     * @return whether the cell is an expiring one or not.
 +     */
 +    public abstract boolean isExpiring();
 +
 +    /**
 +     * Whether the cell is live or not given the current time.
 +     *
 +     * @param nowInSec the current time in seconds. This is used to
 +     * decide if an expiring cell is expired or live.
 +     * @return whether the cell is live or not at {@code nowInSec}.
 +     */
 +    public abstract boolean isLive(int nowInSec);
 +
 +    /**
 +     * For cells belonging to complex types (non-frozen collection and UDT), the
 +     * path to the cell.
 +     *
 +     * @return the cell path for cells of complex column, and {@code null} for other cells.
 +     */
 +    public abstract CellPath path();
 +
 +    public abstract Cell withUpdatedColumn(ColumnDefinition newColumn);
 +
 +    public abstract Cell withUpdatedValue(ByteBuffer newValue);
 +
++    public abstract Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
++
 +    public abstract Cell copy(AbstractAllocator allocator);
 +
 +    @Override
 +    // Overrides super type to provide a more precise return type.
 +    public abstract Cell markCounterLocalToBeCleared();
 +
 +    @Override
 +    // Overrides super type to provide a more precise return type.
 +    public abstract Cell purge(DeletionPurger purger, int nowInSec);
 +
 +    public interface Serializer
 +    {
 +        public void serialize(Cell cell, ColumnDefinition column, DataOutputPlus out, LivenessInfo rowLiveness, SerializationHeader header) throws IOException;
 +
 +        public Cell deserialize(DataInputPlus in, LivenessInfo rowLiveness, ColumnDefinition column, SerializationHeader header, SerializationHelper helper) throws IOException;
 +
 +        public long serializedSize(Cell cell, ColumnDefinition column, LivenessInfo rowLiveness, SerializationHeader header);
 +
 +        // Returns if the skipped cell was an actual cell (i.e. it had its presence flag).
 +        public boolean skip(DataInputPlus in, ColumnDefinition column, SerializationHeader header) throws IOException;
 +    }
 +}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageService.java
index e5a50dd,2c9ac4d..cf8e257
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@@ -2705,22 -2599,28 +2705,27 @@@ public class StorageService extends Not
          return status.statusCode;
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, columnFamilies);
 +        return scrub(disableSnapshot, skipCorrupted, true, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, columnFamilies);
 +        return scrub(disableSnapshot, skipCorrupted, checkData, 0, keyspaceName, tables);
      }
  
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
++        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, tables);
+     }
+ 
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
 -                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
+     {
          CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
 -        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, columnFamilies))
 +        for (ColumnFamilyStore cfStore : getValidColumnFamilies(true, false, keyspaceName, tables))
          {
-             CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
 -            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
++            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, reinsertOverflowedTTL, checkData, jobs);
              if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                  status = oneStatus;
          }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/service/StorageServiceMBean.java
index 7344ca8,f336bcc..10d47f7
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@@ -262,14 -262,17 +262,17 @@@ public interface StorageServiceMBean ex
       * Scrubbed CFs will be snapshotted first, if disableSnapshot is false
       */
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
      @Deprecated
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 +    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException;
+     @Deprecated
      public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
  
 -public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+ 
      /**
       * Verify (checksums of) the given keyspace.
 -     * If columnFamilies array is empty, all CFs are verified.
 +     * If tableNames array is empty, all CFs are verified.
       *
       * The entire sstable will be read to ensure each cell validates if extendedVerify is true
       */

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/thrift/ThriftValidation.java
index 6ad791d,8bdf9dc..2ab0330
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@@ -332,9 -315,9 +332,9 @@@ public class ThriftValidatio
              if (isCommutative)
                  throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative table " + metadata.cfName);
  
-             validateTtl(cosc.column);
+             validateTtl(metadata, cosc.column);
              validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
 -            validateColumnData(metadata, key, null, cosc.column);
 +            validateColumnData(metadata, null, cosc.column);
          }
  
          if (cosc.super_column != null)
@@@ -374,11 -357,13 +374,13 @@@
              if (column.ttl <= 0)
                  throw new org.apache.cassandra.exceptions.InvalidRequestException("ttl must be positive");
  
 -            if (column.ttl > ExpiringCell.MAX_TTL)
 -                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
 -            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
 +            if (column.ttl > Attributes.MAX_TTL)
 +                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, Attributes.MAX_TTL));
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
          }
          else
          {
 -            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
++            ExpirationDateOverflowHandling.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.params.defaultTimeToLive, true);
              // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
              assert column.ttl == 0;
          }
@@@ -450,9 -435,9 +452,9 @@@
      /**
       * Validates the data part of the column (everything in the column object but the name, which is assumed to be valid)
       */
 -    public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
 +    public static void validateColumnData(CFMetaData metadata, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
      {
-         validateTtl(column);
+         validateTtl(metadata, column);
          if (!column.isSetValue())
              throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
          if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/NodeProbe.java
index 172b505,17bef02..0d3c078
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@@ -235,35 -228,27 +235,35 @@@ public class NodeProbe implements AutoC
  
      public void close() throws IOException
      {
 -        jmxc.close();
 +        try
 +        {
 +            jmxc.close();
 +        }
 +        catch (ConnectException e)
 +        {
 +            // result of 'stopdaemon' command - i.e. if close() call fails, the daemon is shutdown
 +            System.out.println("Cassandra has shutdown.");
 +        }
      }
  
 -    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int forceKeyspaceCleanup(int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
 +        return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, tables);
      }
  
-     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
-         return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables);
 -        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
++        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTL, jobs, keyspaceName, tables);
      }
  
 -    public int verify(boolean extendedVerify, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int verify(boolean extendedVerify, String keyspaceName, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.verify(extendedVerify, keyspaceName, columnFamilies);
 +        return ssProxy.verify(extendedVerify, keyspaceName, tableNames);
      }
  
 -    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
 +    public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... tableNames) throws IOException, ExecutionException, InterruptedException
      {
 -        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, columnFamilies);
 +        return ssProxy.upgradeSSTables(keyspaceName, excludeCurrentVersion, jobs, tableNames);
      }
  
      private void checkJobs(PrintStream out, int jobs)
@@@ -288,19 -267,13 +288,19 @@@
          }
      }
  
-     public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
 -    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
++    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTL, int jobs, String keyspaceName, String... tables) throws IOException, ExecutionException, InterruptedException
      {
          checkJobs(out, jobs);
-         switch (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, tables))
 -        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
++        switch (ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTL, jobs, keyspaceName, tables))
          {
 -            failed = true;
 -            out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +            case 1:
 +                failed = true;
 +                out.println("Aborted scrubbing at least one table in keyspace "+keyspaceName+", check server logs for more information.");
 +                break;
 +            case 2:
 +                failed = true;
 +                out.println("Failed marking some sstables compacting in keyspace "+keyspaceName+", check server logs for more information");
 +                break;
          }
      }
  

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
index 6076e32,8319014..19af957
--- a/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
+++ b/src/java/org/apache/cassandra/tools/SSTableMetadataViewer.java
@@@ -90,12 -65,13 +90,13 @@@ public class SSTableMetadataViewe
                  {
                      out.printf("Minimum timestamp: %s%n", stats.minTimestamp);
                      out.printf("Maximum timestamp: %s%n", stats.maxTimestamp);
++                    out.printf("SSTable min local deletion time: %s%n", stats.minLocalDeletionTime);
                      out.printf("SSTable max local deletion time: %s%n", stats.maxLocalDeletionTime);
                      out.printf("Compression ratio: %s%n", stats.compressionRatio);
 -                    out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000)));
 +                    out.printf("Estimated droppable tombstones: %s%n", stats.getEstimatedDroppableTombstoneRatio((int) (System.currentTimeMillis() / 1000) - gcgs));
                      out.printf("SSTable Level: %d%n", stats.sstableLevel);
                      out.printf("Repaired at: %d%n", stats.repairedAt);
 -                    out.printf("Minimum replay position: %s\n", stats.commitLogLowerBound);
 -                    out.printf("Maximum replay position: %s\n", stats.commitLogUpperBound);
 +                    out.printf("Replay positions covered: %s\n", stats.commitLogIntervals);
                      out.println("Estimated tombstone drop times:");
                      for (Map.Entry<Double, Long> entry : stats.estimatedTombstoneDropTime.getAsMap().entrySet())
                      {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index 4249430,f5e84c5..4778d72
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@@ -122,7 -129,7 +129,7 @@@ public class StandaloneScrubbe
                      try (LifecycleTransaction txn = LifecycleTransaction.offline(OperationType.SCRUB, sstable))
                      {
                          txn.obsoleteOriginals(); // make sure originals are deleted and avoid NPE if index is missing, CASSANDRA-9591
-                         try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate))
 -                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinsertOverflowedTTL))
++                        try (Scrubber scrubber = new Scrubber(cfs, txn, options.skipCorrupted, handler, !options.noValidate, options.reinserOverflowedTTL))
                          {
                              scrubber.scrub();
                          }
@@@ -199,6 -207,7 +206,7 @@@
          public boolean manifestCheckOnly;
          public boolean skipCorrupted;
          public boolean noValidate;
 -        public boolean reinsertOverflowedTTL;
++        public boolean reinserOverflowedTTL;
  
          private Options(String keyspaceName, String cfName)
          {
@@@ -239,6 -248,7 +247,7 @@@
                  opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                  opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                  opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
 -                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
++                opts.reinserOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
  
                  return opts;
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
----------------------------------------------------------------------
diff --cc src/java/org/apache/cassandra/tools/nodetool/Scrub.java
index 2345a85,50224a0..ead2fd4
--- a/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
+++ b/src/java/org/apache/cassandra/tools/nodetool/Scrub.java
@@@ -48,11 -49,16 +49,16 @@@ public class Scrub extends NodeToolCm
                     description = "Do not validate columns using column validator")
      private boolean noValidation = false;
  
 -    @Option(title = "jobs",
 -            name = {"-j", "--jobs"},
 -            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 -    private int jobs = 2;
 -
+     @Option(title = "reinsert_overflowed_ttl",
+     name = {"r", "--reinsert-overflowed-ttl"},
+     description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+     private boolean reinsertOverflowedTTL = false;
+ 
 +    @Option(title = "jobs",
 +            name = {"-j", "--jobs"},
 +            description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
 +    private int jobs = 2;
 +
      @Override
      public void execute(NodeProbe probe)
      {
@@@ -63,13 -69,11 +69,13 @@@
          {
              try
              {
-                 probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, tableNames);
 -                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, cfnames);
 -            } catch (IllegalArgumentException e)
++                probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, reinsertOverflowedTTL, jobs, keyspace, tableNames);
 +            }
 +            catch (IllegalArgumentException e)
              {
                  throw e;
 -            } catch (Exception e)
 +            }
 +            catch (Exception e)
              {
                  throw new RuntimeException("Error occurred during scrubbing", e);
              }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-CompressionInfo.db
index 0000000,0000000..d759cec
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Data.db
index 0000000,0000000..e7a72da
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
index 0000000,0000000..a3c633a
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++203700622

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Index.db
index 0000000,0000000..d742724
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Statistics.db
index 0000000,0000000..faf367b
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-Summary.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-Summary.db
index 0000000,0000000..66cf70f
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
index 0000000,0000000..45113dc
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/mc-1-big-TOC.txt
@@@ -1,0 -1,0 +1,8 @@@
++CompressionInfo.db
++Data.db
++Summary.db
++Filter.db
++Statistics.db
++TOC.txt
++Digest.crc32
++Index.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-CompressionInfo.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-CompressionInfo.db
index 0000000,0000000..1759c09
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Data.db
index 0000000,0000000..c1de572
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
index 0000000,0000000..0403b5b
new file mode 100644
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/mc-1-big-Digest.crc32
@@@ -1,0 -1,0 +1,1 @@@
++82785930

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Filter.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Filter.db
index 0000000,0000000..a397f35
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Index.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Index.db
index 0000000,0000000..a0477eb
new file mode 100644
Binary files differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/c231ed5b/test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
----------------------------------------------------------------------
diff --cc test/data/negative-local-expiration-test/table2/mc-1-big-Statistics.db
index 0000000,0000000..e9d6577
new file mode 100644
Binary files differ


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


[06/29] cassandra git commit: Protect against overflow of local expiration time

Posted by pa...@apache.org.
Protect against overflow of local expiration time

Patch by Paulo Motta; Reviewed by Sam Tunnicliffe for CASSANDRA-14092


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

Branch: refs/heads/trunk
Commit: b2949439ec62077128103540e42570238520f4ee
Parents: 5ba9e6d
Author: Paulo Motta <pa...@gmail.com>
Authored: Thu Feb 1 04:01:28 2018 -0200
Committer: Paulo Motta <pa...@apache.org>
Committed: Sat Feb 10 14:33:50 2018 -0200

----------------------------------------------------------------------
 CASSANDRA-14092.txt                             |  81 ++++
 CHANGES.txt                                     |   1 +
 NEWS.txt                                        |  23 +-
 .../cassandra/cql/AbstractModification.java     |   4 +
 .../org/apache/cassandra/cql/Attributes.java    |  19 +
 .../apache/cassandra/cql/BatchStatement.java    |   4 +
 .../org/apache/cassandra/cql/CFPropDefs.java    |   7 +
 .../org/apache/cassandra/cql3/Attributes.java   |  81 +++-
 .../cassandra/cql3/statements/CFPropDefs.java   |   7 +
 .../cql3/statements/ModificationStatement.java  |   2 +-
 .../apache/cassandra/db/AbstractNativeCell.java |   6 +
 .../org/apache/cassandra/db/BufferCell.java     |   6 +
 .../apache/cassandra/db/BufferDeletedCell.java  |   6 +
 .../apache/cassandra/db/BufferExpiringCell.java |  34 +-
 src/java/org/apache/cassandra/db/Cell.java      |   2 +
 .../apache/cassandra/db/ColumnFamilyStore.java  |   4 +-
 .../org/apache/cassandra/db/DeletionTime.java   |   1 +
 .../db/compaction/CompactionManager.java        |  15 +-
 .../cassandra/db/compaction/Scrubber.java       |  97 ++++-
 .../io/sstable/AbstractSSTableSimpleWriter.java |  17 +-
 .../cassandra/service/StorageService.java       |   8 +-
 .../cassandra/service/StorageServiceMBean.java  |  17 +-
 .../cassandra/thrift/ThriftValidation.java      |   9 +-
 .../org/apache/cassandra/tools/NodeProbe.java   |  12 +-
 .../org/apache/cassandra/tools/NodeTool.java    |  23 +-
 .../cassandra/tools/StandaloneScrubber.java     |  12 +-
 ...test_keyspace-table1-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table1-ka-1-Data.db       | Bin 0 -> 103 bytes
 .../cql_test_keyspace-table1-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table1-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table1-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table1-ka-1-Statistics.db | Bin 0 -> 4454 bytes
 .../cql_test_keyspace-table1-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table1-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table2-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table2-ka-1-Data.db       | Bin 0 -> 96 bytes
 .../cql_test_keyspace-table2-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table2-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table2-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table2-ka-1-Statistics.db | Bin 0 -> 4466 bytes
 .../cql_test_keyspace-table2-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table2-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table3-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table3-ka-1-Data.db       | Bin 0 -> 182 bytes
 .../cql_test_keyspace-table3-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table3-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table3-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table3-ka-1-Statistics.db | Bin 0 -> 4470 bytes
 .../cql_test_keyspace-table3-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table3-ka-1-TOC.txt       |   8 +
 ...test_keyspace-table4-ka-1-CompressionInfo.db | Bin 0 -> 43 bytes
 .../cql_test_keyspace-table4-ka-1-Data.db       | Bin 0 -> 181 bytes
 .../cql_test_keyspace-table4-ka-1-Digest.sha1   |   1 +
 .../cql_test_keyspace-table4-ka-1-Filter.db     | Bin 0 -> 16 bytes
 .../cql_test_keyspace-table4-ka-1-Index.db      | Bin 0 -> 36 bytes
 .../cql_test_keyspace-table4-ka-1-Statistics.db | Bin 0 -> 4482 bytes
 .../cql_test_keyspace-table4-ka-1-Summary.db    | Bin 0 -> 92 bytes
 .../cql_test_keyspace-table4-ka-1-TOC.txt       |   8 +
 .../cql3/validation/operations/TTLTest.java     | 410 +++++++++++++++++++
 59 files changed, 898 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CASSANDRA-14092.txt
----------------------------------------------------------------------
diff --git a/CASSANDRA-14092.txt b/CASSANDRA-14092.txt
new file mode 100644
index 0000000..5ac872c
--- /dev/null
+++ b/CASSANDRA-14092.txt
@@ -0,0 +1,81 @@
+CASSANDRA-14092: MAXIMUM TTL EXPIRATION DATE
+---------------------------------------------
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that INSERTS using TTL that would expire
+after this date are not currently supported.
+
+# Expiration Date Overflow Policy
+
+We plan to lift this limitation in newer versions, but while the fix is not available,
+operators can decide which policy to apply when dealing with inserts with TTL exceeding
+the maximum supported expiration date:
+  -     REJECT: this is the default policy and will reject any requests with expiration
+                date timestamp after 2038-01-19T03:14:06+00:00.
+  -        CAP: any insert with TTL expiring after 2038-01-19T03:14:06+00:00 will expire on
+                2038-01-19T03:14:06+00:00 and the client will receive a warning.
+  - CAP_NOWARN: same as previous, except that the client warning will not be emitted.
+
+These policies may be specified via the -Dcassandra.expiration_date_overflow_policy=POLICY
+startup option.
+
+# Potential data loss on earlier versions
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x), there was no protection against
+INSERTS with TTL expiring after the maximum supported date, causing the expiration
+time field to overflow and the records to expire immediately. Expired records due
+to overflow will not be queryable and will be permanently removed after a compaction.
+
+2.1.X, 2.2.X and earlier series are not subject to this bug when assertions are enabled
+since an AssertionError is thrown during INSERT when the expiration time field overflows
+on these versions. When assertions are disabled then it is possible to INSERT entries
+with overflowed local expiration time and even the earlier versions are subject to data
+loss due to this bug.
+
+This issue only affected INSERTs with very large TTLs, close to the maximum allowed value
+of 630720000 seconds (20 years), starting from 2018-01-19T03:14:06+00:00. As time progresses,
+the maximum supported TTL will be gradually reduced as the maximum expiration date approaches.
+For instance, a user on an affected version on 2028-01-19T03:14:06 with a TTL of 10 years
+will be affected by this bug, so we urge users of very large TTLs to upgrade to a version
+where this issue is addressed as soon as possible.
+
+# Data Recovery
+
+SSTables from Cassandra versions prior to 2.1.20/2.2.12/3.0.16/3.11.2 containing entries
+with overflowed expiration time that were backed up or did not go through compaction can
+be recovered by reinserting overflowed entries with a valid expiration time and a higher
+timestamp, since tombstones may have been generated with the original timestamp.
+
+To find out if an SSTable has an entry with overflowed expiration, inspect it with the
+sstable2json tool and look for a negative "local deletion time" field. SSTables in this
+condition should be backed up immediately, as they are subject to data loss during
+compaction.
+
+A "--reinsert-overflowed-ttl" option was added to scrub to rewrite SSTables containing
+rows with overflowed expiration time with the maximum expiration date of
+2038-01-19T03:14:06+00:00 and the original timestamp + 1 (ms). Two methods are offered
+for recovery of SSTables via scrub:
+
+- Offline scrub:
+   - Clone the data directory tree to another location, keeping only the folders and the
+     contents of the system tables.
+   - Clone the configuration directory to another location, setting the data_file_directories
+     property to the cloned data directory in the cloned cassandra.yaml.
+   - Copy the affected SSTables to the cloned data location of the affected table.
+   - Set the environment variable CASSANDRA_CONF=<cloned configuration directory>.
+   - Execute "sstablescrub --reinsert-overflowed-ttl <keyspace> <table>".
+         WARNING: not specifying --reinsert-overflowed-ttl is equivalent to a single-sstable
+         compaction, so the data with overflowed will be removed - make sure to back up
+         your SSTables before running scrub.
+   - Once the scrub is completed, copy the resulting SSTables to the original data directory.
+   - Execute "nodetool refresh" in a live node to load the recovered SSTables.
+
+- Online scrub:
+   - Disable compaction on the node with "nodetool disableautocompaction" - this step is crucial
+     as otherwise, the data may be removed permanently during compaction.
+   - Copy the SSTables containing entries with overflowed expiration time to the data directory.
+   - run "nodetool refresh" to load the SSTables.
+   - run "nodetool scrub --reinsert-overflowed-ttl <keyspace> <table>".
+   - Re-enable compactions after verifying that scrub recovered the missing entries.
+
+See https://issues.apache.org/jira/browse/CASSANDRA-14092 for more details about this issue.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e17093d..9332354 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 2.1.20
+ * Protect against overflow of local expiration time (CASSANDRA-14092)
  * More PEP8 compliance for cqlsh (CASSANDRA-14021)
  * RPM package spec: fix permissions for installed jars and config files (CASSANDRA-14181)
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index 796a424..fb6b4ee 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -1,3 +1,23 @@
+PLEASE READ: MAXIMUM TTL EXPIRATION DATE NOTICE (CASSANDRA-14092)
+------------------------------------------------------------------
+(General upgrading instructions are available in the next section)
+
+The maximum expiration timestamp that can be represented by the storage engine is
+2038-01-19T03:14:06+00:00, which means that inserts with TTL thatl expire after
+this date are not currently supported. By default, INSERTS with TTL exceeding the
+maximum supported date are rejected, but it's possible to choose a different
+ expiration overflow policy. See CASSANDRA-14092.txt for more details.
+
+Prior to 3.0.16 (3.0.X) and 3.11.2 (3.11.x) there was no protection against INSERTS
+with TTL expiring after the maximum supported date, causing the expiration time
+field to overflow and the records to expire immediately. Clusters in the 2.X and
+lower series are not subject to this when assertions are enabled. Backed up SSTables
+can be potentially recovered and recovery instructions can be found on the
+CASSANDRA-14092.txt file.
+
+If you use or plan to use very large TTLS (10 to 20 years), read CASSANDRA-14092.txt
+for more information.
+
 GENERAL UPGRADING ADVICE FOR ANY VERSION
 ========================================
 
@@ -18,8 +38,7 @@ using the provided 'sstableupgrade' tool.
 
 Upgrading
 ---------
-    - Nothing specific to this release, but please see 2.1 if you are upgrading
-      from a previous version.
+   - See MAXIMUM TTL EXPIRATION DATE NOTICE above.
 
 2.1.19
 ======

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/AbstractModification.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/AbstractModification.java b/src/java/org/apache/cassandra/cql/AbstractModification.java
index 8da2611..e98764b 100644
--- a/src/java/org/apache/cassandra/cql/AbstractModification.java
+++ b/src/java/org/apache/cassandra/cql/AbstractModification.java
@@ -20,11 +20,14 @@ package org.apache.cassandra.cql;
 import java.nio.ByteBuffer;
 import java.util.List;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.IMutation;
 import org.apache.cassandra.db.ConsistencyLevel;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
 import org.apache.cassandra.thrift.ThriftClientState;
+import org.w3c.dom.Attr;
 
 public abstract class AbstractModification
 {
@@ -89,6 +92,7 @@ public abstract class AbstractModification
 
     public int getTimeToLive()
     {
+        Attributes.maybeApplyExpirationDateOverflowPolicy(keyspace, columnFamily, timeToLive);
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/Attributes.java b/src/java/org/apache/cassandra/cql/Attributes.java
index faee3b8..c1c37ef 100644
--- a/src/java/org/apache/cassandra/cql/Attributes.java
+++ b/src/java/org/apache/cassandra/cql/Attributes.java
@@ -17,7 +17,10 @@
  */
 package org.apache.cassandra.cql;
 
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ConsistencyLevel;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 
 /**
  * Class to contain attributes for statements
@@ -73,4 +76,20 @@ public class Attributes
         return String.format("Attributes(consistency=%s, timestamp=%s, timeToLive=%s)", cLevel, timestamp, timeToLive);
     }
 
+    public static void maybeApplyExpirationDateOverflowPolicy(String keyspace, String columnFamily, Integer timeToLive)
+    {
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace, columnFamily);
+        if (metadata != null)
+        {
+            try
+            {
+                org.apache.cassandra.cql3.Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, timeToLive, false);
+            }
+            catch (InvalidRequestException e)
+            {
+                throw new RuntimeException(e);
+            }
+        }
+    }
+
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/BatchStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/BatchStatement.java b/src/java/org/apache/cassandra/cql/BatchStatement.java
index b141bcc..e5a95b8 100644
--- a/src/java/org/apache/cassandra/cql/BatchStatement.java
+++ b/src/java/org/apache/cassandra/cql/BatchStatement.java
@@ -72,6 +72,10 @@ public class BatchStatement
 
     public int getTimeToLive()
     {
+        for (AbstractModification statement : statements)
+        {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(statement.keyspace, statement.columnFamily, timeToLive);
+        }
         return timeToLive;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql/CFPropDefs.java b/src/java/org/apache/cassandra/cql/CFPropDefs.java
index f65cb94..a0c8d0d 100644
--- a/src/java/org/apache/cassandra/cql/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql/CFPropDefs.java
@@ -28,6 +28,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.db.marshal.AbstractType;
@@ -191,6 +192,12 @@ public class CFPropDefs {
                         KW_DEFAULT_TIME_TO_LIVE,
                         0,
                         CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE));
+
+            if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+                throw new InvalidRequestException(String.format("%s must be less than or equal to %d (got %s)",
+                                                                KW_DEFAULT_TIME_TO_LIVE,
+                                                                ExpiringCell.MAX_TTL,
+                                                                defaultTimeToLive));
         }
 
         CFMetaData.validateCompactionOptions(compactionStrategyClass, compactionStrategyOptions);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/Attributes.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Attributes.java b/src/java/org/apache/cassandra/cql3/Attributes.java
index 435757b..23571ca 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -18,13 +18,19 @@
 package org.apache.cassandra.cql3;
 
 import java.nio.ByteBuffer;
-import java.util.List;
+import java.util.concurrent.TimeUnit;
 
+import com.google.common.annotations.VisibleForTesting;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.marshal.Int32Type;
 import org.apache.cassandra.db.marshal.LongType;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.serializers.MarshalException;
+import org.apache.cassandra.utils.NoSpamLogger;
 
 /**
  * Utility class for the Parser to gather attributes for modification
@@ -32,6 +38,41 @@ import org.apache.cassandra.serializers.MarshalException;
  */
 public class Attributes
 {
+    private static final int EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES = Integer.getInteger("cassandra.expiration_overflow_warning_interval_minutes", 5);
+
+    private static final Logger logger = LoggerFactory.getLogger(Attributes.class);
+
+    public enum ExpirationDateOverflowPolicy
+    {
+        REJECT, CAP
+    }
+
+    @VisibleForTesting
+    public static ExpirationDateOverflowPolicy policy;
+
+    static {
+        String policyAsString = System.getProperty("cassandra.expiration_date_overflow_policy", ExpirationDateOverflowPolicy.REJECT.name());
+        try
+        {
+            policy = ExpirationDateOverflowPolicy.valueOf(policyAsString.toUpperCase());
+        }
+        catch (RuntimeException e)
+        {
+            logger.warn("Invalid expiration date overflow policy: {}. Using default: {}", policyAsString, ExpirationDateOverflowPolicy.REJECT.name());
+            policy = ExpirationDateOverflowPolicy.REJECT;
+        }
+    }
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING = "Request on table {}.{} with {}ttl of {} seconds exceeds maximum supported expiration " +
+                                                                          "date of 2038-01-19T03:14:06+00:00 and will have its expiration capped to that date. " +
+                                                                          "In order to avoid this use a lower TTL or upgrade to a version where this limitation " +
+                                                                          "is fixed. See CASSANDRA-14092 for more details.";
+
+    public static final String MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE = "Request on table %s.%s with %sttl of %d seconds exceeds maximum supported expiration " +
+                                                                                 "date of 2038-01-19T03:14:06+00:00. In order to avoid this use a lower TTL, change " +
+                                                                                 "the expiration date overflow policy or upgrade to a version where this limitation " +
+                                                                                 "is fixed. See CASSANDRA-14092 for more details.";
+
     private final Term timestamp;
     private final Term timeToLive;
 
@@ -77,10 +118,13 @@ public class Attributes
         return LongType.instance.compose(tval);
     }
 
-    public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+    public int getTimeToLive(QueryOptions options, CFMetaData metadata) throws InvalidRequestException
     {
         if (timeToLive == null)
-            return 0;
+        {
+            maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
+            return metadata.getDefaultTimeToLive();
+        }
 
         ByteBuffer tval = timeToLive.bindAndGet(options);
         if (tval == null)
@@ -102,6 +146,8 @@ public class Attributes
         if (ttl > ExpiringCell.MAX_TTL)
             throw new InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", ttl, ExpiringCell.MAX_TTL));
 
+        maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+
         return ttl;
     }
 
@@ -135,4 +181,33 @@ public class Attributes
             return new ColumnSpecification(ksName, cfName, new ColumnIdentifier("[ttl]", true), Int32Type.instance);
         }
     }
+
+    public static void maybeApplyExpirationDateOverflowPolicy(CFMetaData metadata, int ttl, boolean isDefaultTTL) throws InvalidRequestException
+    {
+        if (ttl == 0)
+            return;
+
+        // Check for localExpirationTime overflow (CASSANDRA-14092)
+        int nowInSecs = (int)(System.currentTimeMillis() / 1000);
+        if (ttl + nowInSecs < 0)
+        {
+            switch (policy)
+            {
+                case CAP:
+                    /**
+                     * Capping at this stage is basically not rejecting the request. The actual capping is done
+                     * by {@link org.apache.cassandra.db.BufferExpiringCell#computeLocalExpirationTime(int)},
+                     * which converts the negative TTL to {@link org.apache.cassandra.db.BufferExpiringCell#MAX_DELETION_TIME}
+                     */
+                    NoSpamLogger.log(logger, NoSpamLogger.Level.WARN, EXPIRATION_OVERFLOW_WARNING_INTERVAL_MINUTES,
+                                     TimeUnit.MINUTES, MAXIMUM_EXPIRATION_DATE_EXCEEDED_WARNING,
+                                     metadata.ksName, metadata.cfName, isDefaultTTL? "default " : "", ttl);
+                    return;
+
+                default: //REJECT
+                    throw new InvalidRequestException(String.format(MAXIMUM_EXPIRATION_DATE_EXCEEDED_REJECT_MESSAGE, metadata.ksName, metadata.cfName,
+                                                                    isDefaultTTL? "default " : "", ttl));
+            }
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
index 17edd6d..27dd57f 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CFPropDefs.java
@@ -22,6 +22,7 @@ import java.util.*;
 import org.apache.cassandra.cache.CachingOptions;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.CFMetaData.SpeculativeRetry;
+import org.apache.cassandra.db.ExpiringCell;
 import org.apache.cassandra.db.compaction.AbstractCompactionStrategy;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.SyntaxException;
@@ -127,6 +128,12 @@ public class CFPropDefs extends PropertyDefinitions
         }
 
         validateMinimumInt(KW_DEFAULT_TIME_TO_LIVE, 0, CFMetaData.DEFAULT_DEFAULT_TIME_TO_LIVE);
+        Integer defaultTimeToLive = getInt(KW_DEFAULT_TIME_TO_LIVE, 0);
+        if (defaultTimeToLive > ExpiringCell.MAX_TTL)
+            throw new ConfigurationException(String.format("%s must be less than or equal to %d (got %s)",
+                                                           KW_DEFAULT_TIME_TO_LIVE,
+                                                           ExpiringCell.MAX_TTL,
+                                                           defaultTimeToLive));
 
         Integer minIndexInterval = getInt(KW_MIN_INDEX_INTERVAL, null);
         Integer maxIndexInterval = getInt(KW_MAX_INDEX_INTERVAL, null);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index f84188a..8038c6c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -124,7 +124,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options);
+        return attrs.getTimeToLive(options, cfm);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/AbstractNativeCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractNativeCell.java b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
index e01d860..1b2c384 100644
--- a/src/java/org/apache/cassandra/db/AbstractNativeCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractNativeCell.java
@@ -575,6 +575,12 @@ public abstract class AbstractNativeCell extends AbstractCell implements CellNam
         throw new UnsupportedOperationException();
     }
 
+    @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
     protected long internalSize()
     {
         return MemoryUtil.getInt(peer);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCell.java b/src/java/org/apache/cassandra/db/BufferCell.java
index a7d632d..ee5fe41 100644
--- a/src/java/org/apache/cassandra/db/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCell.java
@@ -69,6 +69,12 @@ public class BufferCell extends AbstractCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public CellName name() {
         return name;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferDeletedCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferDeletedCell.java b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
index a38f322..3762e1f 100644
--- a/src/java/org/apache/cassandra/db/BufferDeletedCell.java
+++ b/src/java/org/apache/cassandra/db/BufferDeletedCell.java
@@ -54,6 +54,12 @@ public class BufferDeletedCell extends BufferCell implements DeletedCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
     public boolean isLive()
     {
         return false;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/BufferExpiringCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferExpiringCell.java b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
index 25172c8..ea40676 100644
--- a/src/java/org/apache/cassandra/db/BufferExpiringCell.java
+++ b/src/java/org/apache/cassandra/db/BufferExpiringCell.java
@@ -31,19 +31,23 @@ import org.apache.cassandra.utils.memory.MemtableAllocator;
 
 public class BufferExpiringCell extends BufferCell implements ExpiringCell
 {
+    public static final int MAX_DELETION_TIME = Integer.MAX_VALUE - 1;
+
     private final int localExpirationTime;
     private final int timeToLive;
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive)
     {
-        this(name, value, timestamp, timeToLive, (int) (System.currentTimeMillis() / 1000) + timeToLive);
+        super(name, value, timestamp);
+        assert timeToLive > 0 : timeToLive;
+        this.timeToLive = timeToLive;
+        this.localExpirationTime = computeLocalExpirationTime(timeToLive);
     }
 
     public BufferExpiringCell(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime)
     {
         super(name, value, timestamp);
         assert timeToLive > 0 : timeToLive;
-        assert localExpirationTime > 0 : localExpirationTime;
         this.timeToLive = timeToLive;
         this.localExpirationTime = localExpirationTime;
     }
@@ -66,6 +70,12 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     }
 
     @Override
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime)
+    {
+        return new BufferExpiringCell(name(), value(), newTimestamp, timeToLive, newLocalDeletionTime);
+    }
+
+    @Override
     public int cellDataSize()
     {
         return super.cellDataSize() + TypeSizes.NATIVE.sizeof(localExpirationTime) + TypeSizes.NATIVE.sizeof(timeToLive);
@@ -176,7 +186,9 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
     /** @return Either a DeletedCell, or an ExpiringCell. */
     public static Cell create(CellName name, ByteBuffer value, long timestamp, int timeToLive, int localExpirationTime, int expireBefore, ColumnSerializer.Flag flag)
     {
-        if (localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
+        // CASSANDRA-14092 may have written rows with negative localExpirationTime, so we don't turn them into tombstones yet
+        // to be able to recover them with scrub.
+        if (localExpirationTime < 0 || localExpirationTime >= expireBefore || flag == ColumnSerializer.Flag.PRESERVE_SIZE)
             return new BufferExpiringCell(name, value, timestamp, timeToLive, localExpirationTime);
         // The column is now expired, we can safely return a simple tombstone. Note that
         // as long as the expiring column and the tombstone put together live longer than GC grace seconds,
@@ -184,4 +196,20 @@ public class BufferExpiringCell extends BufferCell implements ExpiringCell
         // http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/repair-compaction-and-tombstone-rows-td7583481.html
         return new BufferDeletedCell(name, localExpirationTime - timeToLive, timestamp);
     }
+
+    /**
+     * This method computes the {@link #localExpirationTime}, maybe capping to the maximum representable value
+     * which is {@link #MAX_DELETION_TIME}.
+     *
+     * Please note that the {@link org.apache.cassandra.cql3.Attributes.ExpirationDateOverflowPolicy} is applied
+     * during {@link org.apache.cassandra.cql3.Attributes#maybeApplyExpirationDateOverflowPolicy(CFMetaData, int, boolean)},
+     * so if the request was not denied it means it's expiration date should be capped.
+     *
+     * See CASSANDRA-14092
+     */
+    private int computeLocalExpirationTime(int timeToLive)
+    {
+        int localExpirationTime =  (int) (System.currentTimeMillis() / 1000) + timeToLive;
+        return localExpirationTime >= 0? localExpirationTime : MAX_DELETION_TIME;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/Cell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Cell.java b/src/java/org/apache/cassandra/db/Cell.java
index 7c3926a..274f369 100644
--- a/src/java/org/apache/cassandra/db/Cell.java
+++ b/src/java/org/apache/cassandra/db/Cell.java
@@ -38,6 +38,8 @@ public interface Cell extends OnDiskAtom
 
     public Cell withUpdatedTimestamp(long newTimestamp);
 
+    public Cell withUpdatedTimestampAndLocalDeletionTime(long newTimestamp, int newLocalDeletionTime);
+
     @Override
     public CellName name();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index 6e82745..2989b9d 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -1516,12 +1516,12 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         return CompactionManager.instance.performCleanup(ColumnFamilyStore.this, jobs);
     }
 
-    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs) throws ExecutionException, InterruptedException
+    public CompactionManager.AllSSTableOpStatus scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs) throws ExecutionException, InterruptedException
     {
         // skip snapshot creation during scrub, SEE JIRA 5891
         if(!disableSnapshot)
             snapshotWithoutFlush("pre-scrub-" + System.currentTimeMillis());
-        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, jobs);
+        return CompactionManager.instance.performScrub(ColumnFamilyStore.this, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
     }
 
     public CompactionManager.AllSSTableOpStatus sstablesRewrite(boolean excludeCurrentVersion, int jobs) throws ExecutionException, InterruptedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/DeletionTime.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionTime.java b/src/java/org/apache/cassandra/db/DeletionTime.java
index 99cfe35..c10a15f 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -60,6 +60,7 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
     @VisibleForTesting
     public DeletionTime(long markedForDeleteAt, int localDeletionTime)
     {
+        assert localDeletionTime >= 0 : localDeletionTime;
         this.markedForDeleteAt = markedForDeleteAt;
         this.localDeletionTime = localDeletionTime;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 87819ba..6e3634a 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -358,8 +358,15 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
+    @Deprecated
     public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData, int jobs) throws InterruptedException, ExecutionException
     {
+        return performScrub(cfs, skipCorrupted, checkData, false, jobs);
+    }
+
+    public AllSSTableOpStatus performScrub(final ColumnFamilyStore cfs, final boolean skipCorrupted, final boolean checkData,
+                                           final boolean reinsertOverflowedTTLRows, int jobs) throws InterruptedException, ExecutionException
+    {
         assert !cfs.isIndex();
         return parallelAllSSTableOperation(cfs, new OneSSTableOperation()
         {
@@ -372,7 +379,7 @@ public class CompactionManager implements CompactionManagerMBean
             @Override
             public void execute(SSTableReader input) throws IOException
             {
-                scrubOne(cfs, input, skipCorrupted, checkData);
+                scrubOne(cfs, input, skipCorrupted, checkData, reinsertOverflowedTTLRows);
             }
         }, jobs);
     }
@@ -710,9 +717,9 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData) throws IOException
+    private void scrubOne(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows) throws IOException
     {
-        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData);
+        Scrubber scrubber = new Scrubber(cfs, sstable, skipCorrupted, false, checkData, reinsertOverflowedTTLRows);
 
         CompactionInfo.Holder scrubInfo = scrubber.getScrubInfo();
         metrics.beginCompaction(scrubInfo);
@@ -1352,7 +1359,7 @@ public class CompactionManager implements CompactionManagerMBean
         public void afterExecute(Runnable r, Throwable t)
         {
             DebuggableThreadPoolExecutor.maybeResetTraceSessionWrapper(r);
-    
+
             if (t == null)
                 t = DebuggableThreadPoolExecutor.extractThrowable(r);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/db/compaction/Scrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/Scrubber.java b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
index 2df3665..6d4537c 100644
--- a/src/java/org/apache/cassandra/db/compaction/Scrubber.java
+++ b/src/java/org/apache/cassandra/db/compaction/Scrubber.java
@@ -28,6 +28,7 @@ import com.google.common.collect.Sets;
 
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.columniterator.OnDiskAtomIterator;
+import org.apache.cassandra.db.composites.CellNames;
 import org.apache.cassandra.io.sstable.*;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.io.util.RandomAccessReader;
@@ -35,6 +36,7 @@ import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.OutputHandler;
+import org.apache.cassandra.utils.memory.HeapAllocator;
 
 public class Scrubber implements Closeable
 {
@@ -43,6 +45,7 @@ public class Scrubber implements Closeable
     private final File destination;
     private final boolean skipCorrupted;
     public final boolean validateColumns;
+    private final boolean reinsertOverflowedTTLRows;
 
     private final CompactionController controller;
     private final boolean isCommutative;
@@ -67,6 +70,7 @@ public class Scrubber implements Closeable
     long nextRowPositionFromIndex;
 
     private final OutputHandler outputHandler;
+    private NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics = new NegativeLocalDeletionInfoMetrics();
 
     private static final Comparator<Row> rowComparator = new Comparator<Row>()
     {
@@ -79,10 +83,17 @@ public class Scrubber implements Closeable
 
     public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData) throws IOException
     {
-        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData);
+        this(cfs, sstable, skipCorrupted, isOffline, checkData, false);
     }
 
-    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData) throws IOException
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
+    {
+        this(cfs, sstable, skipCorrupted, new OutputHandler.LogOutput(), isOffline, checkData, reinsertOverflowedTTLRows);
+    }
+
+    public Scrubber(ColumnFamilyStore cfs, SSTableReader sstable, boolean skipCorrupted, OutputHandler outputHandler, boolean isOffline, boolean checkData,
+                    boolean reinsertOverflowedTTLRows) throws IOException
     {
         this.cfs = cfs;
         this.sstable = sstable;
@@ -90,6 +101,7 @@ public class Scrubber implements Closeable
         this.skipCorrupted = skipCorrupted;
         this.isOffline = isOffline;
         this.validateColumns = checkData;
+        this.reinsertOverflowedTTLRows = reinsertOverflowedTTLRows;
 
         List<SSTableReader> toScrub = Collections.singletonList(sstable);
 
@@ -131,6 +143,9 @@ public class Scrubber implements Closeable
 
         this.currentRowPositionFromIndex = 0;
         this.nextRowPositionFromIndex = 0;
+
+        if (reinsertOverflowedTTLRows)
+            outputHandler.output("Starting scrub with reinsert overflowed TTL option");
     }
 
     public void scrub()
@@ -311,6 +326,8 @@ public class Scrubber implements Closeable
         else
         {
             outputHandler.output("Scrub of " + sstable + " complete: " + goodRows + " rows in new sstable and " + emptyRows + " empty (tombstoned) rows dropped");
+            if (negativeLocalDeletionInfoMetrics.fixedRows > 0)
+                outputHandler.output("Fixed " + negativeLocalDeletionInfoMetrics.fixedRows + " rows with overflowed local deletion time.");
             if (badRows > 0)
                 outputHandler.warn("Unable to recover " + badRows + " rows that were skipped.  You can attempt manual recovery from the pre-scrub snapshot.  You can also run nodetool repair to transfer the data from a healthy replica, if any");
         }
@@ -322,7 +339,7 @@ public class Scrubber implements Closeable
         // OrderCheckerIterator will check, at iteration time, that the cells are in the proper order. If it detects
         // that one cell is out of order, it will stop returning them. The remaining cells will be sorted and added
         // to the outOfOrderRows that will be later written to a new SSTable.
-        OrderCheckerIterator atoms = new OrderCheckerIterator(new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns),
+        OrderCheckerIterator atoms = new OrderCheckerIterator(getIterator(key, dataSize),
                                                               cfs.metadata.comparator.onDiskAtomComparator());
         if (prevKey != null && prevKey.compareTo(key) > 0)
         {
@@ -342,6 +359,18 @@ public class Scrubber implements Closeable
         return true;
     }
 
+    /**
+     * Only wrap with {@link FixNegativeLocalDeletionTimeIterator} if {@link #reinsertOverflowedTTLRows} option
+     * is specified
+     */
+    private OnDiskAtomIterator getIterator(DecoratedKey key, long dataSize)
+    {
+        SSTableIdentityIterator sstableIdentityIterator = new SSTableIdentityIterator(sstable, dataFile, key, dataSize, validateColumns);
+        return reinsertOverflowedTTLRows ? new FixNegativeLocalDeletionTimeIterator(sstableIdentityIterator,
+                                                                                    outputHandler,
+                                                                                    negativeLocalDeletionInfoMetrics) : sstableIdentityIterator;
+    }
+
     private void updateIndexKey()
     {
         currentIndexKey = nextIndexKey;
@@ -516,6 +545,11 @@ public class Scrubber implements Closeable
         }
     }
 
+    public class NegativeLocalDeletionInfoMetrics
+    {
+        public volatile int fixedRows = 0;
+    }
+
     /**
      * In some case like CASSANDRA-12127 the cells might have been stored in the wrong order. This decorator check the
      * cells order and collect the out of order cells to correct the problem.
@@ -601,4 +635,61 @@ public class Scrubber implements Closeable
             return cf;
         }
     }
+
+    /**
+     * This iterator converts negative {@link BufferExpiringCell#getLocalDeletionTime()} into {@link BufferExpiringCell#MAX_DELETION_TIME}
+     *
+     * This is to recover entries with overflowed localExpirationTime due to CASSANDRA-14092
+     */
+    private static final class FixNegativeLocalDeletionTimeIterator extends AbstractIterator<OnDiskAtom> implements OnDiskAtomIterator
+    {
+        /**
+         * The decorated iterator.
+         */
+        private final OnDiskAtomIterator iterator;
+
+        private final OutputHandler outputHandler;
+        private final NegativeLocalDeletionInfoMetrics negativeLocalExpirationTimeMetrics;
+
+        public FixNegativeLocalDeletionTimeIterator(OnDiskAtomIterator iterator, OutputHandler outputHandler,
+                                                    NegativeLocalDeletionInfoMetrics negativeLocalDeletionInfoMetrics)
+        {
+            this.iterator = iterator;
+            this.outputHandler = outputHandler;
+            this.negativeLocalExpirationTimeMetrics = negativeLocalDeletionInfoMetrics;
+        }
+
+        public ColumnFamily getColumnFamily()
+        {
+            return iterator.getColumnFamily();
+        }
+
+        public DecoratedKey getKey()
+        {
+            return iterator.getKey();
+        }
+
+        public void close() throws IOException
+        {
+            iterator.close();
+        }
+
+        @Override
+        protected OnDiskAtom computeNext()
+        {
+            if (!iterator.hasNext())
+                return endOfData();
+
+            OnDiskAtom next = iterator.next();
+
+            if (next instanceof ExpiringCell && next.getLocalDeletionTime() < 0)
+            {
+                outputHandler.debug(String.format("Found cell with negative local expiration time: %s", ((ExpiringCell) next).getString(getColumnFamily().getComparator()), getColumnFamily()));
+                negativeLocalExpirationTimeMetrics.fixedRows++;
+                next = ((Cell) next).localCopy(getColumnFamily().metadata(), HeapAllocator.instance).withUpdatedTimestampAndLocalDeletionTime(next.timestamp() + 1, BufferExpiringCell.MAX_DELETION_TIME);
+            }
+
+            return next;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
index 557c3de..d718765 100644
--- a/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
+++ b/src/java/org/apache/cassandra/io/sstable/AbstractSSTableSimpleWriter.java
@@ -28,9 +28,11 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.dht.IPartitioner;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.io.sstable.metadata.MetadataCollector;
 import org.apache.cassandra.service.ActiveRepairService;
 import org.apache.cassandra.utils.CounterId;
@@ -155,7 +157,20 @@ public abstract class AbstractSSTableSimpleWriter implements Closeable
      */
     public void addExpiringColumn(ByteBuffer name, ByteBuffer value, long timestamp, int ttl, long expirationTimestampMS) throws IOException
     {
-        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, (int)(expirationTimestampMS / 1000)));
+        int localExpirationTime = (int) (expirationTimestampMS / 1000);
+        try
+        {
+            // This will throw exception if policy is REJECT and now() + ttl is higher than MAX_DELETION_TIME
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, ttl, false);
+            // If exception was not thrown, this means the policy was CAP, so we check for overflow and cap if that's the case
+            if (localExpirationTime < 0)
+                localExpirationTime = BufferExpiringCell.MAX_DELETION_TIME;
+        }
+        catch (InvalidRequestException e)
+        {
+            throw new RuntimeException(e);
+        }
+        addColumn(new BufferExpiringCell(metadata.comparator.cellFromByteBuffer(name), value, timestamp, ttl, localExpirationTime));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index 27939f9..a7a8ca7 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -2415,10 +2415,16 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
 
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
+        return scrub(disableSnapshot, skipCorrupted, checkData, false, jobs, keyspaceName, columnFamilies);
+    }
+
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows,
+                     int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    {
         CompactionManager.AllSSTableOpStatus status = CompactionManager.AllSSTableOpStatus.SUCCESSFUL;
         for (ColumnFamilyStore cfStore : getValidColumnFamilies(false, false, keyspaceName, columnFamilies))
         {
-            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, jobs);
+            CompactionManager.AllSSTableOpStatus oneStatus = cfStore.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs);
             if (oneStatus != CompactionManager.AllSSTableOpStatus.SUCCESSFUL)
                 status = oneStatus;
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/service/StorageServiceMBean.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageServiceMBean.java b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
index d3a1725..90c0fb5 100644
--- a/src/java/org/apache/cassandra/service/StorageServiceMBean.java
+++ b/src/java/org/apache/cassandra/service/StorageServiceMBean.java
@@ -219,16 +219,16 @@ public interface StorageServiceMBean extends NotificationEmitter
 
     /**
      * Takes the snapshot of a multiple column family from different keyspaces. A snapshot name must be specified.
-     * 
+     *
      * @param tag
      *            the tag given to the snapshot; may not be null or empty
      * @param columnFamilyList
      *            list of columnfamily from different keyspace in the form of ks1.cf1 ks2.cf2
      */
     public void takeMultipleColumnFamilySnapshot(String tag, String... columnFamilyList) throws IOException;
-    
-    
-    
+
+
+
     /**
      * Remove the snapshot with the given name from the given keyspaces.
      * If no tag is specified we will remove all snapshots.
@@ -274,8 +274,11 @@ public interface StorageServiceMBean extends NotificationEmitter
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
     @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+    @Deprecated
     public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
 
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException;
+
     /**
      * Rewrite all sstables to the latest version.
      * Unlike scrub, it doesn't skip bad rows and do not snapshot sstables first.
@@ -381,11 +384,11 @@ public interface StorageServiceMBean extends NotificationEmitter
      * If level cannot be parsed, then the level will be defaulted to DEBUG<br>
      * <br>
      * The logback configuration should have < jmxConfigurator /> set
-     * 
+     *
      * @param classQualifier The logger's classQualifer
      * @param level The log level
-     * @throws Exception 
-     * 
+     * @throws Exception
+     *
      *  @see ch.qos.logback.classic.Level#toLevel(String)
      */
     public void setLoggingLevel(String classQualifier, String level) throws Exception;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/thrift/ThriftValidation.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index d5d9f73..10e7185 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -24,6 +24,7 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.config.*;
+import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.*;
 import org.apache.cassandra.db.composites.*;
@@ -314,7 +315,7 @@ public class ThriftValidation
             if (isCommutative)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("invalid operation for commutative columnfamily " + metadata.cfName);
 
-            validateTtl(cosc.column);
+            validateTtl(metadata, cosc.column);
             validateColumnPath(metadata, new ColumnPath(metadata.cfName).setSuper_column((ByteBuffer)null).setColumn(cosc.column.name));
             validateColumnData(metadata, key, null, cosc.column);
         }
@@ -349,7 +350,7 @@ public class ThriftValidation
         }
     }
 
-    private static void validateTtl(Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
+    private static void validateTtl(CFMetaData metadata, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
         if (column.isSetTtl())
         {
@@ -358,9 +359,11 @@ public class ThriftValidation
 
             if (column.ttl > ExpiringCell.MAX_TTL)
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("ttl is too large. requested (%d) maximum (%d)", column.ttl, ExpiringCell.MAX_TTL));
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, column.ttl, false);
         }
         else
         {
+            Attributes.maybeApplyExpirationDateOverflowPolicy(metadata, metadata.getDefaultTimeToLive(), true);
             // if it's not set, then it should be zero -- here we are just checking to make sure Thrift doesn't change that contract with us.
             assert column.ttl == 0;
         }
@@ -434,7 +437,7 @@ public class ThriftValidation
      */
     public static void validateColumnData(CFMetaData metadata, ByteBuffer key, ByteBuffer scName, Column column) throws org.apache.cassandra.exceptions.InvalidRequestException
     {
-        validateTtl(column);
+        validateTtl(metadata, column);
         if (!column.isSetValue())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Column value is required");
         if (!column.isSetTimestamp())

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeProbe.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeProbe.java b/src/java/org/apache/cassandra/tools/NodeProbe.java
index 13c7acf..fcd4110 100644
--- a/src/java/org/apache/cassandra/tools/NodeProbe.java
+++ b/src/java/org/apache/cassandra/tools/NodeProbe.java
@@ -243,9 +243,9 @@ public class NodeProbe implements AutoCloseable
         return ssProxy.forceKeyspaceCleanup(jobs, keyspaceName, columnFamilies);
     }
 
-    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public int scrub(boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
-        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies);
+        return ssProxy.scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies);
     }
 
     public int upgradeSSTables(String keyspaceName, boolean excludeCurrentVersion, int jobs, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
@@ -268,10 +268,10 @@ public class NodeProbe implements AutoCloseable
         }
     }
 
-    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
+    public void scrub(PrintStream out, boolean disableSnapshot, boolean skipCorrupted, boolean checkData, boolean reinsertOverflowedTTLRows, int jobs, String keyspaceName, String... columnFamilies) throws IOException, ExecutionException, InterruptedException
     {
         checkJobs(out, jobs);
-        if (scrub(disableSnapshot, skipCorrupted, checkData, jobs, keyspaceName, columnFamilies) != 0)
+        if (scrub(disableSnapshot, skipCorrupted, checkData, reinsertOverflowedTTLRows, jobs, keyspaceName, columnFamilies) != 0)
         {
             failed = true;
             out.println("Aborted scrubbing atleast one column family in keyspace "+keyspaceName+", check server logs for more information.");
@@ -562,7 +562,7 @@ public class NodeProbe implements AutoCloseable
 
     /**
      * Take a snapshot of all column family from different keyspaces.
-     * 
+     *
      * @param snapshotName
      *            the name of the snapshot.
      * @param columnfamilylist
@@ -1302,7 +1302,7 @@ public class NodeProbe implements AutoCloseable
         }
         catch (Exception e)
         {
-          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e); 
+          throw new RuntimeException("Error setting log for " + classQualifier +" on level " + level +". Please check logback configuration and ensure to have <jmxConfigurator /> set", e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/NodeTool.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/NodeTool.java b/src/java/org/apache/cassandra/tools/NodeTool.java
index d1afb6f..54d7fb7 100644
--- a/src/java/org/apache/cassandra/tools/NodeTool.java
+++ b/src/java/org/apache/cassandra/tools/NodeTool.java
@@ -551,20 +551,20 @@ public class NodeTool
             try
             {
                 ownerships = probe.effectiveOwnership(keyspace);
-            } 
+            }
             catch (IllegalStateException ex)
             {
                 ownerships = probe.getOwnership();
                 errors.append("Note: " + ex.getMessage() + "%n");
                 showEffectiveOwnership = false;
-            } 
+            }
             catch (IllegalArgumentException ex)
             {
                 System.out.printf("%nError: " + ex.getMessage() + "%n");
                 return;
             }
 
-            
+
             System.out.println();
             for (Entry<String, SetHostStat> entry : getOwnershipByDc(probe, resolveIp, tokensToEndpoints, ownerships).entrySet())
                 printDc(probe, format, entry.getKey(), endpointsToTokens, entry.getValue(),showEffectiveOwnership);
@@ -1282,6 +1282,11 @@ public class NodeTool
                 description = "Number of sstables to scrub simultanously, set to 0 to use all available compaction threads")
         private int jobs = 2;
 
+        @Option(title = "reinsert_overflowed_ttl",
+        name = {"r", "--reinsert-overflowed-ttl"},
+        description = StandaloneScrubber.REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION)
+        private boolean reinsertOverflowedTTL = false;
+
         @Override
         public void execute(NodeProbe probe)
         {
@@ -1292,7 +1297,7 @@ public class NodeTool
             {
                 try
                 {
-                    probe.scrub(System.out, disableSnapshot, skipCorrupted, !noValidation, jobs, keyspace, cfnames);
+                    probe.scrub(System.out, disableSnapshot, skipCorrupted, reinsertOverflowedTTL   , !noValidation, jobs, keyspace, cfnames);
                 } catch (Exception e)
                 {
                     throw new RuntimeException("Error occurred during flushing", e);
@@ -2197,7 +2202,7 @@ public class NodeTool
             unreachableNodes = probe.getUnreachableNodes();
             hostIDMap = probe.getHostIdMap();
             epSnitchInfo = probe.getEndpointSnitchInfoProxy();
-            
+
             StringBuffer errors = new StringBuffer();
 
             Map<InetAddress, Float> ownerships = null;
@@ -2249,9 +2254,9 @@ public class NodeTool
                     printNode(endpoint.getHostAddress(), owns, tokens, hasEffectiveOwns, isTokenPerNode);
                 }
             }
-            
+
             System.out.printf("%n" + errors.toString());
-            
+
         }
 
         private void findMaxAddressLength(Map<String, SetHostStat> dcs)
@@ -2726,7 +2731,7 @@ public class NodeTool
                 probe.truncateHints(endpoint);
         }
     }
-    
+
     @Command(name = "setlogginglevel", description = "Set the log level threshold for a given class. If both class and level are empty/null, it will reset to the initial configuration")
     public static class SetLoggingLevel extends NodeToolCmd
     {
@@ -2741,7 +2746,7 @@ public class NodeTool
             probe.setLoggingLevel(classQualifier, level);
         }
     }
-    
+
     @Command(name = "getlogginglevels", description = "Get the runtime logging levels")
     public static class GetLoggingLevels extends NodeToolCmd
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
index fdf6c8d..59d13d5 100644
--- a/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
+++ b/src/java/org/apache/cassandra/tools/StandaloneScrubber.java
@@ -47,6 +47,12 @@ import static org.apache.cassandra.tools.BulkLoader.CmdLineOptions;
 
 public class StandaloneScrubber
 {
+    public static final String REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION = "Rewrites rows with overflowed expiration date affected by CASSANDRA-14092 with " +
+                                                                            "the maximum supported expiration date of 2038-01-19T03:14:06+00:00. " +
+                                                                            "The rows are rewritten with the original timestamp incremented by one millisecond " +
+                                                                            "to override/supersede any potential tombstone that may have been generated " +
+                                                                            "during compaction of the affected rows.";
+
     private static final String TOOL_NAME = "sstablescrub";
     private static final String VERBOSE_OPTION  = "verbose";
     private static final String DEBUG_OPTION  = "debug";
@@ -54,6 +60,7 @@ public class StandaloneScrubber
     private static final String MANIFEST_CHECK_OPTION  = "manifest-check";
     private static final String SKIP_CORRUPTED_OPTION = "skip-corrupted";
     private static final String NO_VALIDATE_OPTION = "no-validate";
+    private static final String REINSERT_OVERFLOWED_TTL_OPTION = "reinsert-overflowed-ttl";
 
     public static void main(String args[])
     {
@@ -110,7 +117,7 @@ public class StandaloneScrubber
                 {
                     try
                     {
-                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate);
+                        Scrubber scrubber = new Scrubber(cfs, sstable, options.skipCorrupted, handler, true, !options.noValidate, options.reinsertOverflowedTTL);
                         try
                         {
                             scrubber.scrub();
@@ -192,6 +199,7 @@ public class StandaloneScrubber
         public boolean manifestCheckOnly;
         public boolean skipCorrupted;
         public boolean noValidate;
+        public boolean reinsertOverflowedTTL;
 
         private Options(String keyspaceName, String cfName)
         {
@@ -232,6 +240,7 @@ public class StandaloneScrubber
                 opts.manifestCheckOnly = cmd.hasOption(MANIFEST_CHECK_OPTION);
                 opts.skipCorrupted = cmd.hasOption(SKIP_CORRUPTED_OPTION);
                 opts.noValidate = cmd.hasOption(NO_VALIDATE_OPTION);
+                opts.reinsertOverflowedTTL = cmd.hasOption(REINSERT_OVERFLOWED_TTL_OPTION);
 
                 return opts;
             }
@@ -258,6 +267,7 @@ public class StandaloneScrubber
             options.addOption("m",  MANIFEST_CHECK_OPTION, "only check and repair the leveled manifest, without actually scrubbing the sstables");
             options.addOption("s",  SKIP_CORRUPTED_OPTION, "skip corrupt rows in counter tables");
             options.addOption("n",  NO_VALIDATE_OPTION,    "do not validate columns using column validator");
+            options.addOption("r", REINSERT_OVERFLOWED_TTL_OPTION, REINSERT_OVERFLOWED_TTL_OPTION_DESCRIPTION);
             return options;
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..d7cc13b
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db
new file mode 100644
index 0000000..0e3da66
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
new file mode 100644
index 0000000..8a6dcba
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Digest.sha1
@@ -0,0 +1 @@
+4012184764
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db
new file mode 100644
index 0000000..3ab96ee
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db
new file mode 100644
index 0000000..9bde77e
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table1/cql_test_keyspace-table1-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..38373b4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db
new file mode 100644
index 0000000..bdd4549
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
new file mode 100644
index 0000000..f58914a
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Digest.sha1
@@ -0,0 +1 @@
+3463582096
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db
new file mode 100644
index 0000000..38a6e4c
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db
new file mode 100644
index 0000000..8ee9116
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table2/cql_test_keyspace-table2-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..04a7384
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db
new file mode 100644
index 0000000..1fc8ba4
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
new file mode 100644
index 0000000..cd091ad
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1524836732
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db
new file mode 100644
index 0000000..5fb34e8
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db
new file mode 100644
index 0000000..4d961fb
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table3/cql_test_keyspace-table3-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db
new file mode 100644
index 0000000..c814fef
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-CompressionInfo.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db
new file mode 100644
index 0000000..92032a7
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Data.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1 b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
new file mode 100644
index 0000000..a45d821
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Digest.sha1
@@ -0,0 +1 @@
+2189764235
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Filter.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db
new file mode 100644
index 0000000..8291383
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Index.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db
new file mode 100644
index 0000000..68f76ae
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Statistics.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db
new file mode 100644
index 0000000..788b66a
Binary files /dev/null and b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-Summary.db differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/b2949439/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
----------------------------------------------------------------------
diff --git a/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
new file mode 100644
index 0000000..4b6cff8
--- /dev/null
+++ b/test/data/negative-local-expiration-test/table4/cql_test_keyspace-table4-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Index.db
+Data.db
+Statistics.db
+TOC.txt
+Summary.db
+Digest.sha1
+CompressionInfo.db
+Filter.db


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