You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2014/06/19 21:02:34 UTC

[1/2] git commit: Modify reconcile logic to always pick a tombstone over a counter cell

Repository: cassandra
Updated Branches:
  refs/heads/trunk 295fe5dd0 -> 23c2fa8c8


Modify reconcile logic to always pick a tombstone over a counter cell

patch by Aleksey Yeschenko; reviewed by Sylvain Lebresne for
CASSANDRA-7346


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

Branch: refs/heads/trunk
Commit: 6858dd3e3ab354c517503d310e50319be93cf6a2
Parents: 43ffb6b
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Thu Jun 19 11:54:33 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Thu Jun 19 11:54:33 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../org/apache/cassandra/db/AbstractCell.java   | 69 +++++++++-----------
 .../cassandra/db/ArrayBackedSortedColumns.java  |  2 +-
 .../apache/cassandra/db/BufferCounterCell.java  |  4 +-
 .../cassandra/db/BufferCounterUpdateCell.java   | 15 ++---
 .../org/apache/cassandra/db/ColumnFamily.java   |  2 +-
 .../org/apache/cassandra/db/ColumnIndex.java    |  2 +-
 .../org/apache/cassandra/db/DeletionInfo.java   | 35 ++++------
 .../org/apache/cassandra/db/DeletionTime.java   | 10 +++
 .../apache/cassandra/db/NativeCounterCell.java  |  6 +-
 .../apache/cassandra/db/RangeTombstoneList.java | 31 ++++++---
 .../cassandra/db/filter/ColumnCounter.java      |  2 +-
 test/unit/org/apache/cassandra/Util.java        |  5 --
 .../apache/cassandra/db/ColumnFamilyTest.java   | 39 ++++++++++-
 .../apache/cassandra/db/CounterCellTest.java    | 20 ++----
 .../cassandra/utils/EncodedStreamsTest.java     | 10 +--
 16 files changed, 139 insertions(+), 115 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 783bfda..a2205ff 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.1.0
+ * Modify reconcile logic to always pick a tombstone over a counter cell
+   (CASSANDRA-7346)
  * Avoid incremental compaction on Windows (CASSANDRA-7365)
  * Fix exception when querying a composite-keyed table with a collection index
    (CASSANDRA-7372)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/AbstractCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/AbstractCell.java b/src/java/org/apache/cassandra/db/AbstractCell.java
index 238b8c0..9dad6db 100644
--- a/src/java/org/apache/cassandra/db/AbstractCell.java
+++ b/src/java/org/apache/cassandra/db/AbstractCell.java
@@ -181,66 +181,59 @@ public abstract class AbstractCell implements Cell
                 : new BufferCell(name, value, timestamp);
     }
 
-    public static Cell diff(CounterCell a, Cell b)
+    public Cell diffCounter(Cell cell)
     {
-        if (a.timestamp() < b.timestamp())
-            return b;
+        assert this instanceof CounterCell : "Wrong class type: " + getClass();
+
+        if (timestamp() < cell.timestamp())
+            return cell;
 
         // Note that if at that point, cell can't be a tombstone. Indeed,
         // cell is the result of merging us with other nodes results, and
         // merging a CounterCell with a tombstone never return a tombstone
         // unless that tombstone timestamp is greater that the CounterCell
         // one.
-        assert b instanceof CounterCell : "Wrong class type: " + b.getClass();
+        assert cell instanceof CounterCell : "Wrong class type: " + cell.getClass();
 
-        if (a.timestampOfLastDelete() < ((CounterCell) b).timestampOfLastDelete())
-            return b;
+        if (((CounterCell) this).timestampOfLastDelete() < ((CounterCell) cell).timestampOfLastDelete())
+            return cell;
 
-        CounterContext.Relationship rel = CounterCell.contextManager.diff(b.value(), a.value());
-        return (rel == CounterContext.Relationship.GREATER_THAN || rel == CounterContext.Relationship.DISJOINT) ? b : null;
+        CounterContext.Relationship rel = CounterCell.contextManager.diff(cell.value(), value());
+        return (rel == CounterContext.Relationship.GREATER_THAN || rel == CounterContext.Relationship.DISJOINT) ? cell : null;
     }
 
     /** This is temporary until we start creating Cells of the different type (buffer vs. native) */
-    public static Cell reconcile(CounterCell a, Cell b)
+    public Cell reconcileCounter(Cell cell)
     {
-        assert (b instanceof CounterCell) || (b instanceof DeletedCell) : "Wrong class type: " + b.getClass();
-
-        // live + tombstone: track last tombstone
-        if (!b.isLive()) // cannot be an expired cell, so the current time is irrelevant
-        {
-            // live < tombstone
-            if (a.timestamp() < b.timestamp())
-                return b;
+        assert this instanceof CounterCell : "Wrong class type: " + getClass();
 
-            // live last delete >= tombstone
-            if (a.timestampOfLastDelete() >= b.timestamp())
-                return a;
-
-            // live last delete < tombstone
-            return new BufferCounterCell(a.name(), a.value(), a.timestamp(), b.timestamp());
-        }
+        // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
+        if (cell instanceof DeletedCell)
+            return cell;
 
-        assert b instanceof CounterCell : "Wrong class type: " + b.getClass();
+        assert (cell instanceof CounterCell) : "Wrong class type: " + cell.getClass();
 
         // live < live last delete
-        if (a.timestamp() < ((CounterCell) b).timestampOfLastDelete())
-            return b;
+        if (timestamp() < ((CounterCell) cell).timestampOfLastDelete())
+            return cell;
+
+        long timestampOfLastDelete = ((CounterCell) this).timestampOfLastDelete();
 
         // live last delete > live
-        if (a.timestampOfLastDelete() > b.timestamp())
-            return a;
+        if (timestampOfLastDelete > cell.timestamp())
+            return this;
 
         // live + live. return one of the cells if its context is a superset of the other's, or merge them otherwise
-        ByteBuffer context = CounterCell.contextManager.merge(a.value(), b.value());
-        if (context == a.value() && a.timestamp() >= b.timestamp() && a.timestampOfLastDelete() >= ((CounterCell) b).timestampOfLastDelete())
-            return a;
-        else if (context == b.value() && b.timestamp() >= a.timestamp() && ((CounterCell) b).timestampOfLastDelete() >= a.timestampOfLastDelete())
-            return b;
-        else // merge clocks and timsestamps.
-            return new BufferCounterCell(a.name(),
+        ByteBuffer context = CounterCell.contextManager.merge(value(), cell.value());
+        if (context == value() && timestamp() >= cell.timestamp() && timestampOfLastDelete >= ((CounterCell) cell).timestampOfLastDelete())
+            return this;
+        else if (context == cell.value() && cell.timestamp() >= timestamp() && ((CounterCell) cell).timestampOfLastDelete() >= timestampOfLastDelete)
+            return cell;
+        else // merge clocks and timestamps.
+            return new BufferCounterCell(name(),
                                          context,
-                                         Math.max(a.timestamp(), b.timestamp()),
-                                         Math.max(a.timestampOfLastDelete(), ((CounterCell) b).timestampOfLastDelete()));
+                                         Math.max(timestamp(), cell.timestamp()),
+                                         Math.max(timestampOfLastDelete, ((CounterCell) cell).timestampOfLastDelete()));
     }
 
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
index f5624d2..8a2d43e 100644
--- a/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
+++ b/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
@@ -207,7 +207,7 @@ public class ArrayBackedSortedColumns extends ColumnFamily
       */
     public void maybeAppendColumn(Cell cell, DeletionInfo.InOrderTester tester, int gcBefore)
     {
-        if (cell.getLocalDeletionTime() >= gcBefore && !tester.isDeleted(cell.name(), cell.timestamp()))
+        if (cell.getLocalDeletionTime() >= gcBefore && !tester.isDeleted(cell))
         {
             internalAdd(cell);
             sortedSize++;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/BufferCounterCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCounterCell.java b/src/java/org/apache/cassandra/db/BufferCounterCell.java
index a70e274..bdd97a7 100644
--- a/src/java/org/apache/cassandra/db/BufferCounterCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCounterCell.java
@@ -91,7 +91,7 @@ public class BufferCounterCell extends BufferCell implements CounterCell
     @Override
     public Cell diff(Cell cell)
     {
-        return diff(this, cell);
+        return diffCounter(cell);
     }
 
     /*
@@ -115,7 +115,7 @@ public class BufferCounterCell extends BufferCell implements CounterCell
     @Override
     public Cell reconcile(Cell cell)
     {
-        return reconcile(this, cell);
+        return reconcileCounter(cell);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java b/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
index 24c9fad..de62b17 100644
--- a/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
+++ b/src/java/org/apache/cassandra/db/BufferCounterUpdateCell.java
@@ -54,17 +54,14 @@ public class BufferCounterUpdateCell extends BufferCell implements CounterUpdate
     @Override
     public Cell reconcile(Cell cell)
     {
-        // The only time this could happen is if a batchAdd ships two
-        // increment for the same cell. Hence we simply sums the delta and the timestamps.
+        // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
+        if (cell instanceof DeletedCell)
+            return cell;
 
-        // tombstones take precedence
-        if (!cell.isLive()) // can't be an expired cell, so the current time is irrelevant
-            return timestamp > cell.timestamp() ? this : cell;
-
-        // neither is tombstoned
         assert cell instanceof CounterUpdateCell : "Wrong class type.";
-        CounterUpdateCell c = (CounterUpdateCell) cell;
-        return new BufferCounterUpdateCell(name, delta() + c.delta(), timestamp + c.timestamp());
+
+        // The only time this could happen is if a batch ships two increments for the same cell. Hence we simply sum the deltas.
+        return new BufferCounterUpdateCell(name, delta() + ((CounterUpdateCell) cell).delta(), Math.max(timestamp, cell.timestamp()));
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/ColumnFamily.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamily.java b/src/java/org/apache/cassandra/db/ColumnFamily.java
index 38e1591..e00c27f 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamily.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamily.java
@@ -123,7 +123,7 @@ public abstract class ColumnFamily implements Iterable<Cell>, IRowCacheEntry
 
     public void addCounter(CellName name, long value)
     {
-        addColumn(new BufferCounterUpdateCell(name, value, System.currentTimeMillis()));
+        addColumn(new BufferCounterUpdateCell(name, value, FBUtilities.timestampMicros()));
     }
 
     public void addTombstone(CellName name, ByteBuffer localDeletionTime, long timestamp)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/ColumnIndex.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnIndex.java b/src/java/org/apache/cassandra/db/ColumnIndex.java
index ba03f51..58cd564 100644
--- a/src/java/org/apache/cassandra/db/ColumnIndex.java
+++ b/src/java/org/apache/cassandra/db/ColumnIndex.java
@@ -136,7 +136,7 @@ public class ColumnIndex
                     tombstone = rangeIter.hasNext() ? rangeIter.next() : null;
                 }
 
-                // We can skip any cell if it's shadowed by a tombstone already.  This is a more
+                // We can skip any cell if it's shadowed by a tombstone already. This is a more
                 // general case than was handled by CASSANDRA-2589.
                 if (!tester.isDeleted(c))
                     add(c);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/DeletionInfo.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/DeletionInfo.java b/src/java/org/apache/cassandra/db/DeletionInfo.java
index b8988ec..193f8b1 100644
--- a/src/java/org/apache/cassandra/db/DeletionInfo.java
+++ b/src/java/org/apache/cassandra/db/DeletionInfo.java
@@ -119,9 +119,7 @@ public class DeletionInfo implements IMeasurableMemory
      */
     public boolean isLive()
     {
-        return topLevel.markedForDeleteAt == Long.MIN_VALUE
-            && topLevel.localDeletionTime == Integer.MAX_VALUE
-            && (ranges == null || ranges.isEmpty());
+        return topLevel.isLive() && (ranges == null || ranges.isEmpty());
     }
 
     /**
@@ -132,20 +130,19 @@ public class DeletionInfo implements IMeasurableMemory
      */
     public boolean isDeleted(Cell cell)
     {
-        return isDeleted(cell.name(), cell.timestamp());
-    }
-
-    public boolean isDeleted(Composite name, long timestamp)
-    {
         // We do rely on this test: if topLevel.markedForDeleteAt is MIN_VALUE, we should not
         // consider the column deleted even if timestamp=MIN_VALUE, otherwise this break QueryFilter.isRelevant
         if (isLive())
             return false;
 
-        if (timestamp <= topLevel.markedForDeleteAt)
+        if (cell.timestamp() <= topLevel.markedForDeleteAt)
             return true;
 
-        return ranges != null && ranges.isDeleted(name, timestamp);
+        // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
+        if (!topLevel.isLive() && cell instanceof CounterCell)
+            return true;
+
+        return ranges != null && ranges.isDeleted(cell);
     }
 
     /**
@@ -301,11 +298,6 @@ public class DeletionInfo implements IMeasurableMemory
         return ranges == null ? Iterators.<RangeTombstone>emptyIterator() : ranges.iterator(start, finish);
     }
 
-    public DeletionTime deletionTimeFor(Composite name)
-    {
-        return ranges == null ? null : ranges.searchDeletionTime(name);
-    }
-
     public RangeTombstone rangeCovering(Composite name)
     {
         return ranges == null ? null : ranges.search(name);
@@ -454,12 +446,11 @@ public class DeletionInfo implements IMeasurableMemory
 
         public boolean isDeleted(Cell cell)
         {
-            return isDeleted(cell.name(), cell.timestamp());
-        }
+            if (cell.timestamp() <= topLevel.markedForDeleteAt)
+                return true;
 
-        public boolean isDeleted(Composite name, long timestamp)
-        {
-            if (timestamp <= topLevel.markedForDeleteAt)
+            // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
+            if (!topLevel.isLive() && cell instanceof CounterCell)
                 return true;
 
             /*
@@ -467,13 +458,13 @@ public class DeletionInfo implements IMeasurableMemory
              * is always in forward sorted order.
              */
             if (reversed)
-                 return DeletionInfo.this.isDeleted(name, timestamp);
+                 return DeletionInfo.this.isDeleted(cell);
 
             // Maybe create the tester if we hadn't yet and we now have some ranges (see above).
             if (tester == null && ranges != null)
                 tester = ranges.inOrderTester();
 
-            return tester != null && tester.isDeleted(name, timestamp);
+            return tester != null && tester.isDeleted(cell);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/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 f08196d..0e5f13f 100644
--- a/src/java/org/apache/cassandra/db/DeletionTime.java
+++ b/src/java/org/apache/cassandra/db/DeletionTime.java
@@ -28,6 +28,7 @@ import org.apache.cassandra.io.ISerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
 import org.apache.cassandra.io.util.FileUtils;
 import org.apache.cassandra.utils.ObjectSizes;
+import org.codehaus.jackson.annotate.JsonIgnore;
 
 /**
  * A top-level (row) tombstone.
@@ -63,6 +64,15 @@ public class DeletionTime implements Comparable<DeletionTime>, IMeasurableMemory
         this.localDeletionTime = localDeletionTime;
     }
 
+    /**
+     * Returns whether this DeletionTime is live, that is deletes no columns.
+     */
+    @JsonIgnore
+    public boolean isLive()
+    {
+        return markedForDeleteAt == Long.MIN_VALUE && localDeletionTime == Integer.MAX_VALUE;
+    }
+
     @Override
     public boolean equals(Object o)
     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/NativeCounterCell.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/NativeCounterCell.java b/src/java/org/apache/cassandra/db/NativeCounterCell.java
index 374eb23..d0df17e 100644
--- a/src/java/org/apache/cassandra/db/NativeCounterCell.java
+++ b/src/java/org/apache/cassandra/db/NativeCounterCell.java
@@ -17,8 +17,6 @@
  */
 package org.apache.cassandra.db;
 
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
 import java.security.MessageDigest;
 
 import org.apache.cassandra.config.CFMetaData;
@@ -89,13 +87,13 @@ public class NativeCounterCell extends NativeCell implements CounterCell
     @Override
     public Cell diff(Cell cell)
     {
-        return diff(this, cell);
+        return diffCounter(cell);
     }
 
     @Override
     public Cell reconcile(Cell cell)
     {
-        return reconcile(this, cell);
+        return reconcileCounter(cell);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/RangeTombstoneList.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/RangeTombstoneList.java b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
index fd2e100..393ff3c 100644
--- a/src/java/org/apache/cassandra/db/RangeTombstoneList.java
+++ b/src/java/org/apache/cassandra/db/RangeTombstoneList.java
@@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.cache.IMeasurableMemory;
 import org.apache.cassandra.db.composites.CType;
+import org.apache.cassandra.db.composites.CellName;
 import org.apache.cassandra.db.composites.Composite;
 import org.apache.cassandra.io.IVersionedSerializer;
 import org.apache.cassandra.io.util.DataOutputPlus;
@@ -235,10 +236,11 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
      * Returns whether the given name/timestamp pair is deleted by one of the tombstone
      * of this RangeTombstoneList.
      */
-    public boolean isDeleted(Composite name, long timestamp)
+    public boolean isDeleted(Cell cell)
     {
-        int idx = searchInternal(name, 0);
-        return idx >= 0 && markedAts[idx] >= timestamp;
+        int idx = searchInternal(cell.name(), 0);
+        // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
+        return idx >= 0 && (cell instanceof CounterCell || markedAts[idx] >= cell.timestamp());
     }
 
     /**
@@ -833,31 +835,40 @@ public class RangeTombstoneList implements Iterable<RangeTombstone>, IMeasurable
     {
         private int idx;
 
-        public boolean isDeleted(Composite name, long timestamp)
+        public boolean isDeleted(Cell cell)
         {
+            CellName name = cell.name();
+            long timestamp = cell.timestamp();
+
             while (idx < size)
             {
                 int cmp = comparator.compare(name, starts[idx]);
-                if (cmp == 0)
+
+                if (cmp < 0)
+                {
+                    return false;
+                }
+                else if (cmp == 0)
                 {
+                    // No matter what the counter cell's timestamp is, a tombstone always takes precedence. See CASSANDRA-7346.
+                    if (cell instanceof CounterCell)
+                        return true;
+
                     // As for searchInternal, we need to check the previous end
                     if (idx > 0 && comparator.compare(name, ends[idx-1]) == 0 && markedAts[idx-1] > markedAts[idx])
                         return markedAts[idx-1] >= timestamp;
                     else
                         return markedAts[idx] >= timestamp;
                 }
-                else if (cmp < 0)
-                {
-                    return false;
-                }
                 else
                 {
                     if (comparator.compare(name, ends[idx]) <= 0)
-                        return markedAts[idx] >= timestamp;
+                        return markedAts[idx] >= timestamp || cell instanceof CounterCell;
                     else
                         idx++;
                 }
             }
+
             return false;
         }
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
index b8a9563..86cfc40 100644
--- a/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
+++ b/src/java/org/apache/cassandra/db/filter/ColumnCounter.java
@@ -47,7 +47,7 @@ public class ColumnCounter
 
     protected static boolean isLive(Cell cell, DeletionInfo.InOrderTester tester, long timestamp)
     {
-        return cell.isLive(timestamp) && (!tester.isDeleted(cell));
+        return cell.isLive(timestamp) && !tester.isDeleted(cell);
     }
 
     public int live()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/Util.java b/test/unit/org/apache/cassandra/Util.java
index d2fe949..88031a4 100644
--- a/test/unit/org/apache/cassandra/Util.java
+++ b/test/unit/org/apache/cassandra/Util.java
@@ -116,11 +116,6 @@ public class Util
         return new BufferExpiringCell(cellname(name), ByteBufferUtil.bytes(value), timestamp, ttl);
     }
 
-    public static Cell counterColumn(String name, long value, long timestamp)
-    {
-        return new BufferCounterUpdateCell(cellname(name), value, timestamp);
-    }
-
     public static Token token(String key)
     {
         return StorageService.getPartitioner().getToken(ByteBufferUtil.bytes(key));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
index 2141020..69a851e 100644
--- a/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
+++ b/test/unit/org/apache/cassandra/db/ColumnFamilyTest.java
@@ -28,11 +28,16 @@ import com.google.common.collect.Iterables;
 import org.junit.Test;
 
 import org.apache.cassandra.SchemaLoader;
-import org.apache.cassandra.Util;
+import org.apache.cassandra.db.composites.CellName;
+import org.apache.cassandra.db.context.CounterContext;
 import org.apache.cassandra.io.sstable.ColumnStats;
 import org.apache.cassandra.io.util.DataOutputBuffer;
 import org.apache.cassandra.net.MessagingService;
 import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.CounterId;
+import org.apache.cassandra.utils.FBUtilities;
+
+import static junit.framework.Assert.assertTrue;
 
 import static org.apache.cassandra.Util.column;
 import static org.apache.cassandra.Util.cellname;
@@ -207,16 +212,44 @@ public class ColumnFamilyTest extends SchemaLoader
         ColumnStats stats = cf.getColumnStats();
         assertEquals(timestamp, stats.maxTimestamp);
 
-        cf.delete(new RangeTombstone(Util.cellname("col2"), Util.cellname("col21"), timestamp, localDeletionTime));
+        cf.delete(new RangeTombstone(cellname("col2"), cellname("col21"), timestamp, localDeletionTime));
 
         stats = cf.getColumnStats();
         assertEquals(ByteBufferUtil.bytes("col2"), stats.minColumnNames.get(0));
         assertEquals(ByteBufferUtil.bytes("col21"), stats.maxColumnNames.get(0));
 
-        cf.delete(new RangeTombstone(Util.cellname("col6"), Util.cellname("col61"), timestamp, localDeletionTime));
+        cf.delete(new RangeTombstone(cellname("col6"), cellname("col61"), timestamp, localDeletionTime));
         stats = cf.getColumnStats();
 
         assertEquals(ByteBufferUtil.bytes("col2"), stats.minColumnNames.get(0));
         assertEquals(ByteBufferUtil.bytes("col61"), stats.maxColumnNames.get(0));
     }
+
+    @Test
+    public void testCounterDeletion()
+    {
+        long timestamp = FBUtilities.timestampMicros();
+        CellName name = cellname("counter1");
+
+        BufferCounterCell counter = new BufferCounterCell(name,
+                                                          CounterContext.instance().createGlobal(CounterId.fromInt(1), 1, 1),
+                                                          timestamp);
+        BufferDeletedCell tombstone = new BufferDeletedCell(name, (int) (System.currentTimeMillis() / 1000), 0L);
+
+        // check that the tombstone won the reconcile despite the counter cell having a higher timestamp
+        assertTrue(counter.reconcile(tombstone) == tombstone);
+
+        // check that a range tombstone overrides the counter cell, even with a lower timestamp than the counter
+        ColumnFamily cf0 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        cf0.addColumn(counter);
+        cf0.delete(new RangeTombstone(cellname("counter0"), cellname("counter2"), 0L, (int) (System.currentTimeMillis() / 1000)));
+        assertTrue(cf0.deletionInfo().isDeleted(counter));
+        assertTrue(cf0.deletionInfo().inOrderTester(false).isDeleted(counter));
+
+        // check that a top-level deletion info overrides the counter cell, even with a lower timestamp than the counter
+        ColumnFamily cf1 = ArrayBackedSortedColumns.factory.create("Keyspace1", "Counter1");
+        cf1.addColumn(counter);
+        cf1.delete(new DeletionInfo(0L, (int) (System.currentTimeMillis() / 1000)));
+        assertTrue(cf1.deletionInfo().isDeleted(counter));
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/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 645c8a7..21826d2 100644
--- a/test/unit/org/apache/cassandra/db/CounterCellTest.java
+++ b/test/unit/org/apache/cassandra/db/CounterCellTest.java
@@ -103,23 +103,19 @@ public class CounterCellTest extends SchemaLoader
         left  = new BufferDeletedCell(cellname("x"), 1, 1L);
         right = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
 
-        assert left.reconcile(right) == right;
+        assert left.reconcile(right) == left;
 
         // tombstone == live last delete
         left  = new BufferDeletedCell(cellname("x"), 1, 2L);
         right = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
 
-        assert left.reconcile(right) == right;
+        assert left.reconcile(right) == left;
 
         // tombstone > live last delete
         left  = new BufferDeletedCell(cellname("x"), 1, 4L);
         right = BufferCounterCell.createLocal(cellname("x"), 0L, 9L, 1L);
 
-        reconciled = left.reconcile(right);
-        assert reconciled.name() == right.name();
-        assert reconciled.value() == right.value();
-        assert reconciled.timestamp() == right.timestamp();
-        assert ((CounterCell)reconciled).timestampOfLastDelete() == left.timestamp();
+        assert left.reconcile(right) == left;
 
         // live < tombstone
         left  = BufferCounterCell.createLocal(cellname("x"), 0L, 1L, Long.MIN_VALUE);
@@ -131,23 +127,19 @@ public class CounterCellTest extends SchemaLoader
         left  = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
         right = new BufferDeletedCell(cellname("x"), 1, 1L);
 
-        assert left.reconcile(right) == left;
+        assert left.reconcile(right) == right;
 
         // live last delete == tombstone
         left  = BufferCounterCell.createLocal(cellname("x"), 0L, 4L, 2L);
         right = new BufferDeletedCell(cellname("x"), 1, 2L);
 
-        assert left.reconcile(right) == left;
+        assert left.reconcile(right) == right;
 
         // live last delete < tombstone
         left  = BufferCounterCell.createLocal(cellname("x"), 0L, 9L, 1L);
         right = new BufferDeletedCell(cellname("x"), 1, 4L);
 
-        reconciled = left.reconcile(right);
-        assert reconciled.name() == left.name();
-        assert reconciled.value() == left.value();
-        assert reconciled.timestamp() == left.timestamp();
-        assert ((CounterCell)reconciled).timestampOfLastDelete() == right.timestamp();
+        assert left.reconcile(right) == right;
 
         // live < live last delete
         left  = new BufferCounterCell(cellname("x"), cc.createRemote(CounterId.fromInt(1), 2L, 3L), 1L, Long.MIN_VALUE);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/6858dd3e/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
index 0c5d49c..2a67d42 100644
--- a/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
+++ b/test/unit/org/apache/cassandra/utils/EncodedStreamsTest.java
@@ -106,8 +106,8 @@ public class EncodedStreamsTest extends SchemaLoader
     private ColumnFamily createCounterCF()
     {
         ColumnFamily cf = ArrayBackedSortedColumns.factory.create(keyspaceName, counterCFName);
-        cf.addColumn(counterColumn("vijay", 1L, 1));
-        cf.addColumn(counterColumn("wants", 1000000, 1));
+        cf.addCounter(cellname("vijay"), 1);
+        cf.addCounter(cellname("wants"), 1000000);
         return cf;
     }
 
@@ -128,14 +128,16 @@ public class EncodedStreamsTest extends SchemaLoader
     @Test
     public void testCounterCFSerialization() throws IOException
     {
+        ColumnFamily counterCF = createCounterCF();
+
         ByteArrayOutputStream byteArrayOStream1 = new ByteArrayOutputStream();
         EncodedDataOutputStream odos = new EncodedDataOutputStream(byteArrayOStream1);
-        ColumnFamily.serializer.serialize(createCounterCF(), odos, version);
+        ColumnFamily.serializer.serialize(counterCF, odos, version);
 
         ByteArrayInputStream byteArrayIStream1 = new ByteArrayInputStream(byteArrayOStream1.toByteArray());
         EncodedDataInputStream odis = new EncodedDataInputStream(new DataInputStream(byteArrayIStream1));
         ColumnFamily cf = ColumnFamily.serializer.deserialize(odis, version);
-        Assert.assertEquals(cf, createCounterCF());
+        Assert.assertEquals(cf, counterCF);
         Assert.assertEquals(byteArrayOStream1.size(), (int) ColumnFamily.serializer.serializedSize(cf, TypeSizes.VINT, version));
     }
 }


[2/2] git commit: Merge branch 'cassandra-2.1' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-2.1' into trunk


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

Branch: refs/heads/trunk
Commit: 23c2fa8c89ce67ebdb8f211a4c1dd8e2148bc785
Parents: 295fe5d 6858dd3
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Thu Jun 19 11:59:19 2014 -0700
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Thu Jun 19 11:59:19 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../org/apache/cassandra/db/AbstractCell.java   | 69 +++++++++-----------
 .../cassandra/db/ArrayBackedSortedColumns.java  |  2 +-
 .../apache/cassandra/db/BufferCounterCell.java  |  4 +-
 .../cassandra/db/BufferCounterUpdateCell.java   | 15 ++---
 .../org/apache/cassandra/db/ColumnFamily.java   |  2 +-
 .../org/apache/cassandra/db/ColumnIndex.java    |  2 +-
 .../org/apache/cassandra/db/DeletionInfo.java   | 35 ++++------
 .../org/apache/cassandra/db/DeletionTime.java   | 10 +++
 .../apache/cassandra/db/NativeCounterCell.java  |  4 +-
 .../apache/cassandra/db/RangeTombstoneList.java | 31 ++++++---
 .../cassandra/db/filter/ColumnCounter.java      |  2 +-
 test/unit/org/apache/cassandra/Util.java        |  5 --
 .../apache/cassandra/db/ColumnFamilyTest.java   | 39 ++++++++++-
 .../apache/cassandra/db/CounterCellTest.java    | 20 ++----
 .../cassandra/utils/EncodedStreamsTest.java     | 10 +--
 16 files changed, 139 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/23c2fa8c/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index a837d28,a2205ff..ec1dc9d
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,17 -1,6 +1,19 @@@
 +3.0
 + * Move sstable RandomAccessReader to nio2, which allows using the
 +   FILE_SHARE_DELETE flag on Windows (CASSANDRA-4050)
 + * Remove CQL2 (CASSANDRA-5918)
 + * Add Thrift get_multi_slice call (CASSANDRA-6757)
 + * Optimize fetching multiple cells by name (CASSANDRA-6933)
 + * Allow compilation in java 8 (CASSANDRA-7208)
 + * Make incremental repair default (CASSANDRA-7250)
 + * Enable code coverage thru JaCoCo (CASSANDRA-7226)
 + * Switch external naming of 'column families' to 'tables' (CASSANDRA-4369) 
 + 
 +
 +
  2.1.0
+  * Modify reconcile logic to always pick a tombstone over a counter cell
+    (CASSANDRA-7346)
   * Avoid incremental compaction on Windows (CASSANDRA-7365)
   * Fix exception when querying a composite-keyed table with a collection index
     (CASSANDRA-7372)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/23c2fa8c/src/java/org/apache/cassandra/db/ArrayBackedSortedColumns.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/23c2fa8c/src/java/org/apache/cassandra/db/ColumnFamily.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/23c2fa8c/test/unit/org/apache/cassandra/Util.java
----------------------------------------------------------------------