You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by sl...@apache.org on 2020/05/27 15:19:32 UTC

[cassandra] branch cassandra-3.11 updated (3cda9d7 -> ebfd052)

This is an automated email from the ASF dual-hosted git repository.

slebresne pushed a change to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git.


    from 3cda9d7  Merge branch cassandra-3.0 into cassandra-3.11
     new 8358e19  Fix legacy handling of RangeTombstone with collection ones
     new c8a2834  Fix LegacyLayout handling of non-selected collection tombstones
     new ebfd052  Merge commit 'c8a2834606d683ba9945e9cc11bdb4207ce269d1' into cassandra-3.11

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 CHANGES.txt                                        |   1 +
 src/java/org/apache/cassandra/db/LegacyLayout.java | 105 +++++++++++++----
 .../cassandra/db/UnfilteredDeserializer.java       | 129 ++++++++++++++++-----
 .../upgrade/MixedModeRangeTombstoneTest.java       |  73 ++++++++++++
 .../org/apache/cassandra/db/LegacyLayoutTest.java  | 102 +++++++++++++---
 5 files changed, 340 insertions(+), 70 deletions(-)
 create mode 100644 test/distributed/org/apache/cassandra/distributed/upgrade/MixedModeRangeTombstoneTest.java


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


[cassandra] 01/01: Merge commit 'c8a2834606d683ba9945e9cc11bdb4207ce269d1' into cassandra-3.11

Posted by sl...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

slebresne pushed a commit to branch cassandra-3.11
in repository https://gitbox.apache.org/repos/asf/cassandra.git

commit ebfd05254f84000f71fa018650632d24d3761f07
Merge: 3cda9d7 c8a2834
Author: Sylvain Lebresne <le...@gmail.com>
AuthorDate: Wed May 27 17:12:44 2020 +0200

    Merge commit 'c8a2834606d683ba9945e9cc11bdb4207ce269d1' into cassandra-3.11

 CHANGES.txt                                        |   1 +
 src/java/org/apache/cassandra/db/LegacyLayout.java | 105 +++++++++++++----
 .../cassandra/db/UnfilteredDeserializer.java       | 129 ++++++++++++++++-----
 .../upgrade/MixedModeRangeTombstoneTest.java       |  73 ++++++++++++
 .../org/apache/cassandra/db/LegacyLayoutTest.java  | 102 +++++++++++++---
 5 files changed, 340 insertions(+), 70 deletions(-)

diff --cc CHANGES.txt
index 11515c4,46b3f56..a809016
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,7 -1,5 +1,8 @@@
 -3.0.21
 +3.11.7
 + * Fix CQL formatting of read command restrictions for slow query log (CASSANDRA-15503)
 + * Allow sstableloader to use SSL on the native port (CASSANDRA-14904)
 +Merged from 3.0:
+  * Fix duplicated row on 2.x upgrades when multi-rows range tombstones interact with collection ones (CASSANDRA-15805)
   * Rely on snapshotted session infos on StreamResultFuture.maybeComplete to avoid race conditions (CASSANDRA-15667)
   * EmptyType doesn't override writeValue so could attempt to write bytes when expected not to (CASSANDRA-15790)
   * Fix index queries on partition key columns when some partitions contains only static data (CASSANDRA-13666)
