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 2019/06/27 15:30:29 UTC

[cassandra] branch cassandra-3.0 updated: Filter illegal legacy cells when collating rows

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

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


The following commit(s) were added to refs/heads/cassandra-3.0 by this push:
     new f4b6e1d  Filter illegal legacy cells when collating rows
f4b6e1d is described below

commit f4b6e1d51f683e0c77c6ff7f199373052b082b9e
Author: Sam Tunnicliffe <sa...@beobal.com>
AuthorDate: Thu Jun 13 14:44:26 2019 +0100

    Filter illegal legacy cells when collating rows
    
    Alternative solution for CASSANDRA-15086, which allows the illegal cells to
    be read from disk and deserialized as normal so as not to interfere with
    tracking bytes read during reverse iteration of indexed partitions.
    
    Patch by Sam Tunnicliffe; reviewed by Marcus Eriksson for CASSANDRA-15178
---
 CHANGES.txt                                        |   1 +
 .../cassandra/db/IllegalLegacyColumnException.java |  41 --------
 src/java/org/apache/cassandra/db/LegacyLayout.java |  76 +++++---------
 src/java/org/apache/cassandra/db/ReadCommand.java  |   2 +-
 .../cassandra/db/UnfilteredDeserializer.java       |   7 --
 .../cassandra/db/UnknownColumnException.java       |  16 ++-
 .../apache/cassandra/thrift/CassandraServer.java   |  34 +++----
 .../apache/cassandra/thrift/ThriftValidation.java  |  10 +-
 ..._ka_with_illegal_cell_names_indexed-ka-1-CRC.db | Bin 0 -> 8 bytes
 ...ka_with_illegal_cell_names_indexed-ka-1-Data.db | Bin 0 -> 6487 bytes
 ...ith_illegal_cell_names_indexed-ka-1-Digest.sha1 |   1 +
 ..._with_illegal_cell_names_indexed-ka-1-Filter.db | Bin 0 -> 16 bytes
 ...a_with_illegal_cell_names_indexed-ka-1-Index.db | Bin 0 -> 453 bytes
 ...h_illegal_cell_names_indexed-ka-1-Statistics.db | Bin 0 -> 4472 bytes
 ...ka_with_illegal_cell_names_indexed-ka-1-TOC.txt |   8 ++
 .../cql3/validation/ThriftIllegalColumnsTest.java  | 100 +++++++++++++++++++
 .../apache/cassandra/db/LegacyCellNameTest.java    |  18 +++-
 .../cassandra/io/sstable/LegacySSTableTest.java    | 109 ++++++++++++++++-----
 18 files changed, 273 insertions(+), 150 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index eae2815..d34406b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.19
+ * Skipping illegal legacy cells can break reverse iteration of indexed partitions (CASSANDRA-15178)
  * Handle paging states serialized with a different version than the session's (CASSANDRA-15176)
  * Throw IOE instead of asserting on unsupporter peer versions (CASSANDRA-15066)
  * Update token metadata when handling MOVING/REMOVING_TOKEN events (CASSANDRA-15120)
diff --git a/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java b/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java
deleted file mode 100644
index b70d248..0000000
--- a/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.cassandra.db;
-
-import java.nio.ByteBuffer;
-
-import org.apache.cassandra.config.CFMetaData;
-
-import static org.apache.cassandra.db.LegacyLayout.stringify;
-
-/**
- * Exception thrown when we attempt to decode a legacy cellname
- * and the column name component refers to a primary key column.
- */
-public class IllegalLegacyColumnException extends Exception
-{
-    public final ByteBuffer columnName;
-
-    public IllegalLegacyColumnException(CFMetaData metaData, ByteBuffer columnName)
-    {
-        super(String.format("Illegal cell name for CQL3 table %s.%s. %s is defined as a primary key column",
-                            metaData.ksName, metaData.cfName, stringify(columnName)));
-        this.columnName = columnName;
-    }
-}
diff --git a/src/java/org/apache/cassandra/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java
index cfaa71f..b03f56e 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -124,7 +124,7 @@ public abstract class LegacyLayout
     }
 
     public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer cellname)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         assert cellname != null;
         if (metadata.isSuper())
@@ -151,12 +151,12 @@ public abstract class LegacyLayout
         return new LegacyCellName(clustering, def, subcol);
     }
 
-    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException, IllegalLegacyColumnException
+    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException
     {
         return decodeCellName(metadata, cellname, false);
     }
 
