You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2016/03/07 15:01:55 UTC

cassandra git commit: Allow to remove TTLs on table with default_time_to_live

Repository: cassandra
Updated Branches:
  refs/heads/trunk 7d9989145 -> e017f9494


Allow to remove TTLs on table with default_time_to_live

patch by Benjamin Lerer; reviewed by Tyler Hobbs for CASSANDRA-11207


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

Branch: refs/heads/trunk
Commit: e017f9494844234fa73848890347f59c622cea40
Parents: 7d99891
Author: Benjamin Lerer <b....@gmail.com>
Authored: Mon Mar 7 15:01:17 2016 +0100
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Mon Mar 7 15:01:17 2016 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 ++
 NEWS.txt                                        | 10 ++-
 doc/cql3/CQL.textile                            |  8 ++-
 .../org/apache/cassandra/cql3/Attributes.java   | 12 ++--
 .../apache/cassandra/cql3/UpdateParameters.java |  6 +-
 .../cql3/statements/ModificationStatement.java  |  2 +-
 .../org/apache/cassandra/db/LegacyLayout.java   |  2 +-
 .../org/apache/cassandra/db/LivenessInfo.java   | 14 ++--
 .../apache/cassandra/db/RowUpdateBuilder.java   |  4 +-
 .../apache/cassandra/db/rows/BufferCell.java    | 11 +--
 .../cassandra/db/rows/UnfilteredSerializer.java |  2 +-
 src/java/org/apache/cassandra/db/view/View.java |  3 +-
 .../index/internal/CassandraIndex.java          |  4 +-
 .../cassandra/thrift/CassandraServer.java       | 21 ++++--
 .../cassandra/thrift/ThriftValidation.java      |  4 +-
 .../cql3/validation/operations/InsertTest.java  | 31 +++++++++
 .../cql3/validation/operations/UpdateTest.java  | 31 +++++++++
 test/unit/org/apache/cassandra/db/CellTest.java | 12 ++--
 .../apache/cassandra/db/CounterCellTest.java    |  8 +--
 test/unit/org/apache/cassandra/db/RowTest.java  |  2 +-
 .../partition/PartitionImplementationTest.java  |  4 +-
 .../rows/DigestBackwardCompatibilityTest.java   |  4 +-
 .../rows/RowAndDeletionMergeIteratorTest.java   |  6 +-
 .../org/apache/cassandra/db/rows/RowsTest.java  | 70 ++++++++++----------
 .../rows/UnfilteredRowIteratorsMergeTest.java   |  2 +-
 .../index/internal/CustomCassandraIndex.java    |  4 +-
 .../cassandra/index/sasi/SASIIndexTest.java     |  4 +-
 .../sasi/disk/PerSSTableIndexWriterTest.java    |  2 +-
 .../index/sasi/plan/OperationTest.java          |  2 +-
 .../format/SSTableFlushObserverTest.java        | 18 ++---
 .../cassandra/service/DataResolverTest.java     |  2 +-
 .../service/pager/PagingStateTest.java          |  2 +-
 .../cassandra/triggers/TriggerExecutorTest.java | 16 +++--
 33 files changed, 205 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 9699bfd..015c952 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,3 +1,7 @@
+3.6
+ * Allow to remove TTLs on table with default_time_to_live (CASSANDRA-11207)
+
+
 3.5
 Merged from 3.0:
  * Fix filtering on non-primary key columns for queries without index (CASSANDRA-6377)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/NEWS.txt
----------------------------------------------------------------------
diff --git a/NEWS.txt b/NEWS.txt
index d554ac2..74490a8 100644
--- a/NEWS.txt
+++ b/NEWS.txt
@@ -13,8 +13,16 @@ restore snapshots created with the previous major version using the
 'sstableloader' tool. You can upgrade the file format of your snapshots
 using the provided 'sstableupgrade' tool.
 
+3.6
+=====
+
+New features
+------------
+   - for tables having a default_time_to_live specifying a TTL of 0 will remove the TTL
+     from the inserted or updated values.
+
 3.4
-===
+=====
 
 New features
 ------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/doc/cql3/CQL.textile
----------------------------------------------------------------------
diff --git a/doc/cql3/CQL.textile b/doc/cql3/CQL.textile
index 494a218..c942ec6 100644
--- a/doc/cql3/CQL.textile
+++ b/doc/cql3/CQL.textile
@@ -1,6 +1,6 @@
 <link rel="StyleSheet" href="CQL.css" type="text/css" media="screen">
 
-h1. Cassandra Query Language (CQL) v3.4.1
+h1. Cassandra Query Language (CQL) v3.4.2
 
 
 