diff --cc src/java/org/apache/cassandra/db/LegacyLayout.java
index 4ec0c30,8492de5..b28c72a
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@@ -1891,9 -1934,9 +1936,9 @@@ public abstract class LegacyLayou
              if ((start.collectionName == null) != (stop.collectionName == null))
              {
                  if (start.collectionName == null)
-                     stop = new LegacyBound(stop.bound, stop.isStatic, null);
 -                    stop = new LegacyBound(Slice.Bound.inclusiveEndOf(stop.bound.values), stop.isStatic, null);
++                    stop = new LegacyBound(ClusteringBound.inclusiveEndOf(stop.bound.values), stop.isStatic, null);
                  else
-                     start = new LegacyBound(start.bound, start.isStatic, null);
 -                    start = new LegacyBound(Slice.Bound.inclusiveStartOf(start.bound.values), start.isStatic, null);
++                    start = new LegacyBound(ClusteringBound.inclusiveStartOf(start.bound.values), start.isStatic, null);
              }
              else if (!Objects.equals(start.collectionName, stop.collectionName))
              {
@@@ -1920,11 -1963,21 +1965,21 @@@
              return new LegacyRangeTombstone(newStart, stop, deletionTime);
          }
  
 -        public LegacyRangeTombstone withNewStart(Slice.Bound newStart)
++        public LegacyRangeTombstone withNewStart(ClusteringBound newStart)
+         {
+             return withNewStart(new LegacyBound(newStart, start.isStatic, null));
+         }
+ 
          public LegacyRangeTombstone withNewEnd(LegacyBound newStop)
          {
              return new LegacyRangeTombstone(start, newStop, deletionTime);
          }
  
 -        public LegacyRangeTombstone withNewEnd(Slice.Bound newEnd)
++        public LegacyRangeTombstone withNewEnd(ClusteringBound newEnd)
+         {
+             return withNewEnd(new LegacyBound(newEnd, stop.isStatic, null));
+         }
+ 
          public boolean isCell()
          {
              return false;
diff --cc src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index cdcde2e,2d270bc..262b333
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@@ -480,19 -480,9 +480,10 @@@ public abstract class UnfilteredDeseria
                  this.helper = helper;
                  this.grouper = new LegacyLayout.CellGrouper(metadata, helper);
                  this.tombstoneTracker = new TombstoneTracker(partitionDeletion);
-                 this.atoms = new AtomIterator(atomReader);
-             }
- 
-             private boolean isRow(LegacyLayout.LegacyAtom atom)
-             {
-                 if (atom.isCell())
-                     return true;
- 
-                 LegacyLayout.LegacyRangeTombstone tombstone = atom.asRangeTombstone();
-                 return tombstone.isCollectionTombstone() || tombstone.isRowDeletion(metadata);
+                 this.atoms = new AtomIterator(atomReader, metadata);
              }
  
 +
              public boolean hasNext()
              {
                  // Note that we loop on next == null because TombstoneTracker.openNew() could return null below or the atom might be shadowed.
@@@ -540,13 -530,57 +531,57 @@@
                                                   ? LegacyLayout.CellGrouper.staticGrouper(metadata, helper)
                                                   : this.grouper;
                  grouper.reset();
+                 // We know the first atom is not shadowed and is a "row" atom, so can be added blindly.
                  grouper.addAtom(first);
-                 // As long as atoms are part of the same row, consume them. Note that the call to addAtom() uses
-                 // atoms.peek() so that the atom is only consumed (by next) if it's part of the row (addAtom returns true)
-                 while (atoms.hasNext() && grouper.addAtom(atoms.peek()))
+ 
+                 // We're less sure about the next atoms. In particular, CellGrouper want to make sure we only pass it
+                 // "row" atoms (it's the only type it knows how to handle) so we should handle anything else.
+                 while (atoms.hasNext())
                  {
-                     atoms.next();
+                     // Peek, but don't consume the next atom just yet
+                     LegacyLayout.LegacyAtom atom = atoms.peek();
+                     // First, that atom may be shadowed in which case we can simply ignore it. Note that this handles
+                     // the case of repeated RT start marker after we've crossed an index boundary, which could well
+                     // appear in the middle of a row (CASSANDRA-14008).
+                     if (!tombstoneTracker.hasClosingMarkerBefore(atom) && tombstoneTracker.isShadowed(atom))
+                     {
+                         atoms.next(); // consume the atom since we only peeked it so far
+                         continue;
+                     }
+ 
+                     // Second, we should only pass "row" atoms to the cell grouper
+                     if (atom.isRowAtom(metadata))
+                     {
+                         if (!grouper.addAtom(atom))
+                             break; // done with the row; don't consume the atom
+                         atoms.next(); // the grouper "accepted" the atom, consume it since we only peeked above
+                     }
+                     else
+                     {
+                         LegacyLayout.LegacyRangeTombstone rt = (LegacyLayout.LegacyRangeTombstone) atom;
+                         // This means we have a non-row range tombstone. Unfortunately, that does not guarantee the
+                         // current row is finished (though it may), because due to the logic within LegacyRangeTombstone
+                         // constructor, we can get an out-of-order RT that includes on the current row (even if it is
+                         // already started) and extends past it.
+ 
+                         // So first, evacuate the easy case of the range tombstone simply starting after the current
+                         // row, in which case we're done with the current row (but don't consume the new RT yet so it
+                         // gets handled as any other non-row RT).
+                         if (grouper.startsAfterCurrentRow(rt))
+                             break;
+ 
+                         // Otherwise, we "split" the RT in 2: the part covering the current row, which is now an
+                         // inRowAtom and can be passed to the grouper, and the part after that, which we push back into
+                         // the iterator for later processing.
+                         Clustering currentRow = grouper.currentRowClustering();
+                         atoms.next(); // consume since we had only just peeked it so far and we're using it
 -                        atoms.pushOutOfOrder(rt.withNewStart(Slice.Bound.exclusiveStartOf(currentRow)));
++                        atoms.pushOutOfOrder(rt.withNewStart(ClusteringBound.exclusiveStartOf(currentRow)));
+                         // Note: in theory the withNewStart is a no-op here, but not taking any risk
 -                        grouper.addAtom(rt.withNewStart(Slice.Bound.inclusiveStartOf(currentRow))
 -                                          .withNewEnd(Slice.Bound.inclusiveEndOf(currentRow)));
++                        grouper.addAtom(rt.withNewStart(ClusteringBound.inclusiveStartOf(currentRow))
++                                          .withNewEnd(ClusteringBound.inclusiveEndOf(currentRow)));
+                     }
                  }
+ 
                  return grouper.getRow();
              }
  