-    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException, IllegalLegacyColumnException
+    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException
     {
         Clustering clustering = decodeClustering(metadata, cellname);
 
@@ -184,22 +184,16 @@ public abstract class LegacyLayout
             return new LegacyCellName(clustering, null, null);
 
         ColumnDefinition def = metadata.getColumnDefinition(column);
-        if ((def == null) || def.isPrimaryKeyColumn())
+
+        if (metadata.isCompactTable())
         {
-            // If it's a compact table, it means the column is in fact a "dynamic" one
-            if (metadata.isCompactTable())
+            if (def == null || def.isPrimaryKeyColumn())
+                // If it's a compact table, it means the column is in fact a "dynamic" one
                 return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null);
-
-            if (def == null)
-            {
-                throw new UnknownColumnException(metadata, column);
-            }
-            else
-            {
-                noSpamLogger.warn("Illegal cell name for CQL3 table {}.{}. {} is defined as a primary key column",
-                                 metadata.ksName, metadata.cfName, stringify(column));
-                throw new IllegalLegacyColumnException(metadata, column);
-            }
+        }
+        else if (def == null)
+        {
+            throw new UnknownColumnException(metadata, column);
         }
 
         ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null;
@@ -750,13 +744,6 @@ public abstract class LegacyLayout
             {
                 // Simply skip, as the method name implies.
             }
-            catch (IllegalLegacyColumnException e)
-            {
-                // We can arrive here if the table is non-compact and an sstable contains cells whose column name components
-                // refer to a primary key column. This is not possible through CQL, but is through thrift or side loading
-                // sstables. In this case, we treat the column as an unknown and skip, which is equivalent to the pre
-                // 3.0 read path behaviour
-            }
         }
 
     }
@@ -1116,7 +1103,7 @@ public abstract class LegacyLayout
     }
 
     public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, boolean readAllAsDynamic)
-    throws IOException, UnknownColumnException, IllegalLegacyColumnException
+    throws IOException, UnknownColumnException
     {
         ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
         if (!cellname.hasRemaining())
@@ -1148,7 +1135,7 @@ public abstract class LegacyLayout
         }
     }
 
-    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException, IllegalLegacyColumnException
+    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException
     {
         ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
         int b = in.readUnsignedByte();
@@ -1156,7 +1143,7 @@ public abstract class LegacyLayout
     }
 
     public static LegacyCell readLegacyCellBody(CFMetaData metadata, DataInput in, ByteBuffer cellname, int mask, SerializationHelper.Flag flag, boolean readAllAsDynamic)
-    throws IOException, UnknownColumnException, IllegalLegacyColumnException
+    throws IOException, UnknownColumnException
     {
         // Note that we want to call decodeCellName only after we've deserialized other parts, since it can throw
         // and we want to throw only after having deserialized the full cell.
@@ -1228,14 +1215,6 @@ public abstract class LegacyLayout
                     else
                         throw new IOError(e);
                 }
-                catch (IllegalLegacyColumnException e)
-                {
-                    // We can arrive here if the table is non-compact and an sstable contains cells whose column name components
-                    // refer to a primary key column. This is not possible through CQL, but is through thrift or side loading
-                    // sstables. In this case, we treat the atom as though it were unknown and skip on to the next, which is
-                    // equivalent to the pre 3.0 behaviour
-                    return computeNext();
-                }
                 catch (IOException e)
                 {
                     throw new IOError(e);
@@ -1334,6 +1313,13 @@ public abstract class LegacyLayout
                 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))
                 {
                     CellPath path = null;
@@ -1633,13 +1619,13 @@ public abstract class LegacyLayout
         }
 
         public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp)
-        throws UnknownColumnException, IllegalLegacyColumnException
+        throws UnknownColumnException
         {
             return new LegacyCell(Kind.REGULAR, decodeCellName(metadata, superColumnName, name), value, timestamp, Cell.NO_DELETION_TIME, Cell.NO_TTL);
         }
 
         public static LegacyCell expiring(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp, int ttl, int nowInSec)
