Skip to content

Commit

Permalink
Merge branch 'cassandra-3.0' into cassandra-3.11
Browse files Browse the repository at this point in the history
  • Loading branch information
krummas committed Oct 30, 2019
2 parents fb50d82 + 767a68c commit 2d90e3c
Show file tree
Hide file tree
Showing 18 changed files with 144 additions and 15 deletions.
5 changes: 4 additions & 1 deletion CHANGES.txt
Original file line number Diff line number Diff line change
@@ -1,7 +1,10 @@
3.11.6
Merged from 2.2
Merged from 3.0:
* Ensure legacy rows have primary key livenessinfo when they contain illegal cells (CASSANDRA-15365)
Merged from 2.2:
* In-JVM DTest: Set correct internode message version for upgrade test (CASSANDRA-15371)


3.11.5
* Fix SASI non-literal string comparisons (range operators) (CASSANDRA-15169)
* Make sure user defined compaction transactions are always closed (CASSANDRA-15123)
Expand Down
56 changes: 48 additions & 8 deletions src/java/org/apache/cassandra/db/LegacyLayout.java
Original file line number Diff line number Diff line change
Expand Up @@ -1292,6 +1292,22 @@ public static class CellGrouper
private LegacyRangeTombstone rowDeletion;
private LegacyRangeTombstone collectionDeletion;

/**
* Used to track if we need to add pk liveness info (row marker) when removing invalid legacy cells.
*
* In 2.1 these invalid cells existed but were not queryable, in this case specifically because they
* represented values for clustering key columns that were written as data cells.
*
* However, the presence (or not) of such cells on an otherwise empty CQL row (or partition) would decide
* if an empty result row were returned for the CQL row (or partition). To maintain this behaviour we
* insert a row marker containing the liveness info of these invalid cells iff we have no other data
* on the row.
*
* See also CASSANDRA-15365
*/
private boolean hasValidCells = false;
private LivenessInfo invalidLivenessInfo = null;

public CellGrouper(CFMetaData metadata, SerializationHelper helper)
{
this(metadata, helper, false);
Expand All @@ -1318,6 +1334,8 @@ public void reset()
this.clustering = null;
this.rowDeletion = null;
this.collectionDeletion = null;
this.invalidLivenessInfo = null;
this.hasValidCells = false;
}

public boolean addAtom(LegacyAtom atom)
Expand All @@ -1327,7 +1345,7 @@ public boolean addAtom(LegacyAtom atom)
: addRangeTombstone(atom.asRangeTombstone());
}

