You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by bl...@apache.org on 2017/08/24 16:24:34 UTC

[4/8] cassandra git commit: Fix the merging of cells with different user type versions

Fix the merging of cells with different user type versions

patch by Benjamin Lerer; reviewed by Robert Stupp  for CASSANDRA-13776


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

Branch: refs/heads/trunk
Commit: cf0b6d107bade419dada49a5da40d2579c80ade8
Parents: 1a2ad2e
Author: Benjamin Lerer <b....@gmail.com>
Authored: Thu Aug 24 18:19:28 2017 +0200
Committer: Benjamin Lerer <b....@gmail.com>
Committed: Thu Aug 24 18:19:28 2017 +0200

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../cassandra/db/SerializationHeader.java       |  13 +-
 .../cassandra/db/marshal/AbstractType.java      |  10 +
 .../apache/cassandra/db/marshal/TupleType.java  |   5 +
 .../apache/cassandra/db/marshal/UserType.java   |  10 +
 .../db/rows/AbstractTypeVersionComparator.java  | 121 ++++++++++++
 src/java/org/apache/cassandra/db/rows/Row.java  |  18 +-
 src/java/org/apache/cassandra/db/rows/Rows.java |  21 ++-
 .../io/sstable/format/SSTableReader.java        |  23 +--
 .../org/apache/cassandra/cql3/CQLTester.java    |  20 +-
 .../cql3/validation/entities/UserTypesTest.java | 151 +++++++++++++++
 .../rows/AbstractTypeVersionComparatorTest.java | 184 +++++++++++++++++++
 12 files changed, 555 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index a35cc1b..ba35152 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.15
+ * Fix the merging of cells with different user type versions (CASSANDRA-13776)
  * Copy session properties on cqlsh.py do_login (CASSANDRA-13640)
  * Potential AssertionError during ReadRepair of range tombstone and partition deletions (CASSANDRA-13719)
  * Don't let stress write warmup data if n=0 (CASSANDRA-13773)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/SerializationHeader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SerializationHeader.java b/src/java/org/apache/cassandra/db/SerializationHeader.java
index 19dad95..494c2a3 100644
--- a/src/java/org/apache/cassandra/db/SerializationHeader.java
+++ b/src/java/org/apache/cassandra/db/SerializationHeader.java
@@ -108,7 +108,8 @@ public class SerializationHeader
         // but rather on their stats stored in StatsMetadata that are fully accurate.
         EncodingStats.Collector stats = new EncodingStats.Collector();
         PartitionColumns.Builder columns = PartitionColumns.builder();
-        for (SSTableReader sstable : sstables)
+        // We need to order the SSTables by descending generation to be sure that we use latest column definitions.
+        for (SSTableReader sstable : orderByDescendingGeneration(sstables))
         {
             stats.updateTimestamp(sstable.getMinTimestamp());
             stats.updateLocalDeletionTime(sstable.getMinLocalDeletionTime());
@@ -121,6 +122,16 @@ public class SerializationHeader
         return new SerializationHeader(true, metadata, columns.build(), stats.get());
     }
 
+    private static Collection<SSTableReader> orderByDescendingGeneration(Collection<SSTableReader> sstables)
+    {
+        if (sstables.size() < 2)
+            return sstables;
+
+        List<SSTableReader> readers = new ArrayList<>(sstables);
+        readers.sort(SSTableReader.generationReverseComparator);
+        return readers;
+    }
+
     public SerializationHeader(boolean isForSSTable,
                                CFMetaData metadata,
                                PartitionColumns columns,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/marshal/AbstractType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/AbstractType.java b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
index 77e0971..20062bd 100644
--- a/src/java/org/apache/cassandra/db/marshal/AbstractType.java
+++ b/src/java/org/apache/cassandra/db/marshal/AbstractType.java
@@ -325,6 +325,16 @@ public abstract class AbstractType<T> implements Comparator<ByteBuffer>
         return false;
     }
 