-        throws UnknownColumnException, IllegalLegacyColumnException
+        throws UnknownColumnException
         {
             /*
              * CASSANDRA-14092: Max expiration date capping is maybe performed here, expiration overflow policy application
@@ -1649,13 +1635,13 @@ public abstract class LegacyLayout
         }
 
         public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
-        throws UnknownColumnException, IllegalLegacyColumnException
+        throws UnknownColumnException
         {
             return new LegacyCell(Kind.DELETED, decodeCellName(metadata, superColumnName, name), ByteBufferUtil.EMPTY_BYTE_BUFFER, timestamp, nowInSec, LivenessInfo.NO_TTL);
         }
 
         public static LegacyCell counterUpdate(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long value)
-        throws UnknownColumnException, IllegalLegacyColumnException
+        throws UnknownColumnException
         {
             // See UpdateParameters.addCounter() for more details on this
             ByteBuffer counterValue = CounterContext.instance().createUpdate(value);
@@ -2606,16 +2592,4 @@ public abstract class LegacyLayout
             return size;
         }
     }
-
-    public static String stringify(ByteBuffer name)
-    {
-        try
-        {
-            return UTF8Type.instance.getString(name);
-        }
-        catch (Exception e)
-        {
-            return ByteBufferUtil.bytesToHex(name);
-        }
-    }
 }
diff --git a/src/java/org/apache/cassandra/db/ReadCommand.java b/src/java/org/apache/cassandra/db/ReadCommand.java
index eb5cddf..b499daf 100644
--- a/src/java/org/apache/cassandra/db/ReadCommand.java
+++ b/src/java/org/apache/cassandra/db/ReadCommand.java
@@ -1340,7 +1340,7 @@ public abstract class ReadCommand implements ReadQuery
                 {
                     cellName = LegacyLayout.decodeCellName(metadata, buffer);
                 }
-                catch (UnknownColumnException | IllegalLegacyColumnException exc)
+                catch (UnknownColumnException exc)
                 {
                     // TODO this probably needs a new exception class that shares a parent with UnknownColumnFamilyException
                     throw new UnknownColumnFamilyException(
diff --git a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
index 50c9a1a..62ad76a 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -302,13 +302,6 @@ public abstract class UnfilteredDeserializer
                     // we should *not* count the byte of that discarded cell as part of the bytes for the atom
                     // we will eventually return, as doing so could throw the logic bytesReadForNextAtom participates in.
                 }
-                catch (IllegalLegacyColumnException e)
-                {
-                    // This can occur if the table is non-compact and an sstable contains cells whose column name components
-                    // refer to a primary key column. This is not possible through CQL, but is through thrift or side loading
-                    // sstables. We treat this as we wold an unknown column and skip without counting the read bytes as
-                    // part of the following atom
-                }
                 catch (IOException e)
                 {
                     throw new IOError(e);
diff --git a/src/java/org/apache/cassandra/db/UnknownColumnException.java b/src/java/org/apache/cassandra/db/UnknownColumnException.java
index a480022..55dc453 100644
--- a/src/java/org/apache/cassandra/db/UnknownColumnException.java
+++ b/src/java/org/apache/cassandra/db/UnknownColumnException.java
@@ -20,8 +20,8 @@ package org.apache.cassandra.db;
 import java.nio.ByteBuffer;
 
 import org.apache.cassandra.config.CFMetaData;
-
-import static org.apache.cassandra.db.LegacyLayout.stringify;
+import org.apache.cassandra.db.marshal.UTF8Type;
+import org.apache.cassandra.utils.ByteBufferUtil;
 
 /**
  * Exception thrown when we read a column internally that is unknown. Note that
@@ -36,4 +36,16 @@ public class UnknownColumnException extends Exception
         super(String.format("Unknown column %s in table %s.%s", stringify(columnName), metadata.ksName, metadata.cfName));
         this.columnName = columnName;
     }
+
+    private static String stringify(ByteBuffer name)
+    {
+        try
+        {
+            return UTF8Type.instance.getString(name);
+        }
+        catch (Exception e)
+        {
+            return ByteBufferUtil.bytesToHex(name);
+        }
+    }
 }
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 3a41df0..163eb2d 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -416,7 +416,7 @@ public class CassandraServer implements Cassandra.Iface
                 return toInternalFilter(metadata, parent, predicate.slice_range);
             }
         }
-        catch (UnknownColumnException | IllegalLegacyColumnException e)
+        catch (UnknownColumnException e)
         {
             throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage());
         }
@@ -521,7 +521,7 @@ public class CassandraServer implements Cassandra.Iface
                 return makeColumnFilter(metadata, parent, predicate.slice_range);
             }
         }
-        catch (UnknownColumnException | IllegalLegacyColumnException e)
+        catch (UnknownColumnException e)
         {
             throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage());
         }
@@ -660,7 +660,7 @@ public class CassandraServer implements Cassandra.Iface
                 return tcolumns.get(0);
             }
         }
-        catch (UnknownColumnException | IllegalLegacyColumnException e)
+        catch (UnknownColumnException e)
         {
             throw new InvalidRequestException(e.getMessage());
         }
@@ -859,7 +859,7 @@ public class CassandraServer implements Cassandra.Iface
 
             mutation = new org.apache.cassandra.db.Mutation(update);
         }
-        catch (MarshalException | UnknownColumnException | IllegalLegacyColumnException e)
+        catch (MarshalException|UnknownColumnException e)
         {
             throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage());
         }
@@ -964,7 +964,7 @@ public class CassandraServer implements Cassandra.Iface
                      : new CASResult(false).setCurrent_values(thriftifyColumnsAsColumns(metadata, LegacyLayout.fromRowIterator(result).right));
             }
         }
-        catch (UnknownColumnException | IllegalLegacyColumnException e)
+        catch (UnknownColumnException e)
         {
             throw new InvalidRequestException(e.getMessage());
         }
@@ -986,13 +986,13 @@ public class CassandraServer implements Cassandra.Iface
         }
     }
 
-    private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, Column column, int nowInSec) throws UnknownColumnException, IllegalLegacyColumnException
+    private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, Column column, int nowInSec) throws UnknownColumnException
     {
         return toLegacyCell(metadata, null, column, nowInSec);
     }
 
     private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, ByteBuffer superColumnName, Column column, int nowInSec)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         return column.ttl > 0
              ? LegacyLayout.LegacyCell.expiring(metadata, superColumnName, column.name, column.value, column.timestamp, column.ttl, nowInSec)
@@ -1000,25 +1000,25 @@ public class CassandraServer implements Cassandra.Iface
     }
 
     private LegacyLayout.LegacyCell toLegacyDeletion(CFMetaData metadata, ByteBuffer name, long timestamp, int nowInSec)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         return toLegacyDeletion(metadata, null, name, timestamp, nowInSec);
     }
 
     private LegacyLayout.LegacyCell toLegacyDeletion(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         return LegacyLayout.LegacyCell.tombstone(metadata, superColumnName, name, timestamp, nowInSec);
     }
 
     private LegacyLayout.LegacyCell toCounterLegacyCell(CFMetaData metadata, CounterColumn column)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         return toCounterLegacyCell(metadata, null, column);
     }
 
     private LegacyLayout.LegacyCell toCounterLegacyCell(CFMetaData metadata, ByteBuffer superColumnName, CounterColumn column)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         return LegacyLayout.LegacyCell.counterUpdate(metadata, superColumnName, column.name, column.value);
     }
@@ -1085,7 +1085,7 @@ public class CassandraServer implements Cassandra.Iface
     }
 
     private List<LegacyLayout.LegacyCell> toLegacyCells(CFMetaData metadata, List<Column> columns, int nowInSec)
-    throws UnknownColumnException, IllegalLegacyColumnException
+    throws UnknownColumnException
     {
         List<LegacyLayout.LegacyCell> cells = new ArrayList<>(columns.size());
         for (Column column : columns)
@@ -1205,7 +1205,7 @@ public class CassandraServer implements Cassandra.Iface
                 cells.add(toCounterLegacyCell(cfm, cosc.counter_column));
             }
         }
-        catch (UnknownColumnException | IllegalLegacyColumnException e)
+        catch (UnknownColumnException e)
         {
             throw new InvalidRequestException(e.getMessage());
         }
@@ -1232,7 +1232,7 @@ public class CassandraServer implements Cassandra.Iface
                     else
                         cells.add(toLegacyDeletion(cfm, c, del.timestamp, nowInSec));
                 }
-                catch (UnknownColumnException | IllegalLegacyColumnException e)
+                catch (UnknownColumnException e)
                 {
                     throw new InvalidRequestException(e.getMessage());
                 }
@@ -1370,7 +1370,7 @@ public class CassandraServer implements Cassandra.Iface
                 Cell cell = BufferCell.tombstone(name.column, timestamp, nowInSec, path);
                 update = PartitionUpdate.singleRowUpdate(metadata, dk, BTreeRow.singleCellRow(name.clustering, cell));
             }
-            catch (UnknownColumnException | IllegalLegacyColumnException e)
+            catch (UnknownColumnException e)
             {
                 throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage());
             }
@@ -1633,7 +1633,7 @@ public class CassandraServer implements Cassandra.Iface
                     return thriftifyKeySlices(results, new ColumnParent(column_family), limits.perPartitionCount());
                 }
             }
-            catch (UnknownColumnException | IllegalLegacyColumnException e)
+            catch (UnknownColumnException e)
             {
                 throw new InvalidRequestException(e.getMessage());
             }
@@ -2178,7 +2178,7 @@ public class CassandraServer implements Cassandra.Iface
                 org.apache.cassandra.db.Mutation mutation = new org.apache.cassandra.db.Mutation(update);
                 doInsert(consistency_level, Arrays.asList(new CounterMutation(mutation, ThriftConversion.fromThrift(consistency_level))));
             }
-            catch (MarshalException | UnknownColumnException | IllegalLegacyColumnException e)
+            catch (MarshalException|UnknownColumnException e)
             {
                 throw new InvalidRequestException(e.getMessage());
             }
diff --git a/src/java/org/apache/cassandra/thrift/ThriftValidation.java b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
index e7a344a..4b208ba 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -31,7 +31,6 @@ import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.Attributes;
 import org.apache.cassandra.cql3.Operator;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.IllegalLegacyColumnException;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.dht.IPartitioner;
 import org.apache.cassandra.dht.Token;
@@ -266,11 +265,13 @@ public class ThriftValidation
                                                                                                             i, metadata.comparator.size() + 1, metadata.cfName));
                     }
 
+
+
                     // On top of that, if we have a collection component, the (CQL3) column must be a collection
                     if (cname.column != null && cname.collectionElement != null && !cname.column.type.isCollection())
                         throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Invalid collection component, %s is not a collection", cname.column.name));
                 }
-                catch (IllegalArgumentException | UnknownColumnException | IllegalLegacyColumnException e )
+                catch (IllegalArgumentException | UnknownColumnException e)
                 {
                     throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Error validating cell name for CQL3 table %s: %s", metadata.cfName, e.getMessage()));
                 }
@@ -464,10 +465,13 @@ public class ThriftValidation
         try
         {
             LegacyLayout.LegacyCellName cn = LegacyLayout.decodeCellName(metadata, scName, column.name);
+            if (cn.column.isPrimaryKeyColumn())
+                throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Cannot add primary key column %s to partition update", cn.column.name));
+
             cn.column.validateCellValue(column.value);
 
         }
-        catch (UnknownColumnException | IllegalLegacyColumnException e)
+        catch (UnknownColumnException e)
         {
             throw new org.apache.cassandra.exceptions.InvalidRequestException(e.getMessage());
         }
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-CRC.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-CRC.db
new file mode 100644
index 0000000..82ca06a
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-CRC.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Data.db
new file mode 100644
index 0000000..269a739
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Data.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Digest.sha1
new file mode 100644
index 0000000..7c85191
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Digest.sha1
@@ -0,0 +1 @@
+1999183849
\ No newline at end of file
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Filter.db
new file mode 100644
index 0000000..f3f7da5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Filter.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Index.db
new file mode 100644
index 0000000..bff0123
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Index.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Statistics.db
new file mode 100644
index 0000000..febb2be
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-Statistics.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-TOC.txt
new file mode 100644
index 0000000..c360dbf
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names_indexed/legacy_tables-legacy_ka_with_illegal_cell_names_indexed-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+CRC.db
+Statistics.db
+TOC.txt
+Data.db
+Index.db
+Summary.db
+Digest.sha1
+Filter.db
diff --git a/test/unit/org/apache/cassandra/cql3/validation/ThriftIllegalColumnsTest.java b/test/unit/org/apache/cassandra/cql3/validation/ThriftIllegalColumnsTest.java
new file mode 100644
index 0000000..2d922e0
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/validation/ThriftIllegalColumnsTest.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.cassandra.cql3.validation;
+
+import java.nio.ByteBuffer;
+
+import org.junit.Test;
+
+import org.apache.cassandra.cql3.validation.operations.ThriftCQLTester;
+import org.apache.cassandra.db.marshal.CompositeType;
+import org.apache.cassandra.thrift.Cassandra;
+import org.apache.cassandra.thrift.Column;
+import org.apache.cassandra.thrift.ColumnParent;
+import org.apache.cassandra.thrift.InvalidRequestException;
+import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.thrift.ConsistencyLevel.ONE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class ThriftIllegalColumnsTest extends ThriftCQLTester
+{
+    final String NON_COMPACT_TABLE = "t1";
+    final String COMPACT_TABLE = "t2";
+
+    @Test
+    public void testNonCompactUpdateWithPrimaryKeyColumnName() throws Throwable
+    {
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+        String table = createTable(KEYSPACE, "CREATE TABLE %s (k int, c1 int,  c2 int, v int, PRIMARY KEY (k, c1, c2))");
+
+        // A cell name which represents a primary key column
+        ByteBuffer badCellName = CompositeType.build(ByteBufferUtil.bytes(0), ByteBufferUtil.bytes(0), ByteBufferUtil.bytes("c1"));
+        // A cell name which represents a regular column
+        ByteBuffer goodCellName = CompositeType.build(ByteBufferUtil.bytes(0), ByteBufferUtil.bytes(0), ByteBufferUtil.bytes("v"));
+
+        ColumnParent parent = new ColumnParent(table);
+        ByteBuffer key = ByteBufferUtil.bytes(0);
+        Column column = new Column();
+        column.setName(badCellName);
+        column.setValue(ByteBufferUtil.bytes(999));
+        column.setTimestamp(System.currentTimeMillis());
+
+        try
+        {
+            client.insert(key, parent, column, ONE);
+            fail("Expected exception");
+        } catch (InvalidRequestException e) {
+            assertEquals("Cannot add primary key column c1 to partition update", e.getWhy());
+        }
+
+        column.setName(goodCellName);
+        client.insert(key, parent, column, ONE);
+        assertRows(execute("SELECT v from %s WHERE k = 0"), row(999));
+    }
+
+    @Test
+    public void testThriftCompactUpdateWithPrimaryKeyColumnName() throws Throwable
+    {
+        Cassandra.Client client = getClient();
+        client.set_keyspace(KEYSPACE);
+        String table = createTable(KEYSPACE, "CREATE TABLE %s (k int, v int, PRIMARY KEY (k)) WITH COMPACT STORAGE");
+
+        // A cell name which represents a primary key column
+        ByteBuffer badCellName = ByteBufferUtil.bytes("k");
+        // A cell name which represents a regular column
+        ByteBuffer goodCellName = ByteBufferUtil.bytes("v");
+
+        ColumnParent parent = new ColumnParent(table);
+        ByteBuffer key = ByteBufferUtil.bytes(0);
+        Column column = new Column();
+        column.setName(badCellName);
+        column.setValue(ByteBufferUtil.bytes(999));
+        column.setTimestamp(System.currentTimeMillis());
+        // if the table is compact, a cell name which appears to reference a primary
+        // key column is treated as a dynamic column and so the update is allowed
+        client.insert(key, parent, column, ONE);
+
+        column.setName(goodCellName);
+        client.insert(key, parent, column, ONE);
+        assertRows(execute("SELECT v from %s where k=0"), row(999));
+    }
+}
diff --git a/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java b/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
index c4d3988..455fa9f 100644
--- a/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
+++ b/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
@@ -55,7 +55,7 @@ public class LegacyCellNameTest
         assertTrue(cellName.column.isRegular());
     }
 
-    @Test(expected=IllegalLegacyColumnException.class)
+    @Test
     public void testColumnSameNameAsPartitionKeyCql3() throws Exception
     {
         CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
@@ -65,18 +65,26 @@ public class LegacyCellNameTest
             = LegacyLayout.decodeCellName(cfm, 
                                           LegacyLayout.makeLegacyComparator(cfm)
                                                       .fromString("k"));
+
+        // When being grouped into Rows by LegacyLayout.CellGrouper,
+        // primary key columns are filtered out
+        assertTrue(cellName.column.isPrimaryKeyColumn());
     }
 
-    @Test(expected=IllegalLegacyColumnException.class)
+    @Test
     public void testCompositeWithColumnNameSameAsClusteringKeyCql3() throws Exception
     {
         CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
                                             "k int, c text, v int, PRIMARY KEY(k, c))", "ks");
 
         LegacyLayout.LegacyCellName cellName
-        = LegacyLayout.decodeCellName(cfm,
-                                      LegacyLayout.makeLegacyComparator(cfm)
-                                                  .fromString("c_value:c"));
+            = LegacyLayout.decodeCellName(cfm,
+                                          LegacyLayout.makeLegacyComparator(cfm)
+                                                      .fromString("c_value:c"));
+
+        // When being grouped into Rows by LegacyLayout.CellGrouper,
+        // primary key columns are filtered out
+        assertTrue(cellName.column.isPrimaryKeyColumn());
     }
 
     // This throws IllegalArgumentException not because the cellname's value matches
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index eeec76d..ecb8125 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -62,9 +62,12 @@ import org.apache.cassandra.streaming.StreamPlan;
 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;
+
 /**
  * Tests backwards compatibility for SSTables
  */
@@ -172,7 +175,7 @@ public class LegacySSTableTest
                                                              "FROM legacy_tables.legacy_ka_indexed " +
                                                              "WHERE p=1 " +
                                                              "ORDER BY c DESC");