@@ -911,7 +911,7 @@ h4(#updateOptions). @<options>@
 
 The @UPDATE@ and @INSERT@ statements support the following options:
 * @TIMESTAMP@: sets the timestamp for the operation. If not specified, the coordinator will use the current time (in microseconds) at the start of statement execution as the timestamp. This is usually a suitable default.
-* @TTL@: specifies an optional Time To Live (in seconds) for the inserted values. If set, the inserted values are automatically removed from the database after the specified time. Note that the TTL concerns the inserted values, not the columns themselves. This means that any subsequent update of the column will also reset the TTL (to whatever TTL is specified in that update). By default, values never expire. A TTL of 0 or a negative value is equivalent to no TTL.
+* @TTL@: specifies an optional Time To Live (in seconds) for the inserted values. If set, the inserted values are automatically removed from the database after the specified time. Note that the TTL concerns the inserted values, not the columns themselves. This means that any subsequent update of the column will also reset the TTL (to whatever TTL is specified in that update). By default, values never expire. A TTL of 0 is equivalent to no TTL. If the table has a default_time_to_live, a TTL of 0 will remove the TTL for the inserted or updated values.
 
 
 h3(#deleteStmt). DELETE
@@ -2301,6 +2301,10 @@ h2(#changes). Changes
 
 The following describes the changes in each version of CQL.
 
+h3. 3.4.2
+
+* "@INSERT/UPDATE options@":#updateOptions for tables having a default_time_to_live specifying a TTL of 0 will remove the TTL from the inserted or updated values
+
 h3. 3.4.1
 
 * Adds @CAST@ functions. See "@Cast@":#castFun.

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/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 97bdcd1..7cdb478 100644
--- a/src/java/org/apache/cassandra/cql3/Attributes.java
+++ b/src/java/org/apache/cassandra/cql3/Attributes.java
@@ -23,6 +23,7 @@ import java.util.Collections;
 import com.google.common.collect.Iterables;
 
 import org.apache.cassandra.cql3.functions.Function;
+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;
@@ -97,17 +98,17 @@ public class Attributes
         return LongType.instance.compose(tval);
     }
 
-    public int getTimeToLive(QueryOptions options) throws InvalidRequestException
+    public int getTimeToLive(QueryOptions options, int defaultTimeToLive) throws InvalidRequestException
     {
         if (timeToLive == null)
-            return 0;
+            return defaultTimeToLive;
 
         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;
 
         try
         {
@@ -125,6 +126,9 @@ public class Attributes
         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)
+            return LivenessInfo.NO_TTL;
+
         return ttl;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/cql3/UpdateParameters.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/UpdateParameters.java b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
index 572365b..6b0ed59 100644
--- a/src/java/org/apache/cassandra/cql3/UpdateParameters.java
+++ b/src/java/org/apache/cassandra/cql3/UpdateParameters.java
@@ -116,7 +116,7 @@ public class UpdateParameters
 
     public void addPrimaryKeyLivenessInfo()
     {
-        builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(metadata, timestamp, ttl, nowInSec));
+        builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(timestamp, ttl, nowInSec));
     }
 
     public void addRowDeletion()
@@ -149,7 +149,7 @@ public class UpdateParameters
     public void addCell(ColumnDefinition column, CellPath path, ByteBuffer value) throws InvalidRequestException
     {
         Cell cell = ttl == LivenessInfo.NO_TTL
-                  ? BufferCell.live(metadata, column, timestamp, value, path)
+                  ? BufferCell.live(column, timestamp, value, path)
                   : BufferCell.expiring(column, timestamp, ttl, nowInSec, value, path);
         builder.addCell(cell);
     }
@@ -166,7 +166,7 @@ public class UpdateParameters
         // but that makes things a bit more complex as this means we need to be able to distinguish inside
         // PartitionUpdate between counter updates that has been processed by CounterMutation and those that
         // haven't.
-        builder.addCell(BufferCell.live(metadata, column, timestamp, CounterContext.instance().createLocal(increment)));
+        builder.addCell(BufferCell.live(column, timestamp, CounterContext.instance().createLocal(increment)));
     }
 
     public void setComplexDeletionTime(ColumnDefinition column)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/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 ca3b5bc..614e47a 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -188,7 +188,7 @@ public abstract class ModificationStatement implements CQLStatement
 
     public int getTimeToLive(QueryOptions options) throws InvalidRequestException
     {
-        return attrs.getTimeToLive(options);
+        return attrs.getTimeToLive(options, cfm.params.defaultTimeToLive);
     }
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/db/LegacyLayout.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java
index d8446ac..92ecbf5 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -1167,7 +1167,7 @@ public abstract class LegacyLayout
             {
                 // It's the row marker
                 assert !cell.value.hasRemaining();
-                builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, cell.ttl, cell.localDeletionTime));
+                builder.addPrimaryKeyLivenessInfo(LivenessInfo.withExpirationTime(cell.timestamp, cell.ttl, cell.localDeletionTime));
             }
             else
             {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/db/LivenessInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/LivenessInfo.java b/src/java/org/apache/cassandra/db/LivenessInfo.java
index 3c87030..a328c71 100644
--- a/src/java/org/apache/cassandra/db/LivenessInfo.java
+++ b/src/java/org/apache/cassandra/db/LivenessInfo.java
@@ -46,12 +46,8 @@ public class LivenessInfo
         this.timestamp = timestamp;
     }
 
-    public static LivenessInfo create(CFMetaData metadata, long timestamp, int nowInSec)
+    public static LivenessInfo create(long timestamp, int nowInSec)
     {
-        int defaultTTL = metadata.params.defaultTimeToLive;
-        if (defaultTTL != NO_TTL)
-            return expiring(timestamp, defaultTTL, nowInSec);
-
         return new LivenessInfo(timestamp);
     }
 
@@ -60,16 +56,16 @@ public class LivenessInfo
         return new ExpiringLivenessInfo(timestamp, ttl, 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);
     }
 
-    // Note that this ctor ignores the default table ttl and takes the expiration time, not the current time.
+    // Note that this ctor 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)
+    public static LivenessInfo withExpirationTime(long timestamp, int ttl, int localExpirationTime)
     {
         return ttl == NO_TTL ? new LivenessInfo(timestamp) : new ExpiringLivenessInfo(timestamp, ttl, localExpirationTime);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RowUpdateBuilder.java b/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
index 0ceec90..b414eba 100644
--- a/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
+++ b/src/java/org/apache/cassandra/db/RowUpdateBuilder.java
@@ -85,7 +85,7 @@ public class RowUpdateBuilder
 
         // If a CQL table, add the "row marker"
         if (update.metadata().isCQLTable() && useRowMarker)
-            regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(update.metadata(), timestamp, ttl, localDeletionTime));
+            regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(timestamp, ttl, localDeletionTime));
     }
 
     private Row.Builder builder()
@@ -276,7 +276,7 @@ public class RowUpdateBuilder
     {
         return value == null
              ? BufferCell.tombstone(c, timestamp, localDeletionTime)
-             : (ttl == LivenessInfo.NO_TTL ? BufferCell.live(update.metadata(), c, timestamp, value, path) : BufferCell.expiring(c, timestamp, ttl, localDeletionTime, value, path));
+             : (ttl == LivenessInfo.NO_TTL ? BufferCell.live(c, timestamp, value, path) : BufferCell.expiring(c, timestamp, ttl, localDeletionTime, value, path));
     }
 
     public RowUpdateBuilder add(ColumnDefinition columnDefinition, Object value)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/db/rows/BufferCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/BufferCell.java b/src/java/org/apache/cassandra/db/rows/BufferCell.java
index cac63ac..d545e11 100644
--- a/src/java/org/apache/cassandra/db/rows/BufferCell.java
+++ b/src/java/org/apache/cassandra/db/rows/BufferCell.java
@@ -19,12 +19,10 @@ package org.apache.cassandra.db.rows;
 
 import java.nio.ByteBuffer;
 
-import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
 import org.apache.cassandra.db.marshal.ByteType;
 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
@@ -49,16 +47,13 @@ public class BufferCell extends AbstractCell
         this.path = path;
     }
 
-    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value)
+    public static BufferCell live(ColumnDefinition column, long timestamp, ByteBuffer value)
     {
-        return live(metadata, column, timestamp, value, null);
+        return live(column, timestamp, value, null);
     }
 
-    public static BufferCell live(CFMetaData metadata, ColumnDefinition column, long timestamp, ByteBuffer value, CellPath path)
+    public static BufferCell live(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);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
index ae9d6e9..0fa0f3e 100644
--- a/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
+++ b/src/java/org/apache/cassandra/db/rows/UnfilteredSerializer.java
@@ -420,7 +420,7 @@ public class UnfilteredSerializer
                 long timestamp = header.readTimestamp(in);
                 int ttl = hasTTL ? header.readTTL(in) : LivenessInfo.NO_TTL;
                 int localDeletionTime = hasTTL ? header.readLocalDeletionTime(in) : LivenessInfo.NO_EXPIRATION_TIME;