public boolean addCell(LegacyCell cell)
private boolean addCell(LegacyCell cell)
{
if (clustering == null)
{
Expand Down Expand Up @@ -1361,21 +1379,38 @@ else if (metadata.isIndex())
builder.addRowDeletion(Row.Deletion.regular(new DeletionTime(cell.timestamp, cell.localDeletionTime)));
else
builder.addPrimaryKeyLivenessInfo(LivenessInfo.create(cell.timestamp, FAKE_TTL, cell.localDeletionTime));
hasValidCells = true;
}
else if (column.isPrimaryKeyColumn() && metadata.isCQLTable())
{
// SSTables generated offline and side-loaded may include invalid cells which have the column name
// of a primary key column. So that we don't fail when encountering these cells, we treat them the
// same way as 2.1 did, namely we include their clusterings in the new CQL row, but drop the invalid
// column part of the cell
noSpamLogger.warn("Illegal cell name for CQL3 table {}.{}. {} is defined as a primary key column",
metadata.ksName, metadata.cfName, column.name);

if (invalidLivenessInfo != null)
{
// when we have several invalid cells we follow the logic in LivenessInfo#supersedes when picking the PKLI to keep:
LivenessInfo newInvalidLiveness = LivenessInfo.create(cell.timestamp, cell.isTombstone() ? FAKE_TTL : cell.ttl, cell.localDeletionTime);
if (newInvalidLiveness.supersedes(invalidLivenessInfo))
invalidLivenessInfo = newInvalidLiveness;
}
else
{
invalidLivenessInfo = LivenessInfo.create(cell.timestamp, cell.isTombstone() ? FAKE_TTL : cell.ttl, cell.localDeletionTime);
}
return true;
}
else
{
if (collectionDeletion != null && collectionDeletion.start.collectionName.name.equals(column.name) && collectionDeletion.deletionTime.deletes(cell.timestamp))
return true;

if (column.isPrimaryKeyColumn() && metadata.isCQLTable())
{
noSpamLogger.warn("Illegal cell name for CQL3 table {}.{}. {} is defined as a primary key column",
metadata.ksName, metadata.cfName, column.name);
return true;
}

if (helper.includes(column))
{
hasValidCells = true;
CellPath path = null;
if (column.isComplex())
{
Expand Down Expand Up @@ -1424,6 +1459,7 @@ private boolean addRowTombstone(LegacyRangeTombstone tombstone)
{
builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
rowDeletion = tombstone;
hasValidCells = true;
}
return true;
}
Expand All @@ -1436,6 +1472,7 @@ private boolean addRowTombstone(LegacyRangeTombstone tombstone)
builder.newRow(clustering);
builder.addRowDeletion(Row.Deletion.regular(tombstone.deletionTime));
rowDeletion = tombstone;
hasValidCells = true;

return true;
}
Expand Down Expand Up @@ -1466,6 +1503,7 @@ else if (!clustering.equals(tombstone.start.getAsClustering(metadata)))
builder.addComplexDeletion(tombstone.start.collectionName, tombstone.deletionTime);
if (rowDeletion == null || tombstone.deletionTime.supersedes(rowDeletion.deletionTime))
collectionDeletion = tombstone;
hasValidCells = true;

return true;
}
Expand All @@ -1490,6 +1528,8 @@ private boolean addGenericRangeTombstone(LegacyRangeTombstone tombstone)

public Row getRow()
{
if (!hasValidCells && invalidLivenessInfo != null)
builder.addPrimaryKeyLivenessInfo(invalidLivenessInfo);
return builder.build();
}
}
Expand Down
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
2529627719
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
Digest.sha1
Data.db
Filter.db
Summary.db
Index.db
TOC.txt
CompressionInfo.db
Statistics.db
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
3340111295
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Original file line number Diff line number Diff line change
@@ -0,0 +1,8 @@
Data.db
TOC.txt
Filter.db
Summary.db
CompressionInfo.db
Statistics.db
Digest.sha1
Index.db
80 changes: 74 additions & 6 deletions test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.List;
import java.util.Random;

import com.google.common.collect.Iterables;
import org.junit.After;
import org.junit.Assert;
import org.junit.BeforeClass;
Expand All @@ -42,13 +43,17 @@
import org.apache.cassandra.cql3.UntypedResultSet;
import org.apache.cassandra.db.ColumnFamilyStore;
import org.apache.cassandra.db.Keyspace;
import org.apache.cassandra.db.LivenessInfo;
import org.apache.cassandra.db.SinglePartitionSliceCommandTest;
import org.apache.cassandra.db.compaction.Verifier;
import org.apache.cassandra.db.lifecycle.SSTableSet;
import org.apache.cassandra.db.marshal.BytesType;
import org.apache.cassandra.db.marshal.SetType;
import org.apache.cassandra.db.marshal.UTF8Type;
import org.apache.cassandra.db.rows.RangeTombstoneMarker;
import org.apache.cassandra.db.rows.Row;
import org.apache.cassandra.db.rows.Unfiltered;
import org.apache.cassandra.db.rows.UnfilteredRowIterator;
import org.apache.cassandra.dht.IPartitioner;
import org.apache.cassandra.dht.Range;
import org.apache.cassandra.dht.Token;
Expand All @@ -63,11 +68,11 @@
import org.apache.cassandra.streaming.StreamSession;
import org.apache.cassandra.utils.ByteBufferUtil;
import org.apache.cassandra.utils.FBUtilities;
import org.apache.cassandra.utils.Pair;

import static org.apache.cassandra.cql3.CQLTester.assertRows;
import static org.apache.cassandra.cql3.CQLTester.row;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertNotNull;