diff --cc test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
index 1bc3af6,f0d2a02..65565e1
--- a/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
+++ b/test/unit/org/apache/cassandra/db/LegacyLayoutTest.java
@@@ -24,7 -24,10 +24,11 @@@ import java.nio.file.Files
  import java.nio.file.Path;
  import java.nio.file.Paths;
  
 +import org.junit.AfterClass;
+ import org.apache.cassandra.db.LegacyLayout.CellGrouper;
+ import org.apache.cassandra.db.LegacyLayout.LegacyBound;
+ import org.apache.cassandra.db.LegacyLayout.LegacyCell;
+ import org.apache.cassandra.db.LegacyLayout.LegacyRangeTombstone;
  import org.apache.cassandra.db.filter.ColumnFilter;
  import org.apache.cassandra.db.marshal.MapType;
  import org.apache.cassandra.db.marshal.UTF8Type;
@@@ -397,26 -386,89 +398,89 @@@ public class LegacyLayoutTes
          SerializationHelper helper = new SerializationHelper(cfm, MessagingService.VERSION_22, SerializationHelper.Flag.LOCAL, ColumnFilter.all(cfm));
          LegacyLayout.CellGrouper cg = new LegacyLayout.CellGrouper(cfm, helper);
  
-         ClusteringBound startBound = ClusteringBound.create(ClusteringPrefix.Kind.INCL_START_BOUND, new ByteBuffer[] {ByteBufferUtil.bytes(2)});
-         ClusteringBound endBound = ClusteringBound.create(ClusteringPrefix.Kind.EXCL_END_BOUND, new ByteBuffer[] {ByteBufferUtil.bytes(2)});
-         LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(startBound, false, cfm.getColumnDefinition(ByteBufferUtil.bytes("v")));
-         LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(endBound, false, cfm.getColumnDefinition(ByteBufferUtil.bytes("v")));
 -        Slice.Bound startBound = Slice.Bound.create(ClusteringPrefix.Kind.INCL_START_BOUND, new ByteBuffer[] {bytes(2)});
 -        Slice.Bound endBound = Slice.Bound.create(ClusteringPrefix.Kind.EXCL_END_BOUND, new ByteBuffer[] {bytes(2)});
