You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sa...@apache.org on 2018/12/06 16:16:58 UTC

[3/6] cassandra git commit: Differentiate between slices and RTs when decoding legacy bounds

Differentiate between slices and RTs when decoding legacy bounds

Patch by Sam Tunnicliffe; reviewed by Benedict Elliott Smith for CASSANDRA-14919


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

Branch: refs/heads/trunk
Commit: 11043610e38281f650f289a7f9286d306f1369e3
Parents: b82a42f
Author: Sam Tunnicliffe <sa...@beobal.com>
Authored: Fri Nov 30 13:49:56 2018 +0000
Committer: Sam Tunnicliffe <sa...@beobal.com>
Committed: Thu Dec 6 16:12:21 2018 +0000

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../org/apache/cassandra/db/LegacyLayout.java   | 60 +++++++++++++-------
 .../org/apache/cassandra/db/ReadCommand.java    |  6 +-
 .../cassandra/thrift/CassandraServer.java       | 10 ++--
 4 files changed, 49 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/11043610/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index e349674..4520989 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.18
+ * Differentiate between slices and RTs when decoding legacy bounds (CASSANDRA-14919)
  * CommitLogReplayer.handleReplayError should print stack traces (CASSANDRA-14589)
  * Netty epoll IOExceptions caused by unclean client disconnects being logged at INFO (CASSANDRA-14909)
  * Unfiltered.isEmpty conflicts with Row extends AbstractCollection.isEmpty (CASSANDRA-14588)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11043610/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 c80594c..9600355 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -201,7 +201,17 @@ public abstract class LegacyLayout
         return new LegacyCellName(def.isStatic() ? Clustering.STATIC_CLUSTERING : clustering, def, collectionElement);
     }
 