-        Assert.assertEquals(5000, rs.size());
+        assertEquals(5000, rs.size());
     }
 
     @Test
@@ -193,7 +196,7 @@ public class LegacySSTableTest
         UntypedResultSet rs = QueryProcessor.executeInternal("SELECT * " +
                                                              "FROM legacy_tables.legacy_ka_indexed_static " +
                                                              "WHERE p=1 ");
-        Assert.assertEquals(5000, rs.size());
+        assertEquals(5000, rs.size());
     }
 
     @Test
@@ -214,11 +217,11 @@ public class LegacySSTableTest
 
         // read all rows in ASC order, expect all 4 to be returned
         rs = QueryProcessor.executeInternal("SELECT * FROM legacy_tables.legacy_ka_14766 WHERE pk = 0 ORDER BY ck ASC;");
-        Assert.assertEquals(4, rs.size());
+        assertEquals(4, rs.size());
 
         // read all rows in DESC order, expect all 4 to be returned
         rs = QueryProcessor.executeInternal("SELECT * FROM legacy_tables.legacy_ka_14766 WHERE pk = 0 ORDER BY ck DESC;");
-        Assert.assertEquals(4, rs.size());
+        assertEquals(4, rs.size());
     }
 
     @Test
@@ -238,7 +241,7 @@ public class LegacySSTableTest
 
         logger.info("{} - {}", forward.size(), reverse.size());
         Assert.assertFalse(forward.isEmpty());