/**
* Tests backwards compatibility for SSTables
Expand Down Expand Up @@ -448,21 +453,84 @@ public void testReadingLegacyTablesWithIllegalCellNames() throws Exception {
* ["b:bb:pk","00000002",1555000750634000],
* ["b:bb:v1","bbb",1555000750634000]]}
* ]
* and an extra sstable with only the invalid cell name
* [
* {"key": "3",
* "cells": [["a:aa:pk","68656c6c6f30",1570466358949]]}
* ]
*
*/

QueryProcessor.executeInternal("CREATE TABLE legacy_tables.legacy_ka_with_illegal_cell_names (" +
String table = "legacy_ka_with_illegal_cell_names";
QueryProcessor.executeInternal("CREATE TABLE legacy_tables." + table + " (" +
" pk int," +
" c1 text," +
" c2 text," +
" v1 text," +
" PRIMARY KEY(pk, c1, c2))");
loadLegacyTable("legacy_%s_with_illegal_cell_names%s", "ka", "");
UntypedResultSet results =
QueryProcessor.executeOnceInternal(
String.format("SELECT * FROM legacy_tables.legacy_ka_with_illegal_cell_names"));
QueryProcessor.executeOnceInternal("SELECT * FROM legacy_tables."+table);

assertRows(results, row(1, "a", "aa", "aaa"), row(2, "b", "bb", "bbb"));
assertRows(results, row(1, "a", "aa", "aaa"), row(2, "b", "bb", "bbb"), row (3, "a", "aa", null));
Keyspace.open("legacy_tables").getColumnFamilyStore(table).forceMajorCompaction();
}

@Test
public void testReadingLegacyTablesWithIllegalCellNamesPKLI() throws Exception {
/**
*
* Makes sure we grab the correct PKLI when we have illegal columns
*
* sstable looks like this:
* [
* {"key": "3",
* "cells": [["a:aa:","",100],
* ["a:aa:pk","6d656570",200]]}
* ]
*/
/*
this generates the stable on 2.1:
CFMetaData metadata = CFMetaData.compile("create table legacy_tables.legacy_ka_with_illegal_cell_names_2 (pk int, c1 text, c2 text, v1 text, primary key (pk, c1, c2))", "legacy_tables");
try (SSTableSimpleUnsortedWriter writer = new SSTableSimpleUnsortedWriter(new File("/tmp/sstable21"),
metadata,
new ByteOrderedPartitioner(),
10))
{
writer.newRow(bytes(3));
writer.addColumn(new BufferCell(Util.cellname("a", "aa", ""), bytes(""), 100));
writer.addColumn(new BufferCell(Util.cellname("a", "aa", "pk"), bytes("meep"), 200));
}
*/
String table = "legacy_ka_with_illegal_cell_names_2";
QueryProcessor.executeInternal("CREATE TABLE legacy_tables." + table + " (" +
" pk int," +
" c1 text," +
" c2 text," +
" v1 text," +
" PRIMARY KEY(pk, c1, c2))");
loadLegacyTable("legacy_%s_with_illegal_cell_names_2%s", "ka", "");
ColumnFamilyStore cfs = Keyspace.open("legacy_tables").getColumnFamilyStore(table);
assertEquals(1, Iterables.size(cfs.getSSTables(SSTableSet.CANONICAL)));
cfs.forceMajorCompaction();
assertEquals(1, Iterables.size(cfs.getSSTables(SSTableSet.CANONICAL)));
SSTableReader sstable = Iterables.getFirst(cfs.getSSTables(SSTableSet.CANONICAL), null);
LivenessInfo livenessInfo = null;
try (ISSTableScanner scanner = sstable.getScanner())
{
while (scanner.hasNext())
{
try (UnfilteredRowIterator iter = scanner.next())
{
while (iter.hasNext())
{
Unfiltered uf = iter.next();
livenessInfo = ((Row)uf).primaryKeyLivenessInfo();
}
}
}
}
assertNotNull(livenessInfo);
assertEquals(100, livenessInfo.timestamp());
}

@Test
Expand Down

0 comments on commit 2d90e3c

Please sign in to comment.