-    public static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, boolean isStart)
+    public static LegacyBound decodeSliceBound(CFMetaData metadata, ByteBuffer bound, boolean isStart)
+    {
+        return decodeBound(metadata, bound, isStart, false);
+    }
+
+    public static LegacyBound decodeTombstoneBound(CFMetaData metadata, ByteBuffer bound, boolean isStart)
+    {
+        return decodeBound(metadata, bound, isStart, true);
+    }
+
+    private static LegacyBound decodeBound(CFMetaData metadata, ByteBuffer bound, boolean isStart, boolean isDeletion)
     {
         if (!bound.hasRemaining())
             return isStart ? LegacyBound.BOTTOM : LegacyBound.TOP;
@@ -223,25 +233,35 @@ public abstract class LegacyLayout
         assert !isStatic ||
                 (components.size() >= clusteringSize
                         && all(components.subList(0, clusteringSize), ByteBufferUtil.EMPTY_BYTE_BUFFER::equals));
-        // There can be  more components than the clustering size only in the case this is the bound of a collection
-        // range tombstone. In which case, there is exactly one more component, and that component is the name of the
-        // collection being selected/deleted.
         ColumnDefinition collectionName = null;
         if (components.size() > clusteringSize)
         {
+            // For a deletion, there can be more components than the clustering size only in the case this is the
+            // bound of a collection range tombstone. In such a case, there is exactly one more component, and that
+            // component is the name of the collection being selected/deleted.
+            // If the bound is not part of a deletion, it is from slice query filter. In this scnario, the column name
+            // may be a valid, non-collection column or it may be an empty buffer, representing a row marker. In either
+            // case, this needn't be included in the returned bound, so we pop the last element from the components
+            // list but ensure that the collection name remains null.
+
             assert clusteringSize + 1 == components.size() && !metadata.isCompactTable();
-            // pop the collection name from the back of the list of clusterings
-            ByteBuffer collectionNameBytes = components.remove(clusteringSize);
-            collectionName = metadata.getColumnDefinition(collectionNameBytes);
-            if (collectionName == null || !collectionName.isComplex()) {
-                collectionName = metadata.getDroppedColumnDefinition(collectionNameBytes, isStatic);
-                // if no record of the column having ever existed is found, something is badly wrong
-                if (collectionName == null)
-                    throw new RuntimeException("Unknown collection column " + UTF8Type.instance.getString(collectionNameBytes) + " during deserialization");
-                // if we do have a record of dropping this column but it wasn't previously complex, use a fake
-                // column definition for safety (see the comment on the constant declaration for details)
-                if (!collectionName.isComplex())
-                    collectionName = INVALID_DROPPED_COMPLEX_SUBSTITUTE_COLUMN;
+            // pop the final element from the back of the list of clusterings
+            ByteBuffer columnNameBytes = components.remove(clusteringSize);
+            if (isDeletion)
+            {
+                collectionName = metadata.getColumnDefinition(columnNameBytes);
+                if (collectionName == null || !collectionName.isComplex())
+                {
+                    collectionName = metadata.getDroppedColumnDefinition(columnNameBytes, isStatic);
+                    // if no record of the column having ever existed is found, something is badly wrong
+                    if (collectionName == null)
+                        throw new RuntimeException("Unknown collection column " + UTF8Type.instance.getString(columnNameBytes) + " during deserialization");
+
+                    // if we do have a record of dropping this column but it wasn't previously complex, use a fake
+                    // column definition for safety (see the comment on the constant declaration for details)
+                    if (!collectionName.isComplex())
+                        collectionName = INVALID_DROPPED_COMPLEX_SUBSTITUTE_COLUMN;
+                }
             }
         }
 
@@ -1157,8 +1177,8 @@ public abstract class LegacyLayout
 
     public static LegacyRangeTombstone readLegacyRangeTombstoneBody(CFMetaData metadata, DataInputPlus in, ByteBuffer boundname) throws IOException
     {
-        LegacyBound min = decodeBound(metadata, boundname, true);
-        LegacyBound max = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
+        LegacyBound min = decodeTombstoneBound(metadata, boundname, true);
+        LegacyBound max = decodeTombstoneBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
         DeletionTime dt = DeletionTime.serializer.deserialize(in);
         return new LegacyRangeTombstone(min, max, dt);
     }
@@ -1903,8 +1923,8 @@ public abstract class LegacyLayout
             LegacyDeletionInfo delInfo = new LegacyDeletionInfo(new MutableDeletionInfo(topLevel));
             for (int i = 0; i < rangeCount; i++)
             {
-                LegacyBound start = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
-                LegacyBound end = decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
+                LegacyBound start = decodeTombstoneBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
+                LegacyBound end = decodeTombstoneBound(metadata, ByteBufferUtil.readWithShortLength(in), false);
                 int delTime =  in.readInt();
                 long markedAt = in.readLong();
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11043610/src/java/org/apache/cassandra/db/ReadCommand.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index 0135d1e..fd453ef 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -1090,7 +1090,7 @@ public abstract class ReadCommand implements ReadQuery
             int compositesToGroup = in.readInt();
 
             // command-level Composite "start" and "stop"
-            LegacyLayout.LegacyBound startBound = LegacyLayout.decodeBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
+            LegacyLayout.LegacyBound startBound = LegacyLayout.decodeSliceBound(metadata, ByteBufferUtil.readWithShortLength(in), true);
 
             ByteBufferUtil.readWithShortLength(in);  // the composite "stop", which isn't actually needed
 
@@ -1583,8 +1583,8 @@ public abstract class ReadCommand implements ReadQuery
             Slices.Builder slicesBuilder = new Slices.Builder(metadata.comparator);
             for (int i = 0; i < numSlices; i++)
             {
-                LegacyLayout.LegacyBound start = LegacyLayout.decodeBound(metadata, startBuffers[i], true);
-                LegacyLayout.LegacyBound finish = LegacyLayout.decodeBound(metadata, finishBuffers[i], false);
+                LegacyLayout.LegacyBound start = LegacyLayout.decodeSliceBound(metadata, startBuffers[i], true);
+                LegacyLayout.LegacyBound finish = LegacyLayout.decodeSliceBound(metadata, finishBuffers[i], false);
 
                 if (start.isStatic)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/11043610/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 256f651..163eb2d 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -371,7 +371,7 @@ public class CassandraServer implements Cassandra.Iface
         // Note that in thrift, the bounds are reversed if the query is reversed, but not internally.
         ByteBuffer start = range.reversed ? range.finish : range.start;
         ByteBuffer finish = range.reversed ? range.start : range.finish;
-        return Slices.with(metadata.comparator, Slice.make(LegacyLayout.decodeBound(metadata, start, true).bound, LegacyLayout.decodeBound(metadata, finish, false).bound));
+        return Slices.with(metadata.comparator, Slice.make(LegacyLayout.decodeSliceBound(metadata, start, true).bound, LegacyLayout.decodeSliceBound(metadata, finish, false).bound));
     }
 
     private ClusteringIndexFilter toInternalFilter(CFMetaData metadata, ColumnParent parent, SlicePredicate predicate)
@@ -1242,8 +1242,8 @@ public class CassandraServer implements Cassandra.Iface
         {
             if (del.super_column == null)
             {
-                LegacyLayout.LegacyBound start = LegacyLayout.decodeBound(cfm, del.predicate.getSlice_range().start, true);
-                LegacyLayout.LegacyBound end = LegacyLayout.decodeBound(cfm, del.predicate.getSlice_range().finish, false);
+                LegacyLayout.LegacyBound start = LegacyLayout.decodeTombstoneBound(cfm, del.predicate.getSlice_range().start, true);
+                LegacyLayout.LegacyBound end = LegacyLayout.decodeTombstoneBound(cfm, del.predicate.getSlice_range().finish, false);
                 delInfo.add(cfm, new LegacyLayout.LegacyRangeTombstone(start, end, new DeletionTime(del.timestamp, nowInSec)));
             }
             else
@@ -2426,8 +2426,8 @@ public class CassandraServer implements Cassandra.Iface
             for (int i = 0 ; i < request.getColumn_slices().size() ; i++)
             {
                 fixOptionalSliceParameters(request.getColumn_slices().get(i));
-                Slice.Bound start = LegacyLayout.decodeBound(metadata, request.getColumn_slices().get(i).start, true).bound;
-                Slice.Bound finish = LegacyLayout.decodeBound(metadata, request.getColumn_slices().get(i).finish, false).bound;
+                Slice.Bound start = LegacyLayout.decodeSliceBound(metadata, request.getColumn_slices().get(i).start, true).bound;
+                Slice.Bound finish = LegacyLayout.decodeSliceBound(metadata, request.getColumn_slices().get(i).finish, false).bound;
 
                 int compare = metadata.comparator.compare(start, finish);
                 if (!request.reversed && compare > 0)


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