-        Assert.assertEquals(forward.size(), reverse.size());
+        assertEquals(forward.size(), reverse.size());
     }
 
     @Test
@@ -275,8 +278,8 @@ public class LegacySSTableTest
             QueryProcessor.executeOnceInternal(
                 String.format("SELECT * FROM legacy_tables.legacy_ka_14873 WHERE pkc = 0 AND cc > 0 ORDER BY cc ASC;"));
 
-        Assert.assertEquals(5, forward.size());
-        Assert.assertEquals(5, reverse.size());
+        assertEquals(5, forward.size());
+        assertEquals(5, reverse.size());
     }
 
     @Test
@@ -301,7 +304,7 @@ public class LegacySSTableTest
             UntypedResultSet reverse = QueryProcessor.executeOnceInternal(String.format("SELECT * FROM legacy_tables.%s WHERE k=100 ORDER BY c1 DESC, c2 DESC", table));
 
             Assert.assertFalse(forward.isEmpty());
-            Assert.assertEquals(table, forward.size(), reverse.size());
+            assertEquals(table, forward.size(), reverse.size());
         }
     }
 
@@ -320,7 +323,7 @@ public class LegacySSTableTest
 
         String query = "SELECT * FROM legacy_tables.legacy_mc_inaccurate_min_max WHERE k=100 AND c1=1 AND c2=1";
         List<Unfiltered> unfiltereds = SinglePartitionSliceCommandTest.getUnfilteredsFromSinglePartition(query);
-        Assert.assertEquals(2, unfiltereds.size());
+        assertEquals(2, unfiltereds.size());
         Assert.assertTrue(unfiltereds.get(0).isRangeTombstoneMarker());
         Assert.assertTrue(((RangeTombstoneMarker) unfiltereds.get(0)).isOpen(false));
         Assert.assertTrue(unfiltereds.get(1).isRangeTombstoneMarker());
@@ -442,6 +445,66 @@ public class LegacySSTableTest
         assertRows(results, row(1, "a", "aa", "aaa"), row(2, "b", "bb", "bbb"));
     }
 
+    @Test
+    public void testReadingIndexedLegacyTablesWithIllegalCellNames() throws Exception {
+        /**
+         * The sstable can be generated externally with SSTableSimpleUnsortedWriter:
+         * column_index_size_in_kb: 1
+         * [
+         *   {"key": "key",
+         *    "cells": [
+         *               ["00000:000000:a","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0],
+         *               ["00000:000000:b","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00000:000000:c","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00000:000000:z","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00001:000001:a","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0],
+         *               ["00001:000001:b","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00001:000001:c","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00001:000001:z","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               .
+         *               .
+         *               .
+         *               ["00010:000010:a","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0],
+         *               ["00010:000010:b","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00010:000010:c","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *               ["00010:000010:z","00000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",0]
+         *           ]
+         *   }
+         * ]
+         * Each row in the partition contains only 1 valid cell. The ones with the column name components 'a', 'b' & 'z' are illegal as they refer to PRIMARY KEY
+         * columns, but SSTables such as this can be generated with offline tools and loaded via SSTableLoader or nodetool refresh (see CASSANDRA-15086) (see
+         * CASSANDRA-15086) Only 'c' is a valid REGULAR column in the table schema.
+         * In the initial fix for CASSANDRA-15086, the bytes read by OldFormatDeserializer for these invalid cells are not correctly accounted for, causing
+         * ReverseIndexedReader to assert that the end of a block has been reached earlier than it actually has, which in turn causes rows to be incorrectly
+         * ommitted from the results.
+         *
+         * This sstable has been crafted to hit a further potential error condition. Rows 00001:00001 and 00008:00008 interact with the index block boundaries
+         * in a very specific way; for both of these rows, the (illegal) cells 'a' & 'b', along with the valid 'c' cell are at the end of an index block, but
+         * the 'z' cell is over the boundary, in the following block. We need to ensure that the bytes consumed for the 'z' cell are properly accounted for and
+         * not counted toward those for the next row on disk.
+         */
+        QueryProcessor.executeInternal("CREATE TABLE legacy_tables.legacy_ka_with_illegal_cell_names_indexed (" +
+                                       " a text," +
+                                       " b text," +
+                                       " z text," +
+                                       " c text," +
+                                       " PRIMARY KEY(a, b, z))");
+        loadLegacyTable("legacy_%s_with_illegal_cell_names_indexed%s", "ka", "");
+        String queryForward = "SELECT * FROM legacy_tables.legacy_ka_with_illegal_cell_names_indexed WHERE a = 'key'";
+        String queryReverse = queryForward + " ORDER BY b DESC, z DESC";
+
+        List<String> forward = new ArrayList<>();
+        QueryProcessor.executeOnceInternal(queryForward).forEach(r -> forward.add(r.getString("b") + ":" +  r.getString("z")));
+
+        List<String> reverse = new ArrayList<>();
+        QueryProcessor.executeOnceInternal(queryReverse).forEach(r -> reverse.add(r.getString("b") + ":" +  r.getString("z")));
+
+        assertEquals(11, reverse.size());
+        assertEquals(11, forward.size());
+        for (int i=0; i < 11; i++)
+            assertEquals(forward.get(i), reverse.get(10 - i));
+    }
+
     private void assertExpectedRowsWithDroppedCollection(boolean droppedCheckSuccessful)
     {
         for (int i=0; i<=1; i++)
@@ -449,7 +512,7 @@ public class LegacySSTableTest
             UntypedResultSet rows =
                 QueryProcessor.executeOnceInternal(
                     String.format("SELECT * FROM legacy_tables.legacy_ka_14912 WHERE k = %s;", i));
-            Assert.assertEquals(1, rows.size());
+            assertEquals(1, rows.size());
             UntypedResultSet.Row row = rows.one();
 
             // If the best-effort attempt to filter dropped columns was successful, then the row
@@ -462,9 +525,9 @@ public class LegacySSTableTest
             if (droppedCheckSuccessful || i == 0)
                 Assert.assertFalse(row.has("v1"));
             else
-                Assert.assertEquals("", row.getString("v1"));
+                assertEquals("", row.getString("v1"));
 
-            Assert.assertEquals("abc", row.getString("v2"));
+            assertEquals("abc", row.getString("v2"));
         }
     }
 
@@ -517,12 +580,12 @@ public class LegacySSTableTest
         Assert.assertTrue(endCount > startCount);
         CacheService.instance.keyCache.submitWrite(Integer.MAX_VALUE).get();
         CacheService.instance.invalidateKeyCache();
-        Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
+        assertEquals(startCount, CacheService.instance.keyCache.size());
         CacheService.instance.keyCache.loadSaved();
         if (BigFormat.instance.getVersion(legacyVersion).storeRows())
-            Assert.assertEquals(endCount, CacheService.instance.keyCache.size());
+            assertEquals(endCount, CacheService.instance.keyCache.size());
         else
-            Assert.assertEquals(startCount, CacheService.instance.keyCache.size());
+            assertEquals(startCount, CacheService.instance.keyCache.size());
     }
 
     private static void verifyReads(String legacyVersion)
@@ -557,8 +620,8 @@ public class LegacySSTableTest
         UntypedResultSet rs;
         rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_clust_counter%s WHERE pk=? AND ck=?", legacyVersion, compactSuffix), pkValue, ckValue);
         Assert.assertNotNull(rs);
-        Assert.assertEquals(1, rs.size());
-        Assert.assertEquals(1L, rs.one().getLong("val"));
+        assertEquals(1, rs.size());
+        assertEquals(1L, rs.one().getLong("val"));
     }
 
     private static void readClusteringTable(String legacyVersion, String compactSuffix, int ck, String ckValue, String pkValue)
@@ -580,8 +643,8 @@ public class LegacySSTableTest
         UntypedResultSet rs;
         rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple_counter%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
         Assert.assertNotNull(rs);
-        Assert.assertEquals(1, rs.size());
-        Assert.assertEquals(1L, rs.one().getLong("val"));
+        assertEquals(1, rs.size());
+        assertEquals(1L, rs.one().getLong("val"));
     }
 
     private static void readSimpleTable(String legacyVersion, String compactSuffix, String pkValue)
@@ -590,8 +653,8 @@ public class LegacySSTableTest
         UntypedResultSet rs;
         rs = QueryProcessor.executeInternal(String.format("SELECT val FROM legacy_tables.legacy_%s_simple%s WHERE pk=?", legacyVersion, compactSuffix), pkValue);
         Assert.assertNotNull(rs);
-        Assert.assertEquals(1, rs.size());
-        Assert.assertEquals("foo bar baz", rs.one().getString("val"));
+        assertEquals(1, rs.size());
+        assertEquals("foo bar baz", rs.one().getString("val"));
     }
 
     private static void createKeyspace()
@@ -633,12 +696,12 @@ public class LegacySSTableTest
     private static void assertLegacyClustRows(int count, UntypedResultSet rs)
     {
         Assert.assertNotNull(rs);
-        Assert.assertEquals(count, rs.size());
+        assertEquals(count, rs.size());
         for (int i = 0; i < count; i++)
         {
             for (UntypedResultSet.Row r : rs)
             {
-                Assert.assertEquals(128, r.getString("val").length());
+                assertEquals(128, r.getString("val").length());
             }
         }
     }


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