-                rowLiveness = LivenessInfo.create(timestamp, ttl, localDeletionTime);
+                rowLiveness = LivenessInfo.withExpirationTime(timestamp, ttl, localDeletionTime);
             }
 
             builder.addPrimaryKeyLivenessInfo(rowLiveness);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
index c05759c..4998034 100644
--- a/src/java/org/apache/cassandra/db/view/View.java
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -371,8 +371,7 @@ public class View
             clustering.add(value);
         }
         regularBuilder.newRow(clustering.build());
-        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(viewCfm,
-                                                                     temporalRow.viewClusteringTimestamp(),
+        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(temporalRow.viewClusteringTimestamp(),
                                                                      temporalRow.viewClusteringTtl(),
                                                                      temporalRow.viewClusteringLocalDeletionTime()));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
index c3e75e0..1503f0b 100644
--- a/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
+++ b/src/java/org/apache/cassandra/index/internal/CassandraIndex.java
@@ -424,7 +424,7 @@ public abstract class CassandraIndex implements Index
                 insert(key.getKey(),
                        clustering,
                        cell,
-                       LivenessInfo.create(cell.timestamp(), cell.ttl(), cell.localDeletionTime()),
+                       LivenessInfo.withExpirationTime(cell.timestamp(), cell.ttl(), cell.localDeletionTime()),
                        opGroup);
             }
 
@@ -472,7 +472,7 @@ public abstract class CassandraIndex implements Index
                         }
                     }
                 }
-                return LivenessInfo.create(baseCfs.metadata, timestamp, ttl, nowInSec);
+                return LivenessInfo.create(timestamp, ttl, nowInSec);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index dccd8a0..19fd87f 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -48,7 +48,6 @@ import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.*;
-import org.apache.cassandra.index.Index;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.locator.DynamicEndpointSnitch;
 import org.apache.cassandra.metrics.ClientMetrics;
@@ -817,9 +816,21 @@ public class CassandraServer implements Cassandra.Iface
     private Cell cellFromColumn(CFMetaData metadata, LegacyLayout.LegacyCellName name, Column column)
     {
         CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
-        return column.ttl == 0
-             ? BufferCell.live(metadata, name.column, column.timestamp, column.value, path)
-             : BufferCell.expiring(name.column, column.timestamp, column.ttl, FBUtilities.nowInSeconds(), column.value, path);
+        int ttl = getTtl(metadata, column);
+        return ttl == LivenessInfo.NO_TTL
+             ? BufferCell.live(name.column, column.timestamp, column.value, path)
+             : BufferCell.expiring(name.column, column.timestamp, ttl, FBUtilities.nowInSeconds(), column.value, path);
+    }
+
+    private int getTtl(CFMetaData metadata,Column column)
+    {
+        if (!column.isSetTtl())
+            return metadata.params.defaultTimeToLive;
+
+        if (column.ttl == LivenessInfo.NO_TTL && metadata.params.defaultTimeToLive != LivenessInfo.NO_TTL)
+            return LivenessInfo.NO_TTL;
+
+        return column.ttl;
     }
 
     private void internal_insert(ByteBuffer key, ColumnParent column_parent, Column column, ConsistencyLevel consistency_level)
@@ -2171,7 +2182,7 @@ public class CassandraServer implements Cassandra.Iface
                 // See UpdateParameters.addCounter() for more details on this
                 ByteBuffer value = CounterContext.instance().createLocal(column.value);
                 CellPath path = name.collectionElement == null ? null : CellPath.create(name.collectionElement);
-                Cell cell = BufferCell.live(metadata, name.column, FBUtilities.timestampMicros(), value, path);
+                Cell cell = BufferCell.live(name.column, FBUtilities.timestampMicros(), value, path);
 
                 PartitionUpdate update = PartitionUpdate.singleRowUpdate(metadata, key, BTreeRow.singleCellRow(name.clustering, cell));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/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 5e46459..be3e489 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -363,8 +363,8 @@ public class ThriftValidation
     {
         if (column.isSetTtl())
         {
-            if (column.ttl <= 0)
-                throw new org.apache.cassandra.exceptions.InvalidRequestException("ttl must be positive");
+            if (column.ttl < 0)
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("ttl must be greater or equal to 0");
 
             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));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
index ff98f6b..aa738bb 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/InsertTest.java
@@ -18,9 +18,12 @@
 
 package org.apache.cassandra.cql3.validation.operations;
 
+import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
 
 public class InsertTest extends CQLTester
 {
@@ -283,4 +286,32 @@ public class InsertTest extends CQLTester
         assertInvalidMessage("Some clustering keys are missing: clustering_1",
                              "INSERT INTO %s (partitionKey, clustering_2, staticValue) VALUES (0, 0, 'A')");
     }
+
+    @Test
+    public void testInsertWithDefaultTtl() throws Throwable
+    {
+        final int secondsPerMinute = 60;
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH default_time_to_live = " + (10 * secondsPerMinute));
+
+        execute("INSERT INTO %s (a, b) VALUES (1, 1)");
+        UntypedResultSet resultSet = execute("SELECT ttl(b) FROM %s WHERE a = 1");
+        Assert.assertEquals(1, resultSet.size());
+        Row row = resultSet.one();
+        Assert.assertTrue(row.getInt("ttl(b)") >= (9 * secondsPerMinute));
+
+        execute("INSERT INTO %s (a, b) VALUES (2, 2) USING TTL ?", (5 * secondsPerMinute));
+        resultSet = execute("SELECT ttl(b) FROM %s WHERE a = 2");
+        Assert.assertEquals(1, resultSet.size());
+        row = resultSet.one();
+        Assert.assertTrue(row.getInt("ttl(b)") <= (5 * secondsPerMinute));
+
+        execute("INSERT INTO %s (a, b) VALUES (3, 3) USING TTL ?", 0);
+        assertRows(execute("SELECT ttl(b) FROM %s WHERE a = 3"), row(new Object[]{null}));
+
+        execute("INSERT INTO %s (a, b) VALUES (4, 4) USING TTL ?", unset());
+        resultSet = execute("SELECT ttl(b) FROM %s WHERE a = 4");
+        Assert.assertEquals(1, resultSet.size());
+        row = resultSet.one();
+        Assert.assertTrue(row.getInt("ttl(b)") >= (9 * secondsPerMinute));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
index d1209ac..b2e59a3 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/operations/UpdateTest.java
@@ -20,10 +20,13 @@ package org.apache.cassandra.cql3.validation.operations;
 
 import java.util.Arrays;
 
+import org.junit.Assert;
 import org.junit.Test;
 
 import static org.apache.commons.lang3.StringUtils.isEmpty;
 import org.apache.cassandra.cql3.CQLTester;
+import org.apache.cassandra.cql3.UntypedResultSet;
+import org.apache.cassandra.cql3.UntypedResultSet.Row;
 import org.apache.cassandra.utils.ByteBufferUtil;
 
 public class UpdateTest extends CQLTester
@@ -520,4 +523,32 @@ public class UpdateTest extends CQLTester
 
         assertRows(execute("SELECT l FROM %s WHERE k = 0"), row(list("v1", "v4", "v3")));
     }