++        ClusteringBound startBound = ClusteringBound.create(ClusteringPrefix.Kind.INCL_START_BOUND, new ByteBuffer[] {bytes(2)});
++        ClusteringBound endBound = ClusteringBound.create(ClusteringPrefix.Kind.EXCL_END_BOUND, new ByteBuffer[] {bytes(2)});
+         LegacyLayout.LegacyBound start = new LegacyLayout.LegacyBound(startBound, false, cfm.getColumnDefinition(bytes("v")));
+         LegacyLayout.LegacyBound end = new LegacyLayout.LegacyBound(endBound, false, cfm.getColumnDefinition(bytes("v")));
          LegacyLayout.LegacyRangeTombstone lrt = new LegacyLayout.LegacyRangeTombstone(start, end, new DeletionTime(2, 1588598040));
          assertTrue(cg.addAtom(lrt));
  
          // add a real cell
          LegacyLayout.LegacyCell cell = new LegacyLayout.LegacyCell(LegacyLayout.LegacyCell.Kind.REGULAR,
-                                                                    new LegacyLayout.LegacyCellName(Clustering.make(ByteBufferUtil.bytes(2)),
-                                                                                                    cfm.getColumnDefinition(ByteBufferUtil.bytes("v")),
-                                                                                                    ByteBufferUtil.bytes("g")),
-                                                                    ByteBufferUtil.bytes("v"), 3, Integer.MAX_VALUE, 0);
 -                                                                   new LegacyLayout.LegacyCellName(new Clustering(bytes(2)),
++                                                                   new LegacyLayout.LegacyCellName(Clustering.make(bytes(2)),
+                                                                                                    cfm.getColumnDefinition(bytes("v")),
+                                                                                                    bytes("g")),
+                                                                    bytes("v"), 3, Integer.MAX_VALUE, 0);
          assertTrue(cg.addAtom(cell));
  
          // add legacy range tombstone where collection name is null for the end bound (this gets translated to a row tombstone)
-         startBound = ClusteringBound.create(ClusteringPrefix.Kind.EXCL_START_BOUND, new ByteBuffer[] {ByteBufferUtil.bytes(2)});
-         endBound = ClusteringBound.create(ClusteringPrefix.Kind.EXCL_END_BOUND, new ByteBuffer[] {ByteBufferUtil.bytes(2)});
-         start = new LegacyLayout.LegacyBound(startBound, false, cfm.getColumnDefinition(ByteBufferUtil.bytes("v")));
 -        startBound = Slice.Bound.create(ClusteringPrefix.Kind.EXCL_START_BOUND, new ByteBuffer[] {bytes(2)});
 -        endBound = Slice.Bound.create(ClusteringPrefix.Kind.EXCL_END_BOUND, new ByteBuffer[] {bytes(2)});
++        startBound = ClusteringBound.create(ClusteringPrefix.Kind.EXCL_START_BOUND, new ByteBuffer[] {bytes(2)});
++        endBound = ClusteringBound.create(ClusteringPrefix.Kind.EXCL_END_BOUND, new ByteBuffer[] {bytes(2)});
+         start = new LegacyLayout.LegacyBound(startBound, false, cfm.getColumnDefinition(bytes("v")));
          end = new LegacyLayout.LegacyBound(endBound, false, null);
          assertTrue(cg.addAtom(new LegacyLayout.LegacyRangeTombstone(start, end, new DeletionTime(1, 1588598040))));
      }