+    public boolean isTuple()
+    {
+        return false;
+    }
+
+    public boolean isUDT()
+    {
+        return false;
+    }
+
     public AbstractType<?> freeze()
     {
         return this;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/marshal/TupleType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/TupleType.java b/src/java/org/apache/cassandra/db/marshal/TupleType.java
index 2d6363e..5c74332 100644
--- a/src/java/org/apache/cassandra/db/marshal/TupleType.java
+++ b/src/java/org/apache/cassandra/db/marshal/TupleType.java
@@ -347,4 +347,9 @@ public class TupleType extends AbstractType<ByteBuffer>
     {
         return getClass().getName() + TypeParser.stringifyTypeParameters(types, true);
     }
+
+    public boolean isTuple()
+    {
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/marshal/UserType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/marshal/UserType.java b/src/java/org/apache/cassandra/db/marshal/UserType.java
index b91dbf8..03545ca 100644
--- a/src/java/org/apache/cassandra/db/marshal/UserType.java
+++ b/src/java/org/apache/cassandra/db/marshal/UserType.java
@@ -232,4 +232,14 @@ public class UserType extends TupleType
     {
         return serializer;
     }
+
+    public boolean isTuple()
+    {
+        return false;
+    }
+
+    public boolean isUDT()
+    {
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/rows/AbstractTypeVersionComparator.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/AbstractTypeVersionComparator.java b/src/java/org/apache/cassandra/db/rows/AbstractTypeVersionComparator.java
new file mode 100644
index 0000000..e47f681
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/rows/AbstractTypeVersionComparator.java
@@ -0,0 +1,121 @@
+/*
+ * 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.rows;
+
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.cassandra.db.marshal.*;
+
+/**
+ * A {@code Comparator} use to determine which version of a type should be used.
+ * <p>In the case of UDTs it is possible to have 2 versions or more of the same type, if some fields has been added to
+ * the type. To avoid problems the latest type need to be used.</p>
+ */
+final class AbstractTypeVersionComparator implements Comparator<AbstractType<?>>
+{
+    public static final Comparator<AbstractType<?>> INSTANCE = new AbstractTypeVersionComparator();
+
+    private AbstractTypeVersionComparator()
+    {
+    }
+
+    @Override
+    public int compare(AbstractType<?> type, AbstractType<?> otherType)
+    {
+        if (!type.getClass().equals(otherType.getClass()))
+            throw new IllegalArgumentException(String.format("Trying to compare 2 different types: %s and %s",
+                                                             type,
+                                                             otherType));
+
+        if (type.equals(otherType))
+            return 0;
+
+        // The only case where 2 types can differ is if they contains some UDTs and one of them has more
+        // fields (due to an ALTER type ADD) than in the other type. In this case we need to pick the type with
+        // the bigger amount of fields.
+        if (type.isUDT())
+            return compareUserType((UserType) type, (UserType) otherType);
+
+        if (type.isTuple())
+            return compareTuple((TupleType) type, (TupleType) otherType);
+
+        if (type.isCollection())
+            return compareCollectionTypes(type, otherType);
+
+        if (type instanceof CompositeType)
+            return compareCompositeTypes((CompositeType) type, (CompositeType) otherType);
+
+        // In theory we should never reach that point but to be on the safe side we allow it.
+        return 0;
+    }
+
+    private int compareCompositeTypes(CompositeType type, CompositeType otherType)
+    {
+        List<AbstractType<?>> types = type.getComponents();
+        List<AbstractType<?>> otherTypes = otherType.getComponents();
+
+        if (types.size() != otherTypes.size())
+            return Integer.compare(types.size(), otherTypes.size());
+
+        for (int i = 0, m = type.componentsCount(); i < m ; i++)
+        {
+            int test = compare(types.get(i), otherTypes.get(i));
+            if (test != 0);
+                return test;
+        }
+        return 0;
+    }
+
+    private int compareCollectionTypes(AbstractType<?> type, AbstractType<?> otherType)
+    {
+        if (type instanceof MapType)
+            return compareMapType((MapType<?, ?>) type, (MapType<?, ?>) otherType);
+
+        if (type instanceof SetType)
+            return compare(((SetType<?>) type).getElementsType(), ((SetType<?>) otherType).getElementsType());
+
+        return compare(((ListType<?>) type).getElementsType(), ((ListType<?>) otherType).getElementsType());
+    }
+
+    private int compareMapType(MapType<?, ?> type, MapType<?, ?> otherType)
+    {
+        int test = compare(type.getKeysType(), otherType.getKeysType());
+        return test != 0 ? test : compare(type.getValuesType(), otherType.getValuesType());
+    }
+
+    private int compareUserType(UserType type, UserType otherType)
+    {
+        return compareTuple(type, otherType);
+    }
+
+    private int compareTuple(TupleType type, TupleType otherType)
+    {
+        if (type.size() != otherType.size())
+            return Integer.compare(type.size(), otherType.size());
+
+        int test = 0;
+        int i = 0;
+        while (test == 0 && i < type.size())
+        {
+            test = compare(type.type(i), otherType.type(i));
+            i++;
+        }
+        return test;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/rows/Row.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Row.java b/src/java/org/apache/cassandra/db/rows/Row.java
index a61f365..9ab1f09 100644
--- a/src/java/org/apache/cassandra/db/rows/Row.java
+++ b/src/java/org/apache/cassandra/db/rows/Row.java
@@ -601,10 +601,25 @@ public interface Row extends Unfiltered, Collection<ColumnData>
 
             public void reduce(int idx, ColumnData data)
             {
-                column = data.column();
+                if (useColumnDefinition(data.column()))
+                    column = data.column();
+
                 versions.add(data);
             }
 
+            /**
+             * Determines it the {@code ColumnDefinition} is the one that should be used.
+             * @param dataColumn the {@code ColumnDefinition} to use.
+             * @return {@code true} if the {@code ColumnDefinition} is the one that should be used, {@code false} otherwise.
+             */
+            private boolean useColumnDefinition(ColumnDefinition dataColumn)
+            {
+                if (column == null)
+                    return true;
+
+                return AbstractTypeVersionComparator.INSTANCE.compare(column.type, dataColumn.type) < 0;
+            }
+
             protected ColumnData getReduced()
             {
                 if (column.isSimple())
@@ -654,6 +669,7 @@ public interface Row extends Unfiltered, Collection<ColumnData>
 
             protected void onKeyChange()
             {
+                column = null;
                 versions.clear();
             }
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/db/rows/Rows.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/rows/Rows.java b/src/java/org/apache/cassandra/db/rows/Rows.java
index e325091..09213a4 100644
--- a/src/java/org/apache/cassandra/db/rows/Rows.java
+++ b/src/java/org/apache/cassandra/db/rows/Rows.java
@@ -279,7 +279,8 @@ public abstract class Rows
             int comparison = nexta == null ? 1 : nextb == null ? -1 : nexta.column.compareTo(nextb.column);
             ColumnData cura = comparison <= 0 ? nexta : null;
             ColumnData curb = comparison >= 0 ? nextb : null;
-            ColumnDefinition column = (cura != null ? cura : curb).column;
+            ColumnDefinition column = getColumnDefinition(cura, curb);
+
             if (column.isSimple())
             {
                 timeDelta = Math.min(timeDelta, Cells.reconcile((Cell) cura, (Cell) curb, deletion, builder, nowInSec));
@@ -309,4 +310,22 @@ public abstract class Rows
         }
         return timeDelta;
     }
+
+    /**
+     * Returns the {@code ColumnDefinition} to use for merging the columns.
+     * If the 2 column definitions are different the latest one will be returned.
+     */
+    private static ColumnDefinition getColumnDefinition(ColumnData cura, ColumnData curb)
+    {
+        if (cura == null)
+            return curb.column;
+
+        if (curb == null)
+            return cura.column;
+
+        if (AbstractTypeVersionComparator.INSTANCE.compare(cura.column.type, curb.column.type) >= 0)
+            return cura.column;
+
+        return curb.column;
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
index f38738d..cd41b5b 100644
--- a/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
+++ b/src/java/org/apache/cassandra/io/sstable/format/SSTableReader.java
@@ -45,7 +45,6 @@ import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.filter.ColumnFilter;
 import org.apache.cassandra.db.rows.SliceableUnfilteredRowIterator;
 import org.apache.cassandra.dht.AbstractBounds;
@@ -141,26 +140,14 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     }
     private static final RateLimiter meterSyncThrottle = RateLimiter.create(100.0);
 
-    public static final Comparator<SSTableReader> maxTimestampComparator = new Comparator<SSTableReader>()
-    {
-        public int compare(SSTableReader o1, SSTableReader o2)
-        {
-            long ts1 = o1.getMaxTimestamp();
-            long ts2 = o2.getMaxTimestamp();
-            return (ts1 > ts2 ? -1 : (ts1 == ts2 ? 0 : 1));
-        }
-    };
+    public static final Comparator<SSTableReader> maxTimestampComparator = (o1, o2) -> Long.compare(o1.getMaxTimestamp(), o2.getMaxTimestamp());
 
     // it's just an object, which we use regular Object equality on; we introduce a special class just for easy recognition
     public static final class UniqueIdentifier {}
 
-    public static final Comparator<SSTableReader> sstableComparator = new Comparator<SSTableReader>()
-    {
-        public int compare(SSTableReader o1, SSTableReader o2)
-        {
-            return o1.first.compareTo(o2.first);
-        }
-    };
+    public static final Comparator<SSTableReader> sstableComparator = (o1, o2) -> o1.first.compareTo(o2.first);
+
+    public static final Comparator<SSTableReader> generationReverseComparator = (o1, o2) -> -Integer.compare(o1.descriptor.generation, o2.descriptor.generation);
 
     public static final Ordering<SSTableReader> sstableOrdering = Ordering.from(sstableComparator);
 
@@ -1717,7 +1704,7 @@ public abstract class SSTableReader extends SSTable implements SelfRefCounted<SS
     /**
      * Direct I/O SSTableScanner over an iterator of bounds.
      *
-     * @param bounds the keys to cover
+     * @param rangeIterator the keys to cover
      * @return A Scanner for seeking over the rows of the SSTable.
      */
     public abstract ISSTableScanner getScanner(Iterator<AbstractBounds<PartitionPosition>> rangeIterator);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index ba23c67..40aec88 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -396,7 +396,8 @@ public abstract class CQLTester
     public void disableCompaction(String keyspace)
     {
         ColumnFamilyStore store = getCurrentColumnFamilyStore(keyspace);
-        store.disableAutoCompaction();
+        if (store != null)
+            store.disableAutoCompaction();
     }
 
     public void flush(boolean forceFlush)
@@ -437,6 +438,23 @@ public abstract class CQLTester
         }
     }
 
+    public void disableCompaction()
+    {
+        disableCompaction(KEYSPACE);
+    }
+
+    public void enableCompaction(String keyspace)
+    {
+        ColumnFamilyStore store = getCurrentColumnFamilyStore(keyspace);
+        if (store != null)
+            store.enableAutoCompaction();
+    }
+
+    public void enableCompaction()
+    {
+        enableCompaction(KEYSPACE);
+    }
+
     public void cleanupCache()
     {
         ColumnFamilyStore store = getCurrentColumnFamilyStore();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
index c279e00..dfc2e5e 100644
--- a/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/validation/entities/UserTypesTest.java
@@ -562,6 +562,157 @@ public class UserTypesTest extends CQLTester
         assertInvalidMessage("Cannot drop user type " + typeWithKs(t), "DROP TYPE " + typeWithKs(t) + ';');
     }
 
+    @Test
+    public void testReadAfterAlteringUserTypeNestedWithinSet() throws Throwable
+    {
+        String columnType = typeWithKs(createType("CREATE TYPE %s (a int)"));
+
+        try
+        {
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y set<frozen<" + columnType + ">>)");
+            disableCompaction();
+
+            execute("INSERT INTO %s (x, y) VALUES(1, ?)", set(userType(1), userType(2)));
+            assertRows(execute("SELECT * FROM %s"), row(1, set(userType(1), userType(2))));
+            flush();
+
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                       row(1, set(userType(1), userType(2))));
+
+            execute("ALTER TYPE " + columnType + " ADD b int");
+            execute("UPDATE %s SET y = y + ? WHERE x = 1",
+                    set(userType(1, 1), userType(1, 2), userType(2, 1)));
+
+            flush();
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                           row(1, set(userType(1),
+                                      userType(1, 1),
+                                      userType(1, 2),
+                                      userType(2),
+                                      userType(2, 1))));
+
+            compact();
+
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                       row(1, set(userType(1),
+                                  userType(1, 1),
+                                  userType(1, 2),
+                                  userType(2),
+                                  userType(2, 1))));
+        }
+        finally
+        {
+            enableCompaction();
+        }
+    }
+
+    @Test
+    public void testReadAfterAlteringUserTypeNestedWithinMap() throws Throwable
+    {
+        String columnType = typeWithKs(createType("CREATE TYPE %s (a int)"));
+
+        try
+        {
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y map<frozen<" + columnType + ">, int>)");
+            disableCompaction();
+
+            execute("INSERT INTO %s (x, y) VALUES(1, ?)", map(userType(1), 1, userType(2), 2));
+            assertRows(execute("SELECT * FROM %s"), row(1, map(userType(1), 1, userType(2), 2)));
+            flush();
+
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                       row(1, map(userType(1), 1, userType(2), 2)));
+
+            execute("ALTER TYPE " + columnType + " ADD b int");
+            execute("UPDATE %s SET y = y + ? WHERE x = 1",
+                    map(userType(1, 1), 1, userType(1, 2), 1, userType(2, 1), 2));
+
+            flush();
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                           row(1, map(userType(1), 1,
+                                      userType(1, 1), 1,
+                                      userType(1, 2), 1,
+                                      userType(2), 2,
+                                      userType(2, 1), 2)));
+
+            compact();
+
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                       row(1, map(userType(1), 1,
+                                  userType(1, 1), 1,
+                                  userType(1, 2), 1,
+                                  userType(2), 2,
+                                  userType(2, 1), 2)));
+        }
+        finally
+        {
+            enableCompaction();
+        }
+    }
+
+    @Test
+    public void testReadAfterAlteringUserTypeNestedWithinList() throws Throwable
+    {
+        String columnType = typeWithKs(createType("CREATE TYPE %s (a int)"));
+
+        try
+        {
+            createTable("CREATE TABLE %s (x int PRIMARY KEY, y list<frozen<" + columnType + ">>)");
+            disableCompaction();
+
+            execute("INSERT INTO %s (x, y) VALUES(1, ?)", list(userType(1), userType(2)));
+            assertRows(execute("SELECT * FROM %s"), row(1, list(userType(1), userType(2))));
+            flush();
+
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                       row(1, list(userType(1), userType(2))));
+
+            execute("ALTER TYPE " + columnType + " ADD b int");
+            execute("UPDATE %s SET y = y + ? WHERE x = 1",
+                    list(userType(1, 1), userType(1, 2), userType(2, 1)));
+
+            flush();
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                           row(1, list(userType(1),
+                                       userType(2),
+                                       userType(1, 1),
+                                       userType(1, 2),
+                                       userType(2, 1))));
+
+            compact();
+
+            assertRows(execute("SELECT * FROM %s WHERE x = 1"),
+                       row(1, list(userType(1),
+                                   userType(2),
+                                   userType(1, 1),
+                                   userType(1, 2),
+                                   userType(2, 1))));
+        }
+        finally
+        {
+            enableCompaction();
+        }
+    }
+
+    @Test
+    public void testAlteringUserTypeNestedWithinSetWithView() throws Throwable
+    {
+        String columnType = typeWithKs(createType("CREATE TYPE %s (a int)"));
+
+        createTable("CREATE TABLE %s (pk int, c int, v int, s set<frozen<" + columnType + ">>, PRIMARY KEY (pk, c))");
+        execute("CREATE MATERIALIZED VIEW " + keyspace() + ".view1 AS SELECT c, pk, v FROM %s WHERE pk IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL PRIMARY KEY (c, pk)");
+
+        execute("INSERT INTO %s (pk, c, v, s) VALUES(?, ?, ?, ?)", 1, 1, 1, set(userType(1), userType(2)));
+        flush();
+
+        execute("ALTER TYPE " + columnType + " ADD b int");
+        execute("UPDATE %s SET s = s + ?, v = ? WHERE pk = ? AND c = ?",
+                set(userType(1, 1), userType(1, 2), userType(2, 1)), 2, 1, 1);
+
+        assertRows(execute("SELECT * FROM %s WHERE pk = ? AND c = ?", 1, 1),
+                       row(1, 1,set(userType(1), userType(1, 1), userType(1, 2), userType(2), userType(2, 1)), 2));
+    }
+
     private String typeWithKs(String type1)
     {
         return keyspace() + '.' + type1;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cf0b6d10/test/unit/org/apache/cassandra/db/rows/AbstractTypeVersionComparatorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/rows/AbstractTypeVersionComparatorTest.java b/test/unit/org/apache/cassandra/db/rows/AbstractTypeVersionComparatorTest.java
new file mode 100644
index 0000000..ad0c05c
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/rows/AbstractTypeVersionComparatorTest.java
@@ -0,0 +1,184 @@
+/*
+ * 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.rows;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import org.apache.cassandra.db.marshal.*;
+
+import static java.util.Arrays.asList;
+import static org.apache.cassandra.utils.ByteBufferUtil.bytes;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class AbstractTypeVersionComparatorTest
+{
+    private UserType udtWith2Fields;
+    private UserType udtWith3Fields;
+
+    @Before
+    public void setUp()
+    {
+        udtWith2Fields = new UserType("ks",
+                                      bytes("myType"),
+                                      asList(bytes("a"), bytes("b")),
+                                      asList(Int32Type.instance, Int32Type.instance));
+        udtWith3Fields = new UserType("ks",
+                                      bytes("myType"),
+                                      asList(bytes("a"), bytes("b"), bytes("c")),
+                                      asList(Int32Type.instance, Int32Type.instance, Int32Type.instance));
+    }
+
+    @After
+    public void tearDown()
+    {
+        udtWith2Fields = null;
+        udtWith3Fields = null;
+    }
+
+    @Test
+    public void testWithTuples()
+    {
+        checkComparisonResults(new TupleType(asList(Int32Type.instance, Int32Type.instance)),
+                               new TupleType(asList(Int32Type.instance, Int32Type.instance, Int32Type.instance)));
+    }
+
+    @Test
+    public void testWithUDTs()
+    {
+        checkComparisonResults(udtWith2Fields, udtWith3Fields);
+    }
+
+    @Test
+    public void testWithUDTsNestedWithinSet()
+    {
+        for (boolean isMultiCell : new boolean[]{false, true})
+        {
+            SetType<ByteBuffer> set1 = SetType.getInstance(udtWith2Fields, isMultiCell);
+            SetType<ByteBuffer> set2 = SetType.getInstance(udtWith3Fields, isMultiCell);
+            checkComparisonResults(set1, set2);
+        }
+    }
+
+    @Test
+    public void testWithUDTsNestedWithinList()
+    {
+        for (boolean isMultiCell : new boolean[]{false, true})
+        {
+            ListType<ByteBuffer> list1 = ListType.getInstance(udtWith2Fields, isMultiCell);
+            ListType<ByteBuffer> list2 = ListType.getInstance(udtWith3Fields, isMultiCell);
+            checkComparisonResults(list1, list2);
+        }
+    }
+
+    @Test
+    public void testWithUDTsNestedWithinMap()
+    {
+        for (boolean isMultiCell : new boolean[]{false, true})
+        {
+            MapType<ByteBuffer, Integer> map1 = MapType.getInstance(udtWith2Fields, Int32Type.instance, isMultiCell);
+            MapType<ByteBuffer, Integer> map2 = MapType.getInstance(udtWith3Fields, Int32Type.instance, isMultiCell);
+            checkComparisonResults(map1, map2);
+        }
+
+        for (boolean isMultiCell : new boolean[]{false, true})
+        {
+            MapType<Integer, ByteBuffer> map1 = MapType.getInstance(Int32Type.instance, udtWith2Fields, isMultiCell);
+            MapType<Integer, ByteBuffer> map2 = MapType.getInstance(Int32Type.instance, udtWith3Fields, isMultiCell);
+            checkComparisonResults(map1, map2);
+        }
+    }
+
+    @Test
+    public void testWithUDTsNestedWithinTuple()
+    {
+        TupleType tuple1 = new TupleType(asList(udtWith2Fields, Int32Type.instance));
+        TupleType tuple2 = new TupleType(asList(udtWith3Fields, Int32Type.instance));
+        checkComparisonResults(tuple1, tuple2);
+    }
+
+    @Test
+    public void testWithUDTsNestedWithinComposite()
+    {
+        CompositeType composite1 = CompositeType.getInstance(asList(udtWith2Fields, Int32Type.instance));
+        CompositeType composite2 = CompositeType.getInstance(asList(udtWith3Fields, Int32Type.instance));
+        checkComparisonResults(composite1, composite2);
+    }
+
+    @Test
+    public void testWithDeeplyNestedUDT()
+    {
+        for (boolean isMultiCell : new boolean[]{false, true})
+        {
+            ListType<Set<ByteBuffer>> list1 = ListType.getInstance(SetType.getInstance(new TupleType(asList(udtWith2Fields, Int32Type.instance)), isMultiCell), isMultiCell);
+            ListType<Set<ByteBuffer>> list2 = ListType.getInstance(SetType.getInstance(new TupleType(asList(udtWith3Fields, Int32Type.instance)), isMultiCell), isMultiCell);
+            checkComparisonResults(list1, list2);
+        }
+    }
+
+    @Test
+    public void testInvalidComparison()
+    {
+        assertInvalidComparison("Trying to compare 2 different types: org.apache.cassandra.db.marshal.UserType(ks,6d7954797065,61:org.apache.cassandra.db.marshal.Int32Type,62:org.apache.cassandra.db.marshal.Int32Type) and org.apache.cassandra.db.marshal.Int32Type",
+                                udtWith2Fields,
+                                Int32Type.instance);
+        assertInvalidComparison("Trying to compare 2 different types: org.apache.cassandra.db.marshal.UTF8Type and org.apache.cassandra.db.marshal.InetAddressType",
+                                SetType.getInstance(UTF8Type.instance, true),
+                                SetType.getInstance(InetAddressType.instance, true));
+        assertInvalidComparison("Trying to compare 2 different types: org.apache.cassandra.db.marshal.UTF8Type and org.apache.cassandra.db.marshal.InetAddressType",
+                                ListType.getInstance(UTF8Type.instance, true),
+                                ListType.getInstance(InetAddressType.instance, true));
+        assertInvalidComparison("Trying to compare 2 different types: org.apache.cassandra.db.marshal.UTF8Type and org.apache.cassandra.db.marshal.InetAddressType",
+                                MapType.getInstance(UTF8Type.instance, IntegerType.instance, true),
+                                MapType.getInstance(InetAddressType.instance, IntegerType.instance, true));
+        assertInvalidComparison("Trying to compare 2 different types: org.apache.cassandra.db.marshal.UTF8Type and org.apache.cassandra.db.marshal.InetAddressType",
+                                MapType.getInstance(IntegerType.instance, UTF8Type.instance, true),
+                                MapType.getInstance(IntegerType.instance, InetAddressType.instance, true));
+    }
+
+    private void assertInvalidComparison(String expectedMessage, AbstractType<?> oldVersion, AbstractType<?> newVersion)
+    {
+        try
+        {
+            checkComparisonResults(oldVersion, newVersion);
+            fail("comparison doesn't throw expected IllegalArgumentException: " + expectedMessage);
+        }
+        catch (IllegalArgumentException e)
+        {
+            assertEquals(e.getMessage(), expectedMessage);
+        }
+    }
+
+    private void checkComparisonResults(AbstractType<?> oldVersion, AbstractType<?> newVersion)
+    {
+        assertEquals(0, compare(oldVersion, oldVersion));
+        assertEquals(0, compare(newVersion, newVersion));
+        assertEquals(-1, compare(oldVersion, newVersion));
+        assertEquals(1, compare(newVersion, oldVersion));
+    }
+
+    private int compare(AbstractType<?> left, AbstractType<?> right)
+    {
+        return AbstractTypeVersionComparator.INSTANCE.compare(left, right);
+    }
+}


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