+
+    @Test
+    public void testUpdateWithDefaultTtl() throws Throwable
+    {
+        final int secondsPerMinute = 60;
+        createTable("CREATE TABLE %s (a int PRIMARY KEY, b int) WITH default_time_to_live = " + (10 * secondsPerMinute));
+
+        execute("UPDATE %s SET b = 1 WHERE a = 1");
+        UntypedResultSet resultSet = execute("SELECT ttl(b) FROM %s WHERE a = 1");
+        Assert.assertEquals(1, resultSet.size());
+        Row row = resultSet.one();
+        Assert.assertTrue(row.getInt("ttl(b)") >= (9 * secondsPerMinute));
+
+        execute("UPDATE %s USING TTL ? SET b = 3 WHERE a = 1", 0);
+        assertRows(execute("SELECT ttl(b) FROM %s WHERE a = 1"), row(new Object[]{null}));
+
+        execute("UPDATE %s SET b = 3 WHERE a = 1");
+        resultSet = execute("SELECT ttl(b) FROM %s WHERE a = 1");
+        Assert.assertEquals(1, resultSet.size());
+        row = resultSet.one();
+        Assert.assertTrue(row.getInt("ttl(b)") >= (9 * secondsPerMinute));
+
+        execute("UPDATE %s USING TTL ? SET b = 2 WHERE a = 2", unset());
+        resultSet = execute("SELECT ttl(b) FROM %s WHERE a = 2");
+        Assert.assertEquals(1, resultSet.size());
+        row = resultSet.one();
+        Assert.assertTrue(row.getInt("ttl(b)") >= (9 * secondsPerMinute));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/CellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CellTest.java b/test/unit/org/apache/cassandra/db/CellTest.java
index 5953255..e81ac12 100644
--- a/test/unit/org/apache/cassandra/db/CellTest.java
+++ b/test/unit/org/apache/cassandra/db/CellTest.java
@@ -114,15 +114,15 @@ public class CellTest
         long ts1 = now1*1000000;
 
 
-        Cell r1m1 = BufferCell.live(cfm2, m, ts1, bb(1), CellPath.create(bb(1)));
-        Cell r1m2 = BufferCell.live(cfm2, m, ts1, bb(2), CellPath.create(bb(2)));
+        Cell r1m1 = BufferCell.live(m, ts1, bb(1), CellPath.create(bb(1)));
+        Cell r1m2 = BufferCell.live(m, ts1, bb(2), CellPath.create(bb(2)));
         List<Cell> cells1 = Lists.newArrayList(r1m1, r1m2);
 
         int now2 = now1 + 1;
         long ts2 = now2*1000000;
-        Cell r2m2 = BufferCell.live(cfm2, m, ts2, bb(1), CellPath.create(bb(2)));
-        Cell r2m3 = BufferCell.live(cfm2, m, ts2, bb(2), CellPath.create(bb(3)));
-        Cell r2m4 = BufferCell.live(cfm2, m, ts2, bb(3), CellPath.create(bb(4)));
+        Cell r2m2 = BufferCell.live(m, ts2, bb(1), CellPath.create(bb(2)));
+        Cell r2m3 = BufferCell.live(m, ts2, bb(2), CellPath.create(bb(3)));
+        Cell r2m4 = BufferCell.live(m, ts2, bb(3), CellPath.create(bb(4)));
         List<Cell> cells2 = Lists.newArrayList(r2m2, r2m3, r2m4);
 
         RowBuilder builder = new RowBuilder();
@@ -152,7 +152,7 @@ public class CellTest
     private Cell regular(CFMetaData cfm, String columnName, String value, long timestamp)
     {
         ColumnDefinition cdef = cfm.getColumnDefinition(ByteBufferUtil.bytes(columnName));
-        return BufferCell.live(cfm, cdef, timestamp, ByteBufferUtil.bytes(value));
+        return BufferCell.live(cdef, timestamp, ByteBufferUtil.bytes(value));
     }
 
     private Cell expiring(CFMetaData cfm, String columnName, String value, long timestamp, int localExpirationTime)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/CounterCellTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/CounterCellTest.java b/test/unit/org/apache/cassandra/db/CounterCellTest.java
index 08e0b25..b09bfad 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -101,20 +101,20 @@ public class CounterCellTest
     {
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
         ByteBuffer val = CounterContext.instance().createLocal(count);
-        return BufferCell.live(cfs.metadata, cDef, ts, val);
+        return BufferCell.live(cDef, ts, val);
     }
 
     private Cell createCounterCell(ColumnFamilyStore cfs, ByteBuffer colName, CounterId id, long count, long ts)
     {
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
         ByteBuffer val = CounterContext.instance().createGlobal(id, ts, count);
-        return BufferCell.live(cfs.metadata, cDef, ts, val);
+        return BufferCell.live(cDef, ts, val);
     }
 
     private Cell createCounterCellFromContext(ColumnFamilyStore cfs, ByteBuffer colName, ContextState context, long ts)
     {
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(colName);
-        return BufferCell.live(cfs.metadata, cDef, ts, context.context);
+        return BufferCell.live(cDef, ts, context.context);
     }
 
     private Cell createDeleted(ColumnFamilyStore cfs, ByteBuffer colName, long ts, int localDeletionTime)
@@ -274,7 +274,7 @@ public class CounterCellTest
         Cell original = createCounterCellFromContext(cfs, col, state, 5);
 
         ColumnDefinition cDef = cfs.metadata.getColumnDefinition(col);
-        Cell cleared = BufferCell.live(cfs.metadata, cDef, 5, CounterContext.instance().clearAllLocal(state.context));
+        Cell cleared = BufferCell.live(cDef, 5, CounterContext.instance().clearAllLocal(state.context));
 
         CounterContext.instance().updateDigest(digest1, original.value());
         CounterContext.instance().updateDigest(digest2, cleared.value());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/RowTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/RowTest.java b/test/unit/org/apache/cassandra/db/RowTest.java
index 0af183c..58baf82 100644
--- a/test/unit/org/apache/cassandra/db/RowTest.java
+++ b/test/unit/org/apache/cassandra/db/RowTest.java
@@ -193,6 +193,6 @@ public class RowTest
                                       String value,
                                       long timestamp)
     {
-       builder.addCell(BufferCell.live(cfm, columnDefinition, timestamp, ((AbstractType) columnDefinition.cellValueType()).decompose(value)));
+       builder.addCell(BufferCell.live(columnDefinition, timestamp, ((AbstractType) columnDefinition.cellValueType()).decompose(value)));
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
index fe6ac1e..6cbab7a 100644
--- a/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
+++ b/test/unit/org/apache/cassandra/db/partition/PartitionImplementationTest.java
@@ -104,7 +104,7 @@ public class PartitionImplementationTest
         ColumnDefinition defCol = cfm.getColumnDefinition(new ColumnIdentifier("col", true));
         Row.Builder row = BTreeRow.unsortedBuilder(TIMESTAMP);
         row.newRow(clustering);
-        row.addCell(BufferCell.live(cfm, defCol, TIMESTAMP, ByteBufferUtil.bytes(colValue)));
+        row.addCell(BufferCell.live(defCol, TIMESTAMP, ByteBufferUtil.bytes(colValue)));
         return row.build();
     }
 