+ 
+     private static LegacyCell cell(Clustering clustering, ColumnDefinition column, ByteBuffer value, long timestamp)
+     {
+         return new LegacyCell(LegacyCell.Kind.REGULAR,
+                               new LegacyLayout.LegacyCellName(clustering, column, null),
+                               value,
+                               timestamp,
+                               Cell.NO_DELETION_TIME,
+                               Cell.NO_TTL);
+     }
+ 
+     /**
+      * This tests that when {@link CellGrouper} gets a collection tombstone for
+      * a non-fetched collection, then that tombstone does not incorrectly stop the grouping of the current row, as
+      * was done before CASSANDRA-15805.
+      *
+      * <p>Please note that this rely on a query only _fetching_ some of the table columns, which in practice only
+      * happens for thrift queries, and thrift queries shouldn't mess up with CQL tables and collection tombstones,
+      * so this test is not of the utmost importance. Nonetheless, the pre-CASSANDRA-15805 behavior was incorrect and
+      * this ensure it is fixed.
+      */
+     @Test
+     public void testCellGrouperOnNonFecthedCollectionTombstone()
+     {
+         // CREATE TABLE %s (pk int, ck int, a text, b set<text>, c text, PRIMARY KEY (pk, ck))
+         CFMetaData cfm = CFMetaData.Builder.create("ks", "table")
+                                            .addPartitionKey("pk", Int32Type.instance)
+                                            .addClusteringColumn("ck", Int32Type.instance)
+                                            .addRegularColumn("a", UTF8Type.instance)
+                                            .addRegularColumn("b", SetType.getInstance(UTF8Type.instance, true))
+                                            .addRegularColumn("c", UTF8Type.instance)
+                                            .build();
+ 
+         // Creates a filter that _only_ fetches a and c, but not b.
+         ColumnFilter filter = ColumnFilter.selectionBuilder()
+                                           .add(cfm.getColumnDefinition(bytes("a")))
+                                           .add(cfm.getColumnDefinition(bytes("c")))
+                                           .build();
+         SerializationHelper helper = new SerializationHelper(cfm,
+                                                              MessagingService.VERSION_22,
+                                                              SerializationHelper.Flag.LOCAL,
+                                                              filter);
+         CellGrouper grouper = new CellGrouper(cfm, helper);
 -        Clustering clustering = new Clustering(bytes(1));
++        Clustering clustering = new BufferClustering(bytes(1));
+ 
+         // We add a cell for a, then a collection tombstone for b, and then a cell for c (for the same clustering).
+         // All those additions should return 'true' as all belong to the same row.
+         LegacyCell ca = cell(clustering, cfm.getColumnDefinition(bytes("a")), bytes("v1"), 1);
+         assertTrue(grouper.addAtom(ca));
+ 
 -        Slice.Bound startBound = Slice.Bound.inclusiveStartOf(bytes(1));
 -        Slice.Bound endBound = Slice.Bound.inclusiveEndOf(bytes(1));
++        ClusteringBound startBound = ClusteringBound.inclusiveStartOf(bytes(1));
++        ClusteringBound endBound = ClusteringBound.inclusiveEndOf(bytes(1));
+         ColumnDefinition bDef = cfm.getColumnDefinition(bytes("b"));
+         assert bDef != null;
+         LegacyBound start = new LegacyBound(startBound, false, bDef);
+         LegacyBound end = new LegacyBound(endBound, false, bDef);
+         LegacyRangeTombstone rtb = new LegacyRangeTombstone(start, end, new DeletionTime(1, 1588598040));
+         assertTrue(rtb.isCollectionTombstone()); // Ensure we're testing what we think
+         assertTrue(grouper.addAtom(rtb));
+ 
+         LegacyCell cc = cell(clustering, cfm.getColumnDefinition(bytes("c")), bytes("v2"), 1);
+         assertTrue(grouper.addAtom(cc));
+     }
  }


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