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/05/08 09:09:08 UTC

[cassandra] branch cassandra-3.0 updated: Skip cells with illegal column names when reading legacy sstables

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 daf6c85  Skip cells with illegal column names when reading legacy sstables
daf6c85 is described below

commit daf6c85b0530fb45ad1d8433efa0c5e13d7e182c
Author: Sam Tunnicliffe <sa...@beobal.com>
AuthorDate: Thu Apr 11 17:32:17 2019 +0100

    Skip cells with illegal column names when reading legacy sstables
    
    Patch by Sam Tunnicliffe; reviewed by Aleksey Yeschenko for CASSANDRA-15086
---
 CHANGES.txt                                        |   1 +
 ...tion.java => IllegalLegacyColumnException.java} |  28 ++++------
 src/java/org/apache/cassandra/db/LegacyLayout.java |  57 +++++++++++++++++----
 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  |   5 +-
 ...s-legacy_ka_with_illegal_cell_names-ka-1-CRC.db | Bin 0 -> 8 bytes
 ...-legacy_ka_with_illegal_cell_names-ka-1-Data.db | Bin 0 -> 292 bytes
 ...acy_ka_with_illegal_cell_names-ka-1-Digest.sha1 |   1 +
 ...egacy_ka_with_illegal_cell_names-ka-1-Filter.db | Bin 0 -> 16 bytes
 ...legacy_ka_with_illegal_cell_names-ka-1-Index.db | Bin 0 -> 36 bytes
 ...y_ka_with_illegal_cell_names-ka-1-Statistics.db | Bin 0 -> 4464 bytes
 ...gacy_ka_with_illegal_cell_names-ka-1-Summary.db | Bin 0 -> 128 bytes
 ...-legacy_ka_with_illegal_cell_names-ka-1-TOC.txt |   8 +++
 .../apache/cassandra/db/LegacyCellNameTest.java    |  17 +++++-
 .../cassandra/io/sstable/LegacySSTableTest.java    |  36 +++++++++++++
 18 files changed, 147 insertions(+), 65 deletions(-)

diff --git a/CHANGES.txt b/CHANGES.txt
index 596d902..0dac500 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.19
+ * Skip cells with illegal column names when reading legacy sstables (CASSANDRA-15086)
  * Fix assorted gossip races and add related runtime checks (CASSANDRA-15059)
  * Fix mixed mode partition range scans with limit (CASSANDRA-15072)
  * cassandra-stress works with frozen collections: list and set (CASSANDRA-14907)
diff --git a/src/java/org/apache/cassandra/db/UnknownColumnException.java b/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java
similarity index 57%
copy from src/java/org/apache/cassandra/db/UnknownColumnException.java
copy to src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java
index 55dc453..b70d248 100644
--- a/src/java/org/apache/cassandra/db/UnknownColumnException.java
+++ b/src/java/org/apache/cassandra/db/IllegalLegacyColumnException.java
@@ -15,37 +15,27 @@
  * 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 org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.LegacyLayout.stringify;
 
 /**
- * Exception thrown when we read a column internally that is unknown. Note that
- * this is an internal exception and is not meant to be user facing.
+ * Exception thrown when we attempt to decode a legacy cellname
+ * and the column name component refers to a primary key column.
  */
-public class UnknownColumnException extends Exception
+public class IllegalLegacyColumnException extends Exception
 {
     public final ByteBuffer columnName;
 
-    public UnknownColumnException(CFMetaData metadata, ByteBuffer columnName)
+    public IllegalLegacyColumnException(CFMetaData metaData, ByteBuffer columnName)
     {
-        super(String.format("Unknown column %s in table %s.%s", stringify(columnName), metadata.ksName, metadata.cfName));
+        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;
     }
-
-    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/db/LegacyLayout.java b/src/java/org/apache/cassandra/db/LegacyLayout.java
index 0044959..cfaa71f 100644
--- a/src/java/org/apache/cassandra/db/LegacyLayout.java
+++ b/src/java/org/apache/cassandra/db/LegacyLayout.java
@@ -23,6 +23,7 @@ import java.io.IOError;
 import java.nio.ByteBuffer;
 import java.security.MessageDigest;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.cql3.SuperColumnCompatibility;
@@ -55,6 +56,7 @@ import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
 public abstract class LegacyLayout
 {
     private static final Logger logger = LoggerFactory.getLogger(LegacyLayout.class);
+    private static final NoSpamLogger noSpamLogger = NoSpamLogger.getLogger(logger, 1L, TimeUnit.MINUTES);
 
     public final static int MAX_CELL_NAME_LENGTH = FBUtilities.MAX_UNSIGNED_SHORT;
 
@@ -122,7 +124,7 @@ public abstract class LegacyLayout
     }
 
     public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer cellname)
-    throws UnknownColumnException
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         assert cellname != null;
         if (metadata.isSuper())
@@ -149,12 +151,12 @@ public abstract class LegacyLayout
         return new LegacyCellName(clustering, def, subcol);
     }
 
-    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException
+    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname) throws UnknownColumnException, IllegalLegacyColumnException
     {
         return decodeCellName(metadata, cellname, false);
     }
 
-    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException
+    public static LegacyCellName decodeCellName(CFMetaData metadata, ByteBuffer cellname, boolean readAllAsDynamic) throws UnknownColumnException, IllegalLegacyColumnException
     {
         Clustering clustering = decodeClustering(metadata, cellname);
 
@@ -189,9 +191,15 @@ public abstract class LegacyLayout
                 return new LegacyCellName(new Clustering(column), metadata.compactValueColumn(), null);
 
             if (def == null)
+            {
                 throw new UnknownColumnException(metadata, column);
+            }
             else
-                throw new IllegalArgumentException("Cannot add primary key column to partition update");
+            {
+                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);
+            }
         }
 
         ByteBuffer collectionElement = metadata.isCompound() ? CompositeType.extractComponent(cellname, metadata.comparator.size() + 1) : null;
@@ -742,6 +750,13 @@ 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
+            }
         }
 
     }
@@ -1101,7 +1116,7 @@ public abstract class LegacyLayout
     }
 
     public static LegacyAtom readLegacyAtom(CFMetaData metadata, DataInputPlus in, boolean readAllAsDynamic)
-    throws IOException, UnknownColumnException
+    throws IOException, UnknownColumnException, IllegalLegacyColumnException
     {
         ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
         if (!cellname.hasRemaining())
@@ -1133,7 +1148,7 @@ public abstract class LegacyLayout
         }
     }
 
-    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException
+    public static LegacyCell readLegacyCell(CFMetaData metadata, DataInput in, SerializationHelper.Flag flag) throws IOException, UnknownColumnException, IllegalLegacyColumnException
     {
         ByteBuffer cellname = ByteBufferUtil.readWithShortLength(in);
         int b = in.readUnsignedByte();
@@ -1141,7 +1156,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
+    throws IOException, UnknownColumnException, IllegalLegacyColumnException
     {
         // 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.
@@ -1213,6 +1228,14 @@ 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);
@@ -1610,13 +1633,13 @@ public abstract class LegacyLayout
         }
 
         public static LegacyCell regular(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, ByteBuffer value, long timestamp)
-        throws UnknownColumnException
+        throws UnknownColumnException, IllegalLegacyColumnException
         {
             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
+        throws UnknownColumnException, IllegalLegacyColumnException
         {
             /*
              * CASSANDRA-14092: Max expiration date capping is maybe performed here, expiration overflow policy application
@@ -1626,13 +1649,13 @@ public abstract class LegacyLayout
         }
 
         public static LegacyCell tombstone(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
-        throws UnknownColumnException
+        throws UnknownColumnException, IllegalLegacyColumnException
         {
             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
+        throws UnknownColumnException, IllegalLegacyColumnException
         {
             // See UpdateParameters.addCounter() for more details on this
             ByteBuffer counterValue = CounterContext.instance().createUpdate(value);
@@ -2583,4 +2606,16 @@ 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 b499daf..eb5cddf 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 exc)
+                catch (UnknownColumnException | IllegalLegacyColumnException 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 62ad76a..50c9a1a 100644
--- a/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
+++ b/src/java/org/apache/cassandra/db/UnfilteredDeserializer.java
@@ -302,6 +302,13 @@ 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 55dc453..a480022 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 org.apache.cassandra.db.marshal.UTF8Type;
-import org.apache.cassandra.utils.ByteBufferUtil;
+
+import static org.apache.cassandra.db.LegacyLayout.stringify;
 
 /**
  * Exception thrown when we read a column internally that is unknown. Note that
@@ -36,16 +36,4 @@ 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 163eb2d..3a41df0 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 e)
+        catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+        catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+        catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+        catch (MarshalException | UnknownColumnException | IllegalLegacyColumnException 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 e)
+        catch (UnknownColumnException | IllegalLegacyColumnException 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
+    private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, Column column, int nowInSec) throws UnknownColumnException, IllegalLegacyColumnException
     {
         return toLegacyCell(metadata, null, column, nowInSec);
     }
 
     private LegacyLayout.LegacyCell toLegacyCell(CFMetaData metadata, ByteBuffer superColumnName, Column column, int nowInSec)
-    throws UnknownColumnException
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         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
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         return toLegacyDeletion(metadata, null, name, timestamp, nowInSec);
     }
 
     private LegacyLayout.LegacyCell toLegacyDeletion(CFMetaData metadata, ByteBuffer superColumnName, ByteBuffer name, long timestamp, int nowInSec)
-    throws UnknownColumnException
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         return LegacyLayout.LegacyCell.tombstone(metadata, superColumnName, name, timestamp, nowInSec);
     }
 
     private LegacyLayout.LegacyCell toCounterLegacyCell(CFMetaData metadata, CounterColumn column)
-    throws UnknownColumnException
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         return toCounterLegacyCell(metadata, null, column);
     }
 
     private LegacyLayout.LegacyCell toCounterLegacyCell(CFMetaData metadata, ByteBuffer superColumnName, CounterColumn column)
-    throws UnknownColumnException
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         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
+    throws UnknownColumnException, IllegalLegacyColumnException
     {
         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 e)
+        catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+                catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+            catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+            catch (UnknownColumnException | IllegalLegacyColumnException 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 e)
+            catch (MarshalException | UnknownColumnException | IllegalLegacyColumnException 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 2ab0330..e7a344a 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftValidation.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftValidation.java
@@ -31,6 +31,7 @@ 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;
@@ -269,7 +270,7 @@ public class ThriftValidation
                     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 e)
+                catch (IllegalArgumentException | UnknownColumnException | IllegalLegacyColumnException e )
                 {
                     throw new org.apache.cassandra.exceptions.InvalidRequestException(String.format("Error validating cell name for CQL3 table %s: %s", metadata.cfName, e.getMessage()));
                 }
@@ -466,7 +467,7 @@ public class ThriftValidation
             cn.column.validateCellValue(column.value);
 
         }
-        catch (UnknownColumnException e)
+        catch (UnknownColumnException | IllegalLegacyColumnException 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/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-CRC.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-CRC.db
new file mode 100644
index 0000000..ee733ee
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-CRC.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Data.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Data.db
new file mode 100644
index 0000000..6cf2e4c
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Data.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Digest.sha1 b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Digest.sha1
new file mode 100644
index 0000000..f419fd2
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Digest.sha1
@@ -0,0 +1 @@
+3673239127
\ No newline at end of file
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Filter.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Filter.db
new file mode 100644
index 0000000..f8e53be
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Filter.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Index.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Index.db
new file mode 100644
index 0000000..d6d8130
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Index.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Statistics.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Statistics.db
new file mode 100644
index 0000000..281b3da
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Statistics.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Summary.db b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Summary.db
new file mode 100644
index 0000000..f2a5cd5
Binary files /dev/null and b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-Summary.db differ
diff --git a/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-TOC.txt b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-TOC.txt
new file mode 100644
index 0000000..497e06b
--- /dev/null
+++ b/test/data/legacy-sstables/ka/legacy_tables/legacy_ka_with_illegal_cell_names/legacy_tables-legacy_ka_with_illegal_cell_names-ka-1-TOC.txt
@@ -0,0 +1,8 @@
+Statistics.db
+CRC.db
+Data.db
+TOC.txt
+Filter.db
+Index.db
+Digest.sha1
+Summary.db
diff --git a/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java b/test/unit/org/apache/cassandra/db/LegacyCellNameTest.java
index fa29b1e..c4d3988 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=IllegalArgumentException.class)
+    @Test(expected=IllegalLegacyColumnException.class)
     public void testColumnSameNameAsPartitionKeyCql3() throws Exception
     {
         CFMetaData cfm = CFMetaData.compile("CREATE TABLE cs (" +
@@ -67,6 +67,21 @@ public class LegacyCellNameTest
                                                       .fromString("k"));
     }
 
+    @Test(expected=IllegalLegacyColumnException.class)
+    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"));
+    }
+
+    // This throws IllegalArgumentException not because the cellname's value matches
+    // the clustering key name, but because when converted to a Composite, the buffer
+    // contains only a single component and so has no column name component
     @Test(expected=IllegalArgumentException.class)
     public void testColumnSameNameAsClusteringKeyCql3() throws Exception
     {
diff --git a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
index 4d99081..eeec76d 100644
--- a/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
+++ b/test/unit/org/apache/cassandra/io/sstable/LegacySSTableTest.java
@@ -63,6 +63,8 @@ import org.apache.cassandra.streaming.StreamSession;
 import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.FBUtilities;
 
+import static org.apache.cassandra.cql3.CQLTester.assertRows;
+import static org.apache.cassandra.cql3.CQLTester.row;
 /**
  * Tests backwards compatibility for SSTables
  */
@@ -406,6 +408,40 @@ public class LegacySSTableTest
         assertExpectedRowsWithDroppedCollection(false);
     }
 
+    @Test
+    public void testReadingLegacyTablesWithIllegalCellNames() throws Exception {
+        /**
+         * The sstable can be generated externally with SSTableSimpleUnsortedWriter:
+         *
+         * [
+         * {"key": "1",
+         *  "cells": [["a:aa:c1","61",1555000750634000],
+         *            ["a:aa:c2","6161",1555000750634000],
+         *            ["a:aa:pk","00000001",1555000750634000],
+         *            ["a:aa:v1","aaa",1555000750634000]]},
+         * {"key": "2",
+         *  "cells": [["b:bb:c1","62",1555000750634000],
+         *            ["b:bb:c2","6262",1555000750634000],
+         *            ["b:bb:pk","00000002",1555000750634000],
+         *            ["b:bb:v1","bbb",1555000750634000]]}
+         * ]
+         *
+         */
+
+        QueryProcessor.executeInternal("CREATE TABLE legacy_tables.legacy_ka_with_illegal_cell_names (" +
+                                       " 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"));
+
+        assertRows(results, row(1, "a", "aa", "aaa"), row(2, "b", "bb", "bbb"));
+    }
+
     private void assertExpectedRowsWithDroppedCollection(boolean droppedCheckSuccessful)
     {
         for (int i=0; i<=1; i++)


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