@@ -113,7 +113,7 @@ public class PartitionImplementationTest
         ColumnDefinition defCol = cfm.getColumnDefinition(new ColumnIdentifier("static_col", true));
         Row.Builder row = BTreeRow.unsortedBuilder(TIMESTAMP);
         row.newRow(Clustering.STATIC_CLUSTERING);
-        row.addCell(BufferCell.live(cfm, defCol, TIMESTAMP, ByteBufferUtil.bytes("static value")));
+        row.addCell(BufferCell.live(defCol, TIMESTAMP, ByteBufferUtil.bytes("static value")));
         return row.build();
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java b/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
index 5a2db1f..a72d397 100644
--- a/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/DigestBackwardCompatibilityTest.java
@@ -76,7 +76,6 @@ public class DigestBackwardCompatibilityTest extends CQLTester
         createTable("CREATE TABLE %s (k text, t int, v1 text, v2 int, PRIMARY KEY (k, t))");
 
         String key = "someKey";
-        int N = 10;
 
         for (int i = 0; i < 10; i++)
             execute("INSERT INTO %s(k, t, v1, v2) VALUES (?, ?, ?, ?) USING TIMESTAMP ? AND TTL ?", key, i, "v" + i, i, 1L, 200);
@@ -103,7 +102,6 @@ public class DigestBackwardCompatibilityTest extends CQLTester
         createTable("CREATE TABLE %s (k text, t int, v text, PRIMARY KEY (k, t)) WITH COMPACT STORAGE");
 
         String key = "someKey";
-        int N = 10;
 
         for (int i = 0; i < 10; i++)
             execute("INSERT INTO %s(k, t, v) VALUES (?, ?, ?) USING TIMESTAMP ? AND TTL ?", key, i, "v" + i, 1L, 200);
@@ -172,7 +170,7 @@ public class DigestBackwardCompatibilityTest extends CQLTester
         CFMetaData metadata = getCurrentColumnFamilyStore().metadata;
         ColumnDefinition column = metadata.getColumnDefinition(ByteBufferUtil.bytes("c"));
         ByteBuffer value = CounterContext.instance().createGlobal(CounterId.fromInt(1), 1L, 42L);
-        Row row = BTreeRow.singleCellRow(Clustering.STATIC_CLUSTERING, BufferCell.live(metadata, column, 0L, value));
+        Row row = BTreeRow.singleCellRow(Clustering.STATIC_CLUSTERING, BufferCell.live(column, 0L, value));
 
         new Mutation(PartitionUpdate.singleRowUpdate(metadata, Util.dk(key), row)).applyUnsafe();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
index 98ad2bc..e8ac339 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowAndDeletionMergeIteratorTest.java
@@ -380,12 +380,12 @@ public class RowAndDeletionMergeIteratorTest
 
     private void addRow(PartitionUpdate update, int col1, int a)
     {
-        update.add(BTreeRow.singleCellRow(update.metadata().comparator.make(col1), makeCell(cfm, defA, a, 0)));
+        update.add(BTreeRow.singleCellRow(update.metadata().comparator.make(col1), makeCell(defA, a, 0)));
     }
 
-    private Cell makeCell(CFMetaData cfm, ColumnDefinition columnDefinition, int value, long timestamp)
+    private Cell makeCell(ColumnDefinition columnDefinition, int value, long timestamp)
     {
-        return BufferCell.live(cfm, columnDefinition, timestamp, ((AbstractType)columnDefinition.cellValueType()).decompose(value));
+        return BufferCell.live(columnDefinition, timestamp, ((AbstractType)columnDefinition.cellValueType()).decompose(value));
     }
 
     private static RangeTombstone atLeast(int start, long tstamp, int delTime)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/rows/RowsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/RowsTest.java b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
index dede867..00ab6ca 100644
--- a/test/unit/org/apache/cassandra/db/rows/RowsTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/RowsTest.java
@@ -210,15 +210,15 @@ public class RowsTest
         long ts = secondToTs(now);
         Row.Builder builder = BTreeRow.unsortedBuilder(now);
         builder.newRow(c);
-        builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(kcvm, ts, now));
+        builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(ts, now));
         if (vVal != null)
         {
-            builder.addCell(BufferCell.live(kcvm, v, ts, vVal));
+            builder.addCell(BufferCell.live(v, ts, vVal));
         }
         if (mKey != null && mVal != null)
         {
             builder.addComplexDeletion(m, new DeletionTime(ts - 1, now));
-            builder.addCell(BufferCell.live(kcvm, m, ts, mVal, CellPath.create(mKey)));
+            builder.addCell(BufferCell.live(m, ts, mVal, CellPath.create(mKey)));
         }
 
         return builder;
@@ -231,13 +231,13 @@ public class RowsTest
         long ts = secondToTs(now);
         Row.Builder originalBuilder = BTreeRow.unsortedBuilder(now);
         originalBuilder.newRow(c1);
-        LivenessInfo liveness = LivenessInfo.create(kcvm, ts, now);
+        LivenessInfo liveness = LivenessInfo.create(ts, now);
         originalBuilder.addPrimaryKeyLivenessInfo(liveness);
         DeletionTime complexDeletion = new DeletionTime(ts-1, now);
         originalBuilder.addComplexDeletion(m, complexDeletion);
-        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(kcvm, v, secondToTs(now), BB1),
-                                                      BufferCell.live(kcvm, m, secondToTs(now), BB1, CellPath.create(BB1)),
-                                                      BufferCell.live(kcvm, m, secondToTs(now), BB2, CellPath.create(BB2)));
+        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(v, secondToTs(now), BB1),
+                                                      BufferCell.live(m, secondToTs(now), BB1, CellPath.create(BB1)),
+                                                      BufferCell.live(m, secondToTs(now), BB2, CellPath.create(BB2)));
         expectedCells.forEach(originalBuilder::addCell);
         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
         originalBuilder.addRowDeletion(rowDeletion);
@@ -259,13 +259,13 @@ public class RowsTest
         long ts = secondToTs(now);
         Row.Builder builder = BTreeRow.unsortedBuilder(now);
         builder.newRow(c1);
-        LivenessInfo liveness = LivenessInfo.create(kcvm, ts, now);
+        LivenessInfo liveness = LivenessInfo.create(ts, now);
         builder.addPrimaryKeyLivenessInfo(liveness);
         DeletionTime complexDeletion = new DeletionTime(ts-1, now);
         builder.addComplexDeletion(m, complexDeletion);
-        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(kcvm, v, ts, BB1),
-                                                      BufferCell.live(kcvm, m, ts, BB1, CellPath.create(BB1)),
-                                                      BufferCell.live(kcvm, m, ts, BB2, CellPath.create(BB2)));
+        List<Cell> expectedCells = Lists.newArrayList(BufferCell.live(v, ts, BB1),
+                                                      BufferCell.live(m, ts, BB1, CellPath.create(BB1)),
+                                                      BufferCell.live(m, ts, BB2, CellPath.create(BB2)));
         expectedCells.forEach(builder::addCell);
         Row.Deletion rowDeletion = new Row.Deletion(new DeletionTime(ts, now), false);
         builder.addRowDeletion(rowDeletion);
@@ -296,14 +296,14 @@ public class RowsTest
         long ts1 = secondToTs(now1);
         Row.Builder r1Builder = BTreeRow.unsortedBuilder(now1);
         r1Builder.newRow(c1);
-        LivenessInfo r1Liveness = LivenessInfo.create(kcvm, ts1, now1);
+        LivenessInfo r1Liveness = LivenessInfo.create(ts1, now1);
         r1Builder.addPrimaryKeyLivenessInfo(r1Liveness);
         DeletionTime r1ComplexDeletion = new DeletionTime(ts1-1, now1);
         r1Builder.addComplexDeletion(m, r1ComplexDeletion);
 
-        Cell r1v = BufferCell.live(kcvm, v, ts1, BB1);
-        Cell r1m1 = BufferCell.live(kcvm, m, ts1, BB1, CellPath.create(BB1));
-        Cell r1m2 = BufferCell.live(kcvm, m, ts1, BB2, CellPath.create(BB2));
+        Cell r1v = BufferCell.live(v, ts1, BB1);
+        Cell r1m1 = BufferCell.live(m, ts1, BB1, CellPath.create(BB1));
+        Cell r1m2 = BufferCell.live(m, ts1, BB2, CellPath.create(BB2));
         List<Cell> r1ExpectedCells = Lists.newArrayList(r1v, r1m1, r1m2);
 
         r1ExpectedCells.forEach(r1Builder::addCell);
@@ -312,12 +312,12 @@ public class RowsTest
         long ts2 = secondToTs(now2);
         Row.Builder r2Builder = BTreeRow.unsortedBuilder(now2);
         r2Builder.newRow(c1);
-        LivenessInfo r2Liveness = LivenessInfo.create(kcvm, ts2, now2);
+        LivenessInfo r2Liveness = LivenessInfo.create(ts2, now2);
         r2Builder.addPrimaryKeyLivenessInfo(r2Liveness);
-        Cell r2v = BufferCell.live(kcvm, v, ts2, BB2);
-        Cell r2m2 = BufferCell.live(kcvm, m, ts2, BB1, CellPath.create(BB2));
-        Cell r2m3 = BufferCell.live(kcvm, m, ts2, BB2, CellPath.create(BB3));
-        Cell r2m4 = BufferCell.live(kcvm, m, ts2, BB3, CellPath.create(BB4));
+        Cell r2v = BufferCell.live(v, ts2, BB2);
+        Cell r2m2 = BufferCell.live(m, ts2, BB1, CellPath.create(BB2));
+        Cell r2m3 = BufferCell.live(m, ts2, BB2, CellPath.create(BB3));
+        Cell r2m4 = BufferCell.live(m, ts2, BB3, CellPath.create(BB4));
         List<Cell> r2ExpectedCells = Lists.newArrayList(r2v, r2m2, r2m3, r2m4);
 
         r2ExpectedCells.forEach(r2Builder::addCell);
@@ -372,7 +372,7 @@ public class RowsTest
         long ts1 = secondToTs(now1);
         Row.Builder r1Builder = BTreeRow.unsortedBuilder(now1);
         r1Builder.newRow(c1);
-        LivenessInfo r1Liveness = LivenessInfo.create(kcvm, ts1, now1);
+        LivenessInfo r1Liveness = LivenessInfo.create(ts1, now1);
         r1Builder.addPrimaryKeyLivenessInfo(r1Liveness);
 
         // mergedData == null
@@ -380,14 +380,14 @@ public class RowsTest
         long ts2 = secondToTs(now2);
         Row.Builder r2Builder = BTreeRow.unsortedBuilder(now2);
         r2Builder.newRow(c1);
-        LivenessInfo r2Liveness = LivenessInfo.create(kcvm, ts2, now2);
+        LivenessInfo r2Liveness = LivenessInfo.create(ts2, now2);
         r2Builder.addPrimaryKeyLivenessInfo(r2Liveness);
         DeletionTime r2ComplexDeletion = new DeletionTime(ts2-1, now2);
         r2Builder.addComplexDeletion(m, r2ComplexDeletion);
-        Cell r2v = BufferCell.live(kcvm, v, ts2, BB2);
-        Cell r2m2 = BufferCell.live(kcvm, m, ts2, BB1, CellPath.create(BB2));
-        Cell r2m3 = BufferCell.live(kcvm, m, ts2, BB2, CellPath.create(BB3));
-        Cell r2m4 = BufferCell.live(kcvm, m, ts2, BB3, CellPath.create(BB4));
+        Cell r2v = BufferCell.live(v, ts2, BB2);
+        Cell r2m2 = BufferCell.live(m, ts2, BB1, CellPath.create(BB2));
+        Cell r2m3 = BufferCell.live(m, ts2, BB2, CellPath.create(BB3));
+        Cell r2m4 = BufferCell.live(m, ts2, BB3, CellPath.create(BB4));
         List<Cell> r2ExpectedCells = Lists.newArrayList(r2v, r2m2, r2m3, r2m4);
 
         r2ExpectedCells.forEach(r2Builder::addCell);
@@ -426,7 +426,7 @@ public class RowsTest
         long ts1 = secondToTs(now1);
         Row.Builder r1Builder = BTreeRow.unsortedBuilder(now1);
         r1Builder.newRow(c1);
-        LivenessInfo r1Liveness = LivenessInfo.create(kcvm, ts1, now1);
+        LivenessInfo r1Liveness = LivenessInfo.create(ts1, now1);
         r1Builder.addPrimaryKeyLivenessInfo(r1Liveness);
 
         // mergedData == null
@@ -434,14 +434,14 @@ public class RowsTest
         long ts2 = secondToTs(now2);
         Row.Builder r2Builder = BTreeRow.unsortedBuilder(now2);
         r2Builder.newRow(c1);
-        LivenessInfo r2Liveness = LivenessInfo.create(kcvm, ts2, now2);
+        LivenessInfo r2Liveness = LivenessInfo.create(ts2, now2);
         r2Builder.addPrimaryKeyLivenessInfo(r2Liveness);
         DeletionTime r2ComplexDeletion = new DeletionTime(ts2-1, now2);
         r2Builder.addComplexDeletion(m, r2ComplexDeletion);
-        Cell r2v = BufferCell.live(kcvm, v, ts2, BB2);
-        Cell r2m2 = BufferCell.live(kcvm, m, ts2, BB1, CellPath.create(BB2));
-        Cell r2m3 = BufferCell.live(kcvm, m, ts2, BB2, CellPath.create(BB3));
-        Cell r2m4 = BufferCell.live(kcvm, m, ts2, BB3, CellPath.create(BB4));
+        Cell r2v = BufferCell.live(v, ts2, BB2);
+        Cell r2m2 = BufferCell.live(m, ts2, BB1, CellPath.create(BB2));
+        Cell r2m3 = BufferCell.live(m, ts2, BB2, CellPath.create(BB3));
+        Cell r2m4 = BufferCell.live(m, ts2, BB3, CellPath.create(BB4));
         List<Cell> r2ExpectedCells = Lists.newArrayList(r2v, r2m2, r2m3, r2m4);
 
         r2ExpectedCells.forEach(r2Builder::addCell);
@@ -479,8 +479,8 @@ public class RowsTest
         int now2 = now1 + 1;
         long ts2 = secondToTs(now2);
 
-        Cell expectedVCell = BufferCell.live(kcvm, v, ts2, BB2);
-        Cell expectedMCell = BufferCell.live(kcvm, m, ts2, BB2, CellPath.create(BB1));
+        Cell expectedVCell = BufferCell.live(v, ts2, BB2);
+        Cell expectedMCell = BufferCell.live(m, ts2, BB2, CellPath.create(BB1));
         DeletionTime expectedComplexDeletionTime = new DeletionTime(ts2 - 1, now2);
 
         Row.Builder updateBuilder = createBuilder(c1, now2, null, null, null);
@@ -492,7 +492,7 @@ public class RowsTest
         long td = Rows.merge(existingBuilder.build(), updateBuilder.build(), builder, now2 + 1);
 
         Assert.assertEquals(c1, builder.clustering);
-        Assert.assertEquals(LivenessInfo.create(kcvm, ts2, now2), builder.livenessInfo);
+        Assert.assertEquals(LivenessInfo.create(ts2, now2), builder.livenessInfo);
         Assert.assertEquals(Lists.newArrayList(Pair.create(m, new DeletionTime(ts2-1, now2))), builder.complexDeletions);
 
         Assert.assertEquals(2, builder.cells.size());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
index 6f1985a..0c47cbb 100644
--- a/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
+++ b/test/unit/org/apache/cassandra/db/rows/UnfilteredRowIteratorsMergeTest.java
@@ -370,7 +370,7 @@ public class UnfilteredRowIteratorsMergeTest
     static Row emptyRowAt(int pos, Function<Integer, Integer> timeGenerator)
     {
         final Clustering clustering = clusteringFor(pos);
-        final LivenessInfo live = LivenessInfo.create(metadata, timeGenerator.apply(pos), nowInSec);
+        final LivenessInfo live = LivenessInfo.create(timeGenerator.apply(pos), nowInSec);
         return BTreeRow.noCellLiveRow(clustering, live);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
index 9cb2460..51457c9 100644
--- a/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
+++ b/test/unit/org/apache/cassandra/index/internal/CustomCassandraIndex.java
@@ -356,7 +356,7 @@ public class CustomCassandraIndex implements Index
                 insert(key.getKey(),
                        clustering,
                        cell,
-                       LivenessInfo.create(cell.timestamp(), cell.ttl(), cell.localDeletionTime()),
+                       LivenessInfo.withExpirationTime(cell.timestamp(), cell.ttl(), cell.localDeletionTime()),
                        opGroup);
             }
 
@@ -404,7 +404,7 @@ public class CustomCassandraIndex implements Index
                         }
                     }
                 }
-                return LivenessInfo.create(baseCfs.metadata, timestamp, ttl, nowInSec);
+                return LivenessInfo.create(timestamp, ttl, nowInSec);
             }
         };
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
index c9d66f7..17bb331 100644
--- a/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/SASIIndexTest.java
@@ -2201,14 +2201,14 @@ public class SASIIndexTest
     private static Cell buildCell(ByteBuffer name, ByteBuffer value, long timestamp)
     {
         CFMetaData cfm = Keyspace.open(KS_NAME).getColumnFamilyStore(CF_NAME).metadata;
-        return BufferCell.live(cfm, cfm.getColumnDefinition(name), timestamp, value);
+        return BufferCell.live(cfm.getColumnDefinition(name), timestamp, value);
     }
 
     private static Cell buildCell(CFMetaData cfm, ByteBuffer name, ByteBuffer value, long timestamp)
     {
         ColumnDefinition column = cfm.getColumnDefinition(name);
         assert column != null;
-        return BufferCell.live(cfm, column, timestamp, value);
+        return BufferCell.live(column, timestamp, value);
     }
 
     private static Expression buildExpression(ByteBuffer name, Operator op, ByteBuffer value)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
index 4663692..826ece1 100644
--- a/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/disk/PerSSTableIndexWriterTest.java
@@ -88,7 +88,7 @@ public class PerSSTableIndexWriterTest extends SchemaLoader
             ByteBuffer key = ByteBufferUtil.bytes(String.format(keyFormat, i));
             expectedKeys.put(cfs.metadata.partitioner.decorateKey(key),
                              BTreeRow.singleCellRow(Clustering.EMPTY,
-                                                    BufferCell.live(cfs.metadata, column, timestamp, Int32Type.instance.decompose(i))));
+                                                    BufferCell.live(column, timestamp, Int32Type.instance.decompose(i))));
         }
 
         indexWriter.begin();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
index cf2b8c0..22d5c88 100644
--- a/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
+++ b/test/unit/org/apache/cassandra/index/sasi/plan/OperationTest.java
@@ -625,7 +625,7 @@ public class OperationTest extends SchemaLoader
 
     private static Cell buildCell(ColumnDefinition column, ByteBuffer value, long timestamp)
     {
-        return BufferCell.live(BACKEND.metadata, column, timestamp, value);
+        return BufferCell.live(column, timestamp, value);
     }
 
     private static Cell deletedCell(ColumnDefinition column, long timestamp, int nowInSeconds)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
index 9cac007..27c76dd 100644
--- a/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/format/SSTableFlushObserverTest.java
@@ -100,23 +100,23 @@ public class SSTableFlushObserverTest
             final long now = System.currentTimeMillis();
 
             ByteBuffer key = UTF8Type.instance.fromString("key1");
-            expected.putAll(key, Arrays.asList(BufferCell.live(cfm, getColumn(cfm, "first_name"), now,UTF8Type.instance.fromString("jack")),
-                                               BufferCell.live(cfm, getColumn(cfm, "age"), now, Int32Type.instance.decompose(27)),
-                                               BufferCell.live(cfm, getColumn(cfm, "height"), now, LongType.instance.decompose(183L))));
+            expected.putAll(key, Arrays.asList(BufferCell.live(getColumn(cfm, "first_name"), now,UTF8Type.instance.fromString("jack")),
+                                               BufferCell.live(getColumn(cfm, "age"), now, Int32Type.instance.decompose(27)),
+                                               BufferCell.live(getColumn(cfm, "height"), now, LongType.instance.decompose(183L))));
 
             writer.append(new RowIterator(cfm, key.duplicate(), Collections.singletonList(buildRow(expected.get(key)))));
 
             key = UTF8Type.instance.fromString("key2");
-            expected.putAll(key, Arrays.asList(BufferCell.live(cfm, getColumn(cfm, "first_name"), now,UTF8Type.instance.fromString("jim")),
-                                               BufferCell.live(cfm, getColumn(cfm, "age"), now, Int32Type.instance.decompose(30)),
-                                               BufferCell.live(cfm, getColumn(cfm, "height"), now, LongType.instance.decompose(180L))));
+            expected.putAll(key, Arrays.asList(BufferCell.live(getColumn(cfm, "first_name"), now,UTF8Type.instance.fromString("jim")),
+                                               BufferCell.live(getColumn(cfm, "age"), now, Int32Type.instance.decompose(30)),
+                                               BufferCell.live(getColumn(cfm, "height"), now, LongType.instance.decompose(180L))));
 
             writer.append(new RowIterator(cfm, key, Collections.singletonList(buildRow(expected.get(key)))));
 
             key = UTF8Type.instance.fromString("key3");
-            expected.putAll(key, Arrays.asList(BufferCell.live(cfm, getColumn(cfm, "first_name"), now,UTF8Type.instance.fromString("ken")),
-                                               BufferCell.live(cfm, getColumn(cfm, "age"), now, Int32Type.instance.decompose(30)),
-                                               BufferCell.live(cfm, getColumn(cfm, "height"), now, LongType.instance.decompose(178L))));
+            expected.putAll(key, Arrays.asList(BufferCell.live(getColumn(cfm, "first_name"), now,UTF8Type.instance.fromString("ken")),
+                                               BufferCell.live(getColumn(cfm, "age"), now, Int32Type.instance.decompose(30)),
+                                               BufferCell.live(getColumn(cfm, "height"), now, LongType.instance.decompose(178L))));
 
             writer.append(new RowIterator(cfm, key, Collections.singletonList(buildRow(expected.get(key)))));
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/service/DataResolverTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/DataResolverTest.java b/test/unit/org/apache/cassandra/service/DataResolverTest.java
index b753362..b20dfc0 100644
--- a/test/unit/org/apache/cassandra/service/DataResolverTest.java
+++ b/test/unit/org/apache/cassandra/service/DataResolverTest.java
@@ -447,7 +447,7 @@ public class DataResolverTest
 
     private Cell mapCell(int k, int v, long ts)
     {
-        return BufferCell.live(cfm2, m, ts, bb(v), CellPath.create(bb(k)));
+        return BufferCell.live(m, ts, bb(v), CellPath.create(bb(k)));
     }
 
     @Test

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java b/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java
index f4ac377..98fa959 100644
--- a/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java
+++ b/test/unit/org/apache/cassandra/service/pager/PagingStateTest.java
@@ -50,7 +50,7 @@ public class PagingStateTest
 
         ColumnDefinition def = metadata.getColumnDefinition(new ColumnIdentifier("myCol", false));
         Clustering c = Clustering.make(ByteBufferUtil.bytes("c1"), ByteBufferUtil.bytes(42));
-        Row row = BTreeRow.singleCellRow(c, BufferCell.live(metadata, def, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER));
+        Row row = BTreeRow.singleCellRow(c, BufferCell.live(def, 0, ByteBufferUtil.EMPTY_BYTE_BUFFER));
         PagingState.RowMark mark = PagingState.RowMark.create(metadata, row, protocolVersion);
         return new PagingState(pk, mark, 10, 0);
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/e017f949/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
index 44391c8..36efabd 100644
--- a/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
+++ b/test/unit/org/apache/cassandra/triggers/TriggerExecutorTest.java
@@ -46,12 +46,14 @@ public class TriggerExecutorTest
         CFMetaData metadata = makeCfMetaData("ks1", "cf1", TriggerMetadata.create("test", SameKeySameCfTrigger.class.getName()));
         PartitionUpdate mutated = TriggerExecutor.instance.execute(makeCf(metadata, "k1", "v1", null));
 
-        RowIterator rowIterator = UnfilteredRowIterators.filter(mutated.unfilteredIterator(), FBUtilities.nowInSeconds());
-
-        Iterator<Cell> cells = rowIterator.next().cells().iterator();
-        assertEquals(bytes("trigger"), cells.next().value());
+        try (RowIterator rowIterator = UnfilteredRowIterators.filter(mutated.unfilteredIterator(),
+                                                                     FBUtilities.nowInSeconds()))
+        {
+            Iterator<Cell> cells = rowIterator.next().cells().iterator();
+            assertEquals(bytes("trigger"), cells.next().value());
 
-        assertTrue(!rowIterator.hasNext());
+            assertTrue(!rowIterator.hasNext());
+        }
     }
 
     @Test(expected = InvalidRequestException.class)
@@ -272,9 +274,9 @@ public class TriggerExecutorTest
         builder.newRow(Clustering.EMPTY);
         long ts = FBUtilities.timestampMicros();
         if (columnValue1 != null)
-            builder.addCell(BufferCell.live(metadata, metadata.getColumnDefinition(bytes("c1")), ts, bytes(columnValue1)));
+            builder.addCell(BufferCell.live(metadata.getColumnDefinition(bytes("c1")), ts, bytes(columnValue1)));
         if (columnValue2 != null)
-            builder.addCell(BufferCell.live(metadata, metadata.getColumnDefinition(bytes("c2")), ts, bytes(columnValue2)));
+            builder.addCell(BufferCell.live(metadata.getColumnDefinition(bytes("c2")), ts, bytes(columnValue2)));
 
         return PartitionUpdate.singleRowUpdate(metadata, Util.dk(key), builder.build());
     }