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:55 UTC

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

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