You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by al...@apache.org on 2015/09/16 16:35:36 UTC

[1/7] cassandra git commit: Improve MV schema representation

Repository: cassandra
Updated Branches:
  refs/heads/trunk 2f82a69ea -> 43ddec31b


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/cql3/ViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
new file mode 100644
index 0000000..9c47707
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -0,0 +1,1355 @@
+/*
+ * 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;
+
+import java.math.BigDecimal;
+import java.math.BigInteger;
+import java.net.InetAddress;
+import java.text.SimpleDateFormat;
+import java.util.*;
+import java.util.concurrent.TimeUnit;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.*;
+import com.datastax.driver.core.ResultSet;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
+import junit.framework.Assert;
+import org.apache.cassandra.concurrent.SEPExecutor;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.serializers.SimpleDateSerializer;
+import org.apache.cassandra.serializers.TimeSerializer;
+import org.apache.cassandra.utils.ByteBufferUtil;
+import org.apache.cassandra.utils.FBUtilities;
+
+public class ViewTest extends CQLTester
+{
+    int protocolVersion = 4;
+    private final List<String> views = new ArrayList<>();
+
+    @BeforeClass
+    public static void startup()
+    {
+        requireNetwork();
+    }
+    @Before
+    public void begin()
+    {
+        views.clear();
+    }
+
+    @After
+    public void end() throws Throwable
+    {
+        for (String viewName : views)
+            executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + viewName);
+    }
+
+    private void createView(String name, String query) throws Throwable
+    {
+        executeNet(protocolVersion, String.format(query, name));
+        // If exception is thrown, the view will not be added to the list; since it shouldn't have been created, this is
+        // the desired behavior
+        views.add(name);
+    }
+
+    private void updateView(String query, Object... params) throws Throwable
+    {
+        executeNet(protocolVersion, query, params);
+        while (!(((SEPExecutor) StageManager.getStage(Stage.VIEW_MUTATION)).getPendingTasks() == 0
+                 && ((SEPExecutor) StageManager.getStage(Stage.VIEW_MUTATION)).getActiveCount() == 0))
+        {
+            Thread.sleep(1);
+        }
+    }
+
+    @Test
+    public void testCaseSensitivity() throws Throwable
+    {
+        createTable("CREATE TABLE %s (\"theKey\" int, \"theClustering\" int, \"theValue\" int, PRIMARY KEY (\"theKey\", \"theClustering\"))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        execute("INSERT INTO %s (\"theKey\", \"theClustering\", \"theValue\") VALUES (?, ?, ?)", 0, 0, 0);
+
+        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+                              "WHERE \"theKey\" IS NOT NULL AND \"theClustering\" IS NOT NULL AND \"theValue\" IS NOT NULL " +
+                              "PRIMARY KEY (\"theKey\", \"theClustering\")");
+
+        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test"))
+            Thread.sleep(10);
+        createView("mv_test2", "CREATE MATERIALIZED VIEW %s AS SELECT \"theKey\", \"theClustering\", \"theValue\" FROM %%s " +
+                               "WHERE \"theKey\" IS NOT NULL AND \"theClustering\" IS NOT NULL AND \"theValue\" IS NOT NULL " +
+                               "PRIMARY KEY (\"theKey\", \"theClustering\")");
+        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test2"))
+            Thread.sleep(10);
+
+        for (String mvname : Arrays.asList("mv_test", "mv_test2"))
+        {
+            assertRows(execute("SELECT \"theKey\", \"theClustering\", \"theValue\" FROM " + mvname),
+                       row(0, 0, 0)
+            );
+        }
+
+        executeNet(protocolVersion, "ALTER TABLE %s RENAME \"theClustering\" TO \"Col\"");
+
+        for (String mvname : Arrays.asList("mv_test", "mv_test2"))
+        {
+            assertRows(execute("SELECT \"theKey\", \"Col\", \"theValue\" FROM " + mvname),
+                       row(0, 0, 0)
+            );
+        }
+    }
+
+    @Test
+    public void testPartitionTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+
+        updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
+        updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
+
+        Assert.assertEquals(1, execute("select * from %s").size());
+        Assert.assertEquals(1, execute("select * from view1").size());
+
+        updateView("DELETE FROM %s WHERE k1 = 1");
+
+        Assert.assertEquals(0, execute("select * from %s").size());
+        Assert.assertEquals(0, execute("select * from view1").size());
+    }
+
+    @Test
+    public void testClusteringKeyTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+
+        updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
+        updateView("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
+
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from view1").size());
+
+        updateView("DELETE FROM %s WHERE k1 = 1 and c1 = 3");
+
+        Assert.assertEquals(1, execute("select * from %s").size());
+        Assert.assertEquals(1, execute("select * from view1").size());
+    }
+
+    @Test
+    public void testPrimaryKeyIsNotNull() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "PRIMARY KEY((k, asciival)))");
+
+        // Must include "IS NOT NULL" for primary keys
+        try
+        {
+            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s");
+            Assert.fail("Should fail if no primary key is filtered as NOT NULL");
+        }
+        catch (Exception e)
+        {
+        }
+
+        // Must include both when the partition key is composite
+        try
+        {
+            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
+            Assert.fail("Should fail if compound primary is not completely filtered as NOT NULL");
+        }
+        catch (Exception e)
+        {
+        }
+
+        dropTable("DROP TABLE %s");
+
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "PRIMARY KEY(k, asciival))");
+        try
+        {
+            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s");
+            Assert.fail("Should fail if no primary key is filtered as NOT NULL");
+        }
+        catch (Exception e)
+        {
+        }
+
+        // Can omit "k IS NOT NULL" because we have a sinlge partition key
+        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
+    }
+
+    @Test
+    public void testAccessAndSchema() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "PRIMARY KEY((k, asciival)))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv1_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
+        updateView("INSERT INTO %s(k,asciival,bigintval)VALUES(?,?,?)", 0, "foo", 1L);
+
+        try
+        {
+            updateView("INSERT INTO mv1_test(k,asciival,bigintval) VALUES(?,?,?)", 1, "foo", 2L);
+            Assert.fail("Shouldn't be able to modify a MV directly");
+        }
+        catch (Exception e)
+        {
+        }
+
+        try
+        {
+            executeNet(protocolVersion, "ALTER TABLE mv1_test ADD foo text");
+            Assert.fail("Should not be able to use alter table with MV");
+        }
+        catch (Exception e)
+        {
+        }
+
+        try
+        {
+            executeNet(protocolVersion, "ALTER TABLE mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
+            Assert.fail("Should not be able to use alter table with MV");
+        }
+        catch (Exception e)
+        {
+        }
+
+        executeNet(protocolVersion, "ALTER MATERIALIZED VIEW mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
+
+        //Test alter add
+        executeNet(protocolVersion, "ALTER TABLE %s ADD foo text");
+        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test");
+        Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("foo")));
+
+        updateView("INSERT INTO %s(k,asciival,bigintval,foo)VALUES(?,?,?,?)", 0, "foo", 1L, "bar");
+        assertRows(execute("SELECT foo from %s"), row("bar"));
+
+        //Test alter rename
+        executeNet(protocolVersion, "ALTER TABLE %s RENAME asciival TO bar");
+
+        assertRows(execute("SELECT bar from %s"), row("foo"));
+        metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test");
+        Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("bar")));
+    }
+
+
+    @Test
+    public void testStaticTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "c int, " +
+                    "sval text static, " +
+                    "val text, " +
+                    "PRIMARY KEY(k,c))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        try
+        {
+            createView("mv_static", "CREATE MATERIALIZED VIEW %%s AS SELECT * FROM %s WHERE sval IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (sval,k,c)");
+            Assert.fail("MV on static should fail");
+        }
+        catch (InvalidQueryException e)
+        {
+        }
+
+        createView("mv_static", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
+
+        for (int i = 0; i < 100; i++)
+            updateView("INSERT into %s (k,c,sval,val)VALUES(?,?,?,?)", 0, i % 2, "bar" + i, "baz");
+
+        Assert.assertEquals(2, execute("select * from %s").size());
+
+        assertRows(execute("SELECT sval from %s"), row("bar99"), row("bar99"));
+
+        Assert.assertEquals(2, execute("select * from mv_static").size());
+
+        assertInvalid("SELECT sval from mv_static");
+    }
+
+
+    @Test
+    public void testOldTimestamps() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "c int, " +
+                    "val text, " +
+                    "PRIMARY KEY(k,c))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv_tstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
+
+        for (int i = 0; i < 100; i++)
+            updateView("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz");
+
+        Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush();
+
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from mv_tstest").size());
+
+        assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz"));
+        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1));
+
+        //Make sure an old TS does nothing
+        updateView("UPDATE %s USING TIMESTAMP 100 SET val = ? where k = ? AND c = ?", "bar", 0, 0);
+        assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz"));
+        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1));
+        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"));
+
+        //Latest TS
+        updateView("UPDATE %s SET val = ? where k = ? AND c = ?", "bar", 0, 0);
+        assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("bar"));
+        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"), row(0));
+        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(1));
+    }
+
+    @Test
+    public void testCountersTable() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int PRIMARY KEY, " +
+                    "count counter)");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        try
+        {
+            createView("mv_counter", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE count IS NOT NULL AND k IS NOT NULL PRIMARY KEY (count,k)");
+            Assert.fail("MV on counter should fail");
+        }
+        catch (InvalidQueryException e)
+        {
+        }
+    }
+
+    @Test
+    public void complexTimestampUpdateTestWithFlush() throws Throwable
+    {
+        complexTimestampUpdateTest(true);
+    }
+
+    @Test
+    public void complexTimestampUpdateTestWithoutFlush() throws Throwable
+    {
+        complexTimestampUpdateTest(false);
+    }
+
+    public void complexTimestampUpdateTest(boolean flush) throws Throwable
+    {
+        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+        Keyspace ks = Keyspace.open(keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, a, b)");
+        ks.getColumnFamilyStore("mv").disableAutoCompaction();
+
+        //Set initial values TS=0, leaving e null and verify view
+        executeNet(protocolVersion, "INSERT INTO %s (a, b, c, d) VALUES (0, 0, 1, 0) USING TIMESTAMP 0");
+        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
+
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+        //update c's timestamp TS=2
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
+
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+            //change c's value and TS=3, tombstones c=1 and adds c=0 record
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? and b = ? ", 0, 0, 0);
+        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0));
+
+        if(flush)
+        {
+            ks.getColumnFamilyStore("mv").forceMajorCompaction();
+            FBUtilities.waitOnFutures(ks.flush());
+        }
+
+
+        //change c's value back to 1 with TS=4, check we can see d
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        if (flush)
+        {
+            ks.getColumnFamilyStore("mv").forceMajorCompaction();
+            FBUtilities.waitOnFutures(ks.flush());
+        }
+
+        assertRows(execute("SELECT d,e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, null));
+
+
+            //Add e value @ TS=1
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 1 SET e = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        assertRows(execute("SELECT d,e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, 1));
+
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+
+        //Change d value @ TS=2
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET d = ? WHERE a = ? and b = ? ", 2, 0, 0);
+        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(2));
+
+        if (flush)
+            FBUtilities.waitOnFutures(ks.flush());
+
+
+        //Change d value @ TS=3
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(1));
+
+
+        //Tombstone c
+        executeNet(protocolVersion, "DELETE FROM %s WHERE a = ? and b = ?", 0, 0);
+        assertRows(execute("SELECT d from mv"));
+
+        //Add back without D
+        executeNet(protocolVersion, "INSERT INTO %s (a, b, c) VALUES (0, 0, 1)");
+
+        //Make sure D doesn't pop back in.
+        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row((Object) null));
+
+
+        //New partition
+        // insert a row with timestamp 0
+        executeNet(protocolVersion, "INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP 0", 1, 0, 0, 0, 0);
+
+        // overwrite pk and e with timestamp 1, but don't overwrite d
+        executeNet(protocolVersion, "INSERT INTO %s (a, b, c, e) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 0, 0, 0);
+
+        // delete with timestamp 0 (which should only delete d)
+        executeNet(protocolVersion, "DELETE FROM %s USING TIMESTAMP 0 WHERE a = ? AND b = ?", 1, 0);
+        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 0, 1, 0),
+                   row(1, 0, 0, null, 0)
+        );
+
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? AND b = ?", 1, 1, 0);
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? AND b = ?", 0, 1, 0);
+        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 0, 1, 0),
+                   row(1, 0, 0, null, 0)
+        );
+
+        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? AND b = ?", 0, 1, 0);
+        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 0, 1, 0),
+                   row(1, 0, 0, 0, 0)
+        );
+
+
+    }
+
+    @Test
+    public void testBuilderWidePartition() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "c int, " +
+                    "intval int, " +
+                    "PRIMARY KEY (k, c))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+
+        for(int i = 0; i < 1024; i++)
+            execute("INSERT INTO %s (k, c, intval) VALUES (?, ?, ?)", 0, i, 0);
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, c, k)");
+
+
+        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv"))
+            Thread.sleep(1000);
+
+        assertRows(execute("SELECT count(*) from %s WHERE k = ?", 0), row(1024L));
+        assertRows(execute("SELECT count(*) from mv WHERE intval = ?", 0), row(1024L));
+    }
+
+    @Test
+    public void testRangeTombstone() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "textval1 text, " +
+                    "textval2 text, " +
+                    "PRIMARY KEY((k, asciival), bigintval, textval1)" +
+                    ")");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv_test1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), asciival, bigintval, textval1)");
+
+        for (int i = 0; i < 100; i++)
+            updateView("INSERT into %s (k,asciival,bigintval,textval1,textval2)VALUES(?,?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i, "baz");
+
+        Assert.assertEquals(50, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size());
+        Assert.assertEquals(50, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size());
+
+        Assert.assertEquals(100, execute("select * from mv_test1").size());
+
+        //Check the builder works
+        createView("mv_test2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), asciival, bigintval, textval1)");
+
+        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test2"))
+            Thread.sleep(1000);
+
+        Assert.assertEquals(100, execute("select * from mv_test2").size());
+
+        createView("mv_test3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), bigintval, textval1, asciival)");
+
+        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test3"))
+            Thread.sleep(1000);
+
+        Assert.assertEquals(100, execute("select * from mv_test3").size());
+        Assert.assertEquals(100, execute("select asciival from mv_test3 where textval2 = ? and k = ?", "baz", 0).size());
+
+        //Write a RT and verify the data is removed from index
+        updateView("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval = ?", 0, "foo", 0L);
+
+        Assert.assertEquals(50, execute("select asciival from mv_test3 where textval2 = ? and k = ?", "baz", 0).size());
+    }
+
+
+    @Test
+    public void testRangeTombstone2() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "textval1 text, " +
+                    "PRIMARY KEY((k, asciival), bigintval)" +
+                    ")");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval1 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL PRIMARY KEY ((textval1, k), asciival, bigintval)");
+
+        for (int i = 0; i < 100; i++)
+            updateView("INSERT into %s (k,asciival,bigintval,textval1)VALUES(?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i);
+
+        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size());
+        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size());
+
+
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from mv").size());
+
+        //Write a RT and verify the data is removed from index
+        updateView("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval = ?", 0, "foo", 0L);
+
+        Assert.assertEquals(1, execute("select * from %s").size());
+        Assert.assertEquals(1, execute("select * from mv").size());
+    }
+
+    @Test
+    public void testRangeTombstone3() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "textval1 text, " +
+                    "PRIMARY KEY((k, asciival), bigintval)" +
+                    ")");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval1 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL PRIMARY KEY ((textval1, k), asciival, bigintval)");
+
+        for (int i = 0; i < 100; i++)
+            updateView("INSERT into %s (k,asciival,bigintval,textval1)VALUES(?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i);
+
+        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size());
+        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size());
+
+
+        Assert.assertEquals(2, execute("select * from %s").size());
+        Assert.assertEquals(2, execute("select * from mv").size());
+
+        //Write a RT and verify the data is removed from index
+        updateView("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval >= ?", 0, "foo", 0L);
+
+        Assert.assertEquals(0, execute("select * from %s").size());
+        Assert.assertEquals(0, execute("select * from mv").size());
+    }
+
+    @Test
+    public void testCompoundPartitionKey() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "PRIMARY KEY((k, asciival)))");
+
+        CFMetaData metadata = currentTableMetadata();
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        for (ColumnDefinition def : new HashSet<>(metadata.allColumns()))
+        {
+            try
+            {
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ("
+                               + def.name + ", k" + (def.name.toString().equals("asciival") ? "" : ", asciival") + ")";
+                createView("mv1_" + def.name, query);
+
+                if (def.type.isMultiCell())
+                    Assert.fail("MV on a multicell should fail " + def);
+            }
+            catch (InvalidQueryException e)
+            {
+                if (!def.type.isMultiCell() && !def.isPartitionKey())
+                    Assert.fail("MV creation failed on " + def);
+            }
+
+
+            try
+            {
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + " PRIMARY KEY ("
+                               + def.name + ", asciival" + (def.name.toString().equals("k") ? "" : ", k") + ")";
+                createView("mv2_" + def.name, query);
+
+                if (def.type.isMultiCell())
+                    Assert.fail("MV on a multicell should fail " + def);
+            }
+            catch (InvalidQueryException e)
+            {
+                if (!def.type.isMultiCell() && !def.isPartitionKey())
+                    Assert.fail("MV creation failed on " + def);
+            }
+
+            try
+            {
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), asciival)";
+                createView("mv3_" + def.name, query);
+
+                if (def.type.isMultiCell())
+                    Assert.fail("MV on a multicell should fail " + def);
+            }
+            catch (InvalidQueryException e)
+            {
+                if (!def.type.isMultiCell() && !def.isPartitionKey())
+                    Assert.fail("MV creation failed on " + def);
+            }
+
+
+            try
+            {
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), asciival)";
+                createView("mv3_" + def.name, query);
+
+                Assert.fail("Should fail on duplicate name");
+            }
+            catch (Exception e)
+            {
+            }
+
+            try
+            {
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), nonexistentcolumn)";
+                createView("mv3_" + def.name, query);
+                Assert.fail("Should fail with unknown base column");
+            }
+            catch (InvalidQueryException e)
+            {
+            }
+        }
+
+        updateView("INSERT INTO %s (k, asciival, bigintval) VALUES (?, ?, fromJson(?))", 0, "ascii text", "123123123123");
+        updateView("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
+        assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"), row(123123123123L));
+
+        //Check the MV
+        assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"), row(0, 123123123123L));
+        assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0), row(0, 123123123123L));
+        assertRows(execute("SELECT k from mv1_bigintval WHERE bigintval = ?", 123123123123L), row(0));
+        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 123123123123L, 0), row("ascii text"));
+
+
+        //UPDATE BASE
+        updateView("INSERT INTO %s (k, asciival, bigintval) VALUES (?, ?, fromJson(?))", 0, "ascii text", "1");
+        assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"), row(1L));
+
+        //Check the MV
+        assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"), row(0, 1L));
+        assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0), row(0, 1L));
+        assertRows(execute("SELECT k from mv1_bigintval WHERE bigintval = ?", 123123123123L));
+        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 123123123123L, 0));
+        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 1L, 0), row("ascii text"));
+
+
+        //test truncate also truncates all MV
+        updateView("TRUNCATE %s");
+
+        assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"));
+        assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"));
+        assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0));
+        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 1L, 0));
+    }
+
+    @Test
+    public void testCollections() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "intval int, " +
+                    "listval list<int>, " +
+                    "PRIMARY KEY (k))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
+
+        updateView("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))", 0, 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+        assertRows(execute("SELECT k, listval from mv WHERE intval = ?", 0), row(0, list(1, 2, 3)));
+
+        updateView("INSERT INTO %s (k, intval) VALUES (?, ?)", 1, 1);
+        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 1, "[1, 2, 3]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 1), row(1, list(1, 2, 3)));
+        assertRows(execute("SELECT k, listval from mv WHERE intval = ?", 1), row(1, list(1, 2, 3)));
+    }
+
+    @Test
+    public void testUpdate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "intval int, " +
+                    "PRIMARY KEY (k))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
+
+        updateView("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0);
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 0));
+        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 0), row(0, 0));
+
+        updateView("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 1);
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 1));
+        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1));
+    }
+
+    @Test
+    public void testDecimalUpdate() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "decimalval decimal, " +
+                    "asciival ascii, " +
+                    "PRIMARY KEY (k))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND decimalval IS NOT NULL PRIMARY KEY (decimalval, k)");
+
+        updateView("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "ascii text");
+        updateView("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
+        assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "123123.123123"));
+        assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "123123"), row(0, "ascii text"));
+
+        // accept strings for numbers that cannot be represented as doubles
+        updateView("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
+        assertRows(execute("SELECT k, decimalval, asciival FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123"), "ascii text"));
+        assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "\"123123.123123\""), row(0, "ascii text"));
+    }
+
+    @Test
+    public void testReuseName() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "k int, " +
+                    "intval int, " +
+                    "PRIMARY KEY (k))");
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
+
+        updateView("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0);
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 0));
+        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 0), row(0, 0));
+
+        executeNet(protocolVersion, "DROP MATERIALIZED VIEW mv");
+        views.remove("mv");
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
+
+        updateView("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 1);
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 1));
+        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1));
+    }
+
+    @Test
+    public void testAllTypes() throws Throwable
+    {
+        String myType = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
+
+        createTable("CREATE TABLE %s (" +
+                    "k int PRIMARY KEY, " +
+                    "asciival ascii, " +
+                    "bigintval bigint, " +
+                    "blobval blob, " +
+                    "booleanval boolean, " +
+                    "dateval date, " +
+                    "decimalval decimal, " +
+                    "doubleval double, " +
+                    "floatval float, " +
+                    "inetval inet, " +
+                    "intval int, " +
+                    "textval text, " +
+                    "timeval time, " +
+                    "timestampval timestamp, " +
+                    "timeuuidval timeuuid, " +
+                    "uuidval uuid," +
+                    "varcharval varchar, " +
+                    "varintval varint, " +
+                    "listval list<int>, " +
+                    "frozenlistval frozen<list<int>>, " +
+                    "setval set<uuid>, " +
+                    "frozensetval frozen<set<uuid>>, " +
+                    "mapval map<ascii, int>," +
+                    "frozenmapval frozen<map<ascii, int>>," +
+                    "tupleval frozen<tuple<int, ascii, uuid>>," +
+                    "udtval frozen<" + myType + ">)");
+
+        CFMetaData metadata = currentTableMetadata();
+
+        execute("USE " + keyspace());
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        for (ColumnDefinition def : new HashSet<>(metadata.allColumns()))
+        {
+            try
+            {
+                createView("mv_" + def.name, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL PRIMARY KEY (" + def.name + ",k)");
+
+                if (def.type.isMultiCell())
+                    Assert.fail("MV on a multicell should fail " + def);
+
+                if (def.isPartitionKey())
+                    Assert.fail("MV on partition key should fail " + def);
+            }
+            catch (InvalidQueryException e)
+            {
+                if (!def.type.isMultiCell() && !def.isPartitionKey())
+                    Assert.fail("MV creation failed on " + def);
+            }
+        }
+
+        // fromJson() can only be used when the receiver type is known
+        assertInvalidMessage("fromJson() cannot be used in the selection clause", "SELECT fromJson(asciival) FROM %s", 0, 0);
+
+        String func1 = createFunction(KEYSPACE, "int", "CREATE FUNCTION %s (a int) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return a.toString(); $$");
+        createFunctionOverload(func1, "int", "CREATE FUNCTION %s (a text) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return new String(a); $$");
+
+        // ================ ascii ================
+        updateView("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii \\\" text\"");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii \" text"));
+
+        // test that we can use fromJson() in other valid places in queries
+        assertRows(execute("SELECT asciival FROM %s WHERE k = fromJson(?)", "0"), row("ascii \" text"));
+
+        //Check the MV
+        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii text"));
+        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"), row(0, null));
+
+        updateView("UPDATE %s SET asciival = fromJson(?) WHERE k = fromJson(?)", "\"ascii \\\" text\"", "0");
+        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"), row(0, null));
+
+        updateView("DELETE FROM %s WHERE k = fromJson(?)", "0");
+        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0));
+        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"));
+
+        updateView("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
+        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii text"), row(0, null));
+
+        // ================ bigint ================
+        updateView("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
+        assertRows(execute("SELECT k, asciival from mv_bigintval WHERE bigintval = ?", 123123123123L), row(0, "ascii text"));
+
+        // ================ blob ================
+        updateView("INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x00000001\"");
+        assertRows(execute("SELECT k, blobval FROM %s WHERE k = ?", 0), row(0, ByteBufferUtil.bytes(1)));
+        assertRows(execute("SELECT k, asciival from mv_blobval WHERE blobval = ?", ByteBufferUtil.bytes(1)), row(0, "ascii text"));
+
+        // ================ boolean ================
+        updateView("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "true");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, true));
+        assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", true), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "false");
+        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
+        assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", true));
+        assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", false), row(0, "ascii text"));
+
+        // ================ date ================
+        updateView("INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"1987-03-23\"");
+        assertRows(execute("SELECT k, dateval FROM %s WHERE k = ?", 0), row(0, SimpleDateSerializer.dateStringToDays("1987-03-23")));
+        assertRows(execute("SELECT k, asciival from mv_dateval WHERE dateval = fromJson(?)", "\"1987-03-23\""), row(0, "ascii text"));
+
+        // ================ decimal ================
+        updateView("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
+        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123.123123"), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
+        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123.123123"));
+        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123"), row(0, "ascii text"));
+
+        // accept strings for numbers that cannot be represented as doubles
+        updateView("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
+
+        updateView("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"-1.23E-12\"");
+        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("-1.23E-12")));
+        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "\"-1.23E-12\""), row(0, "ascii text"));
+
+        // ================ double ================
+        updateView("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.123123d));
+        assertRows(execute("SELECT k, asciival from mv_doubleval WHERE doubleval = fromJson(?)", "123123.123123"), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
+        assertRows(execute("SELECT k, asciival from mv_doubleval WHERE doubleval = fromJson(?)", "123123"), row(0, "ascii text"));
+
+        // ================ float ================
+        updateView("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123.123123");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.123123f));
+        assertRows(execute("SELECT k, asciival from mv_floatval WHERE floatval = fromJson(?)", "123123.123123"), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
+        assertRows(execute("SELECT k, asciival from mv_floatval WHERE floatval = fromJson(?)", "123123"), row(0, "ascii text"));
+
+        // ================ inet ================
+        updateView("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"127.0.0.1\"");
+        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("127.0.0.1")));
+        assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"127.0.0.1\""), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"::1\"");
+        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("::1")));
+        assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"127.0.0.1\""));
+        assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"::1\""), row(0, "ascii text"));
+
+        // ================ int ================
+        updateView("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123123");
+        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
+        assertRows(execute("SELECT k, asciival from mv_intval WHERE intval = fromJson(?)", "123123"), row(0, "ascii text"));
+
+        // ================ text (varchar) ================
+        updateView("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"some \\\" text\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "some \" text"));
+
+        updateView("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\\u2013\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "\u2013"));
+        assertRows(execute("SELECT k, asciival from mv_textval WHERE textval = fromJson(?)", "\"\\u2013\""), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
+        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "abcd"));
+        assertRows(execute("SELECT k, asciival from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, "ascii text"));
+
+        // ================ time ================
+        updateView("INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"07:35:07.000111222\"");
+        assertRows(execute("SELECT k, timeval FROM %s WHERE k = ?", 0), row(0, TimeSerializer.timeStringToLong("07:35:07.000111222")));
+        assertRows(execute("SELECT k, asciival from mv_timeval WHERE timeval = fromJson(?)", "\"07:35:07.000111222\""), row(0, "ascii text"));
+
+        // ================ timestamp ================
+        updateView("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new Date(123123123123L)));
+        assertRows(execute("SELECT k, asciival from mv_timestampval WHERE timestampval = fromJson(?)", "123123123123"), row(0, "ascii text"));
+
+        updateView("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"2014-01-01\"");
+        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new SimpleDateFormat("y-M-d").parse("2014-01-01")));
+        assertRows(execute("SELECT k, asciival from mv_timestampval WHERE timestampval = fromJson(?)", "\"2014-01-01\""), row(0, "ascii text"));
+
+        // ================ timeuuid ================
+        updateView("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
+        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        updateView("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
+        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+        assertRows(execute("SELECT k, asciival from mv_timeuuidval WHERE timeuuidval = fromJson(?)", "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""), row(0, "ascii text"));
+
+        // ================ uuidval ================
+        updateView("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
+        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+
+        updateView("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
+        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
+        assertRows(execute("SELECT k, asciival from mv_uuidval WHERE uuidval = fromJson(?)", "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""), row(0, "ascii text"));
+
+        // ================ varint ================
+        updateView("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123123123");
+        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("123123123123")));
+        assertRows(execute("SELECT k, asciival from mv_varintval WHERE varintval = fromJson(?)", "123123123123"), row(0, "ascii text"));
+
+        // accept strings for numbers that cannot be represented as longs
+        updateView("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"1234567890123456789012345678901234567890\"");
+        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("1234567890123456789012345678901234567890")));
+        assertRows(execute("SELECT k, asciival from mv_varintval WHERE varintval = fromJson(?)", "\"1234567890123456789012345678901234567890\""), row(0, "ascii text"));
+
+        // ================ lists ================
+        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2, 3)));
+
+        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1)));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1)));
+
+        updateView("UPDATE %s SET listval = listval + fromJson(?) WHERE k = ?", "[2]", 0);
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2)));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2)));
+
+        updateView("UPDATE %s SET listval = fromJson(?) + listval WHERE k = ?", "[0]", 0);
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 1, 2)));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 1, 2)));
+
+        updateView("UPDATE %s SET listval[1] = fromJson(?) WHERE k = ?", "10", 0);
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 10, 2)));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 10, 2)));
+
+        updateView("DELETE listval[1] FROM %s WHERE k = ?", 0);
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 2)));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 2)));
+
+        updateView("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, null));
+        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, null));
+
+        // frozen
+        updateView("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
+        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
+        assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2, 3)));
+        assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[1, 2, 3]"), row(0, "abcd"));
+
+        updateView("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[3, 2, 1]");
+        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(3, 2, 1)));
+        assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[1, 2, 3]"));
+        assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[3, 2, 1]"), row(0, "abcd"));
+        assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(3, 2, 1)));
+
+        updateView("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list()));
+        assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list()));
+
+        // ================ sets ================
+        updateView("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
+                   0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
+
+        // duplicates are okay, just like in CQL
+        updateView("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
+                   0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
+
+        updateView("UPDATE %s SET setval = setval + fromJson(?) WHERE k = ?", "[\"6bddc89a-5644-0000-97fc-56847afe9799\"]", 0);
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                   row(0, set(UUID.fromString("6bddc89a-5644-0000-97fc-56847afe9799"), UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, set(UUID.fromString("6bddc89a-5644-0000-97fc-56847afe9799"), UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
+
+        updateView("UPDATE %s SET setval = setval - fromJson(?) WHERE k = ?", "[\"6bddc89a-5644-0000-97fc-56847afe9799\"]", 0);
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
+
+        updateView("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[]");
+        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), row(0, null));
+        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, null));
+
+
+        // frozen
+        updateView("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
+                   0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
+        );
+        assertRows(execute("SELECT k, frozensetval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
+
+        updateView("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
+                   0, "[\"6bddc89a-0000-11e4-97fc-56847afe9799\", \"6bddc89a-5644-11e4-97fc-56847afe9798\"]");
+        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
+                   row(0, set(UUID.fromString("6bddc89a-0000-11e4-97fc-56847afe9799"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"))))
+        );
+        assertRows(execute("SELECT k, frozensetval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, set(UUID.fromString("6bddc89a-0000-11e4-97fc-56847afe9799"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798")))));
+
+        // ================ maps ================
+        updateView("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, map("a", 1, "b", 2)));
+
+        updateView("UPDATE %s SET mapval[?] = ?  WHERE k = ?", "c", 3, 0);
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0),
+                   row(0, map("a", 1, "b", 2, "c", 3))
+        );
+        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, map("a", 1, "b", 2, "c", 3)));
+
+        updateView("UPDATE %s SET mapval[?] = ?  WHERE k = ?", "b", 10, 0);
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0),
+                   row(0, map("a", 1, "b", 10, "c", 3))
+        );
+        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, map("a", 1, "b", 10, "c", 3)));
+
+        updateView("DELETE mapval[?] FROM %s WHERE k = ?", "b", 0);
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0),
+                   row(0, map("a", 1, "c", 3))
+        );
+        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, map("a", 1, "c", 3)));
+
+        updateView("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{}");
+        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, null));
+        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
+                   row(0, null));
+
+        // frozen
+        updateView("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
+        assertRows(execute("SELECT k, textval FROM mv_frozenmapval WHERE frozenmapval = fromJson(?)", "{\"a\": 1, \"b\": 2}"), row(0, "abcd"));
+
+        updateView("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"b\": 2, \"a\": 3}");
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 3, "b", 2)));
+        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 3, "b", 2)));
+
+        // ================ tuples ================
+        updateView("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
+                   row(0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
+        );
+        assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))),
+                   row(0, "abcd"));
+
+        updateView("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, null, \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
+        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
+                   row(0, tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
+        );
+        assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
+        assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))),
+                   row(0, "abcd"));
+
+        // ================ UDTs ================
+        updateView("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                   row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"),
+                   row(0, "abcd"));
+
+        // order of fields shouldn't matter
+        updateView("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"a\": 1, \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                   row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"),
+                   row(0, "abcd"));
+
+        // test nulls
+        updateView("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                   row(0, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
+        );
+        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"));
+        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"),
+                   row(0, "abcd"));
+
+        // test missing fields
+        updateView("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}");
+        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
+                   row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), null)
+        );
+        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"));
+        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}"),
+                   row(0, "abcd"));
+    }
+
+
+    @Test
+    public void ttlTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 5", 1, 1, 1, 1);
+
+        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2);
+
+        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
+        List<Row> results = executeNet(protocolVersion, "SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all();
+        Assert.assertEquals(1, results.size());
+        Assert.assertTrue("There should be a null result given back due to ttl expiry", results.get(0).isNull(0));
+    }
+
+    @Test
+    public void conflictingTimestampTest() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        for (int i = 0; i < 50; i++)
+        {
+            updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP 1", 1, 1, i);
+        }
+
+        ResultSet mvRows = executeNet(protocolVersion, "SELECT c FROM mv");
+        List<Row> rows = executeNet(protocolVersion, "SELECT c FROM %s").all();
+        Assert.assertEquals("There should be exactly one row in base", 1, rows.size());
+        int expected = rows.get(0).getInt("c");
+        assertRowsNet(protocolVersion, mvRows, row(expected));
+    }
+
+    @Test
+    public void testClusteringOrder() throws Throwable
+    {
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b, c))" +
+                    "WITH CLUSTERING ORDER BY (b ASC, c DESC)");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c) WITH CLUSTERING ORDER BY (b DESC)");
+        createView("mv2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, c, b) WITH CLUSTERING ORDER BY (c ASC)");
+        createView("mv3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, b, c)");
+        createView("mv4", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (a, c, b) WITH CLUSTERING ORDER BY (c DESC)");
+
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 1, 1, 1);
+        updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 2, 2, 2);
+
+        ResultSet mvRows = executeNet(protocolVersion, "SELECT b FROM mv1");
+        assertRowsNet(protocolVersion, mvRows,
+                      row(2),
+                      row(1));
+
+        mvRows = executeNet(protocolVersion, "SELECT c FROM mv2");
+        assertRowsNet(protocolVersion, mvRows,
+                      row(1),
+                      row(2));
+
+        mvRows = executeNet(protocolVersion, "SELECT b FROM mv3");
+        assertRowsNet(protocolVersion, mvRows,
+                      row(1),
+                      row(2));
+
+        mvRows = executeNet(protocolVersion, "SELECT c FROM mv4");
+        assertRowsNet(protocolVersion, mvRows,
+                      row(2),
+                      row(1));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/db/view/MaterializedViewUtilsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/view/MaterializedViewUtilsTest.java b/test/unit/org/apache/cassandra/db/view/MaterializedViewUtilsTest.java
deleted file mode 100644
index 2544714..0000000
--- a/test/unit/org/apache/cassandra/db/view/MaterializedViewUtilsTest.java
+++ /dev/null
@@ -1,115 +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.view;
-
-import java.net.InetAddress;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import junit.framework.Assert;
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.locator.IEndpointSnitch;
-import org.apache.cassandra.locator.NetworkTopologyStrategy;
-import org.apache.cassandra.locator.PropertyFileSnitch;
-import org.apache.cassandra.locator.TokenMetadata;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.schema.KeyspaceParams;
-import org.apache.cassandra.schema.ReplicationParams;
-import org.apache.cassandra.service.StorageService;
-
-public class MaterializedViewUtilsTest
-{
-    @BeforeClass
-    public static void setUp() throws ConfigurationException
-    {
-        IEndpointSnitch snitch = new PropertyFileSnitch();
-        DatabaseDescriptor.setEndpointSnitch(snitch);
-        Keyspace.setInitialized();
-    }
-
-    @Test
-    public void testGetIndexNaturalEndpoint() throws Exception
-    {
-        TokenMetadata metadata = StorageService.instance.getTokenMetadata();
-        metadata.clearUnsafe();
-
-        // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
-
-        // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
-
-        Map<String, String> replicationMap = new HashMap<>();
-        replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
-
-        replicationMap.put("DC1", "1");
-        replicationMap.put("DC2", "1");
-
-        Keyspace.clear("Keyspace1");
-        KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
-        Schema.instance.setKeyspaceMetadata(meta);
-
-        InetAddress naturalEndpoint = MaterializedViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                                   new StringToken("CA"),
-                                                                                   new StringToken("BB"));
-
-        Assert.assertEquals(InetAddress.getByName("127.0.0.2"), naturalEndpoint);
-    }
-
-
-    @Test
-    public void testLocalHostPreference() throws Exception
-    {
-        TokenMetadata metadata = StorageService.instance.getTokenMetadata();
-        metadata.clearUnsafe();
-
-        // DC1
-        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
-        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
-
-        // DC2
-        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
-        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
-
-        Map<String, String> replicationMap = new HashMap<>();
-        replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
-
-        replicationMap.put("DC1", "2");
-        replicationMap.put("DC2", "2");
-
-        Keyspace.clear("Keyspace1");
-        KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
-        Schema.instance.setKeyspaceMetadata(meta);
-
-        InetAddress naturalEndpoint = MaterializedViewUtils.getViewNaturalEndpoint("Keyspace1",
-                                                                                   new StringToken("CA"),
-                                                                                   new StringToken("BB"));
-
-        Assert.assertEquals(InetAddress.getByName("127.0.0.1"), naturalEndpoint);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
new file mode 100644
index 0000000..8fd0cfb
--- /dev/null
+++ b/test/unit/org/apache/cassandra/db/view/ViewUtilsTest.java
@@ -0,0 +1,115 @@
+/*
+ * 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.view;
+
+import java.net.InetAddress;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import junit.framework.Assert;
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.OrderPreservingPartitioner.StringToken;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.locator.IEndpointSnitch;
+import org.apache.cassandra.locator.NetworkTopologyStrategy;
+import org.apache.cassandra.locator.PropertyFileSnitch;
+import org.apache.cassandra.locator.TokenMetadata;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.schema.KeyspaceParams;
+import org.apache.cassandra.schema.ReplicationParams;
+import org.apache.cassandra.service.StorageService;
+
+public class ViewUtilsTest
+{
+    @BeforeClass
+    public static void setUp() throws ConfigurationException
+    {
+        IEndpointSnitch snitch = new PropertyFileSnitch();
+        DatabaseDescriptor.setEndpointSnitch(snitch);
+        Keyspace.setInitialized();
+    }
+
+    @Test
+    public void testGetIndexNaturalEndpoint() throws Exception
+    {
+        TokenMetadata metadata = StorageService.instance.getTokenMetadata();
+        metadata.clearUnsafe();
+
+        // DC1
+        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+
+        // DC2
+        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+
+        Map<String, String> replicationMap = new HashMap<>();
+        replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
+
+        replicationMap.put("DC1", "1");
+        replicationMap.put("DC2", "1");
+
+        Keyspace.clear("Keyspace1");
+        KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
+        Schema.instance.setKeyspaceMetadata(meta);
+
+        InetAddress naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                       new StringToken("CA"),
+                                                                       new StringToken("BB"));
+
+        Assert.assertEquals(InetAddress.getByName("127.0.0.2"), naturalEndpoint);
+    }
+
+
+    @Test
+    public void testLocalHostPreference() throws Exception
+    {
+        TokenMetadata metadata = StorageService.instance.getTokenMetadata();
+        metadata.clearUnsafe();
+
+        // DC1
+        metadata.updateNormalToken(new StringToken("A"), InetAddress.getByName("127.0.0.1"));
+        metadata.updateNormalToken(new StringToken("C"), InetAddress.getByName("127.0.0.2"));
+
+        // DC2
+        metadata.updateNormalToken(new StringToken("B"), InetAddress.getByName("127.0.0.4"));
+        metadata.updateNormalToken(new StringToken("D"), InetAddress.getByName("127.0.0.5"));
+
+        Map<String, String> replicationMap = new HashMap<>();
+        replicationMap.put(ReplicationParams.CLASS, NetworkTopologyStrategy.class.getName());
+
+        replicationMap.put("DC1", "2");
+        replicationMap.put("DC2", "2");
+
+        Keyspace.clear("Keyspace1");
+        KeyspaceMetadata meta = KeyspaceMetadata.create("Keyspace1", KeyspaceParams.create(false, replicationMap));
+        Schema.instance.setKeyspaceMetadata(meta);
+
+        InetAddress naturalEndpoint = ViewUtils.getViewNaturalEndpoint("Keyspace1",
+                                                                       new StringToken("CA"),
+                                                                       new StringToken("BB"));
+
+        Assert.assertEquals(InetAddress.getByName("127.0.0.1"), naturalEndpoint);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/dht/LengthPartitioner.java b/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
index 40a6774..9cefbf2 100644
--- a/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
+++ b/test/unit/org/apache/cassandra/dht/LengthPartitioner.java
@@ -126,7 +126,7 @@ public class LengthPartitioner implements IPartitioner
 
         for (String ks : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfmd : Schema.instance.getTables(ks))
+            for (CFMetaData cfmd : Schema.instance.getTablesAndViews(ks))
             {
                 for (Range<Token> r : sortedRanges)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/hints/HintTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/hints/HintTest.java b/test/unit/org/apache/cassandra/hints/HintTest.java
index c198149..4c7ec70 100644
--- a/test/unit/org/apache/cassandra/hints/HintTest.java
+++ b/test/unit/org/apache/cassandra/hints/HintTest.java
@@ -67,7 +67,7 @@ public class HintTest
     @Before
     public void resetGcGraceSeconds()
     {
-        for (CFMetaData table : Schema.instance.getTables(KEYSPACE))
+        for (CFMetaData table : Schema.instance.getTablesAndViews(KEYSPACE))
             table.gcGraceSeconds(TableParams.DEFAULT_GC_GRACE_SECONDS);
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
index 73eed06..d841e91 100644
--- a/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
+++ b/test/unit/org/apache/cassandra/schema/LegacySchemaMigratorTest.java
@@ -316,6 +316,7 @@ public class LegacySchemaMigratorTest
         return KeyspaceMetadata.create(keyspace,
                                        KeyspaceParams.simple(1),
                                        Tables.none(),
+                                       Views.none(),
                                        Types.of(udt1, udt2, udt3),
                                        Functions.none());
     }
@@ -353,6 +354,7 @@ public class LegacySchemaMigratorTest
         return KeyspaceMetadata.create(keyspace,
                                        KeyspaceParams.simple(1),
                                        Tables.none(),
+                                       Views.none(),
                                        Types.none(),
                                        Functions.of(udf1, udf2, udf3));
     }
@@ -365,6 +367,7 @@ public class LegacySchemaMigratorTest
         return KeyspaceMetadata.create(keyspace,
                                        KeyspaceParams.simple(1),
                                        Tables.none(),
+                                       Views.none(),
                                        Types.none(),
                                        Functions.of());
     }


[4/7] cassandra git commit: Improve MV schema representation

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/MaterializedViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/MaterializedViewBuilder.java b/src/java/org/apache/cassandra/db/view/MaterializedViewBuilder.java
deleted file mode 100644
index e23fd84..0000000
--- a/src/java/org/apache/cassandra/db/view/MaterializedViewBuilder.java
+++ /dev/null
@@ -1,215 +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.view;
-
-import java.util.Collection;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-
-import com.google.common.base.Function;
-import com.google.common.collect.Iterables;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import org.apache.cassandra.concurrent.ScheduledExecutors;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.ReadOrderGroup;
-import org.apache.cassandra.db.SinglePartitionReadCommand;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.compaction.CompactionInfo;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.compaction.OperationType;
-import org.apache.cassandra.db.lifecycle.SSTableSet;
-import org.apache.cassandra.db.lifecycle.View;
-import org.apache.cassandra.db.partitions.FilteredPartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.rows.RowIterator;
-import org.apache.cassandra.dht.Range;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.WriteTimeoutException;
-import org.apache.cassandra.io.sstable.ReducingKeyIterator;
-import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.service.pager.QueryPager;
-import org.apache.cassandra.utils.FBUtilities;
-import org.apache.cassandra.utils.NoSpamLogger;
-import org.apache.cassandra.utils.Pair;
-import org.apache.cassandra.utils.UUIDGen;
-import org.apache.cassandra.utils.concurrent.Refs;
-
-public class MaterializedViewBuilder extends CompactionInfo.Holder
-{
-    private final ColumnFamilyStore baseCfs;
-    private final MaterializedView view;
-    private final UUID compactionId;
-    private volatile Token prevToken = null;
-
-    private static final Logger logger = LoggerFactory.getLogger(MaterializedViewBuilder.class);
-
-    private volatile boolean isStopped = false;
-
-    public MaterializedViewBuilder(ColumnFamilyStore baseCfs, MaterializedView view)
-    {
-        this.baseCfs = baseCfs;
-        this.view = view;
-        compactionId = UUIDGen.getTimeUUID();
-    }
-
-    private void buildKey(DecoratedKey key)
-    {
-        QueryPager pager = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata, FBUtilities.nowInSeconds(), key).getPager(null);
-
-        while (!pager.isExhausted())
-        {
-           try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                PartitionIterator partitionIterator = pager.fetchPageInternal(128, orderGroup))
-           {
-               if (!partitionIterator.hasNext())
-                   return;
-
-               try (RowIterator rowIterator = partitionIterator.next())
-               {
-                   FilteredPartition partition = FilteredPartition.create(rowIterator);
-                   TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
-
-                   Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
-
-                   if (mutations != null)
-                       StorageProxy.mutateMV(key.getKey(), mutations, true);
-               }
-           }
-        }
-    }
-
-    public void run()
-    {
-        String ksname = baseCfs.metadata.ksName, viewName = view.name;
-
-        if (SystemKeyspace.isViewBuilt(ksname, viewName))
-            return;
-
-        Iterable<Range<Token>> ranges = StorageService.instance.getLocalRanges(baseCfs.metadata.ksName);
-        final Pair<Integer, Token> buildStatus = SystemKeyspace.getMaterializedViewBuildStatus(ksname, viewName);
-        Token lastToken;
-        Function<View, Iterable<SSTableReader>> function;
-        if (buildStatus == null)
-        {
-            baseCfs.forceBlockingFlush();
-            function = View.select(SSTableSet.CANONICAL);
-            int generation = Integer.MIN_VALUE;
-
-            try (Refs<SSTableReader> temp = baseCfs.selectAndReference(function).refs)
-            {
-                for (SSTableReader reader : temp)
-                {
-                    generation = Math.max(reader.descriptor.generation, generation);
-                }
-            }
-
-            SystemKeyspace.beginMaterializedViewBuild(ksname, viewName, generation);
-            lastToken = null;
-        }
-        else
-        {
-            function = new Function<View, Iterable<SSTableReader>>()
-            {
-                @Nullable
-                public Iterable<SSTableReader> apply(View view)
-                {
-                    Iterable<SSTableReader> readers = View.select(SSTableSet.CANONICAL).apply(view);
-                    if (readers != null)
-                        return Iterables.filter(readers, ssTableReader -> ssTableReader.descriptor.generation <= buildStatus.left);
-                    return null;
-                }
-            };
-            lastToken = buildStatus.right;
-        }
-
-        prevToken = lastToken;
-        try (Refs<SSTableReader> sstables = baseCfs.selectAndReference(function).refs;
-             ReducingKeyIterator iter = new ReducingKeyIterator(sstables))
-        {
-            while (!isStopped && iter.hasNext())
-            {
-                DecoratedKey key = iter.next();
-                Token token = key.getToken();
-                if (lastToken == null || lastToken.compareTo(token) < 0)
-                {
-                    for (Range<Token> range : ranges)
-                    {
-                        if (range.contains(token))
-                        {
-                            buildKey(key);
-
-                            if (prevToken == null || prevToken.compareTo(token) != 0)
-                            {
-                                SystemKeyspace.updateMaterializedViewBuildStatus(ksname, viewName, key.getToken());
-                                prevToken = token;
-                            }
-                        }
-                    }
-                    lastToken = null;
-                }
-            }
-
-            if (!isStopped)
-                SystemKeyspace.finishMaterializedViewBuildStatus(ksname, viewName);
-
-        }
-        catch (Exception e)
-        {
-            final MaterializedViewBuilder builder = new MaterializedViewBuilder(baseCfs, view);
-            ScheduledExecutors.nonPeriodicTasks.schedule(() -> CompactionManager.instance.submitMaterializedViewBuilder(builder),
-                                                         5,
-                                                         TimeUnit.MINUTES);
-            logger.warn("Materialized View failed to complete, sleeping 5 minutes before restarting", e);
-        }
-    }
-
-    public CompactionInfo getCompactionInfo()
-    {
-        long rangesLeft = 0, rangesTotal = 0;
-        Token lastToken = prevToken;
-
-        // This approximation is not very accurate, but since we do not have a method which allows us to calculate the
-        // percentage of a range covered by a second range, this is the best approximation that we can calculate.
-        // Instead, we just count the total number of ranges that haven't been seen by the node (we use the order of
-        // the tokens to determine whether they have been seen yet or not), and the total number of ranges that a node
-        // has.
-        for (Range<Token> range : StorageService.instance.getLocalRanges(baseCfs.keyspace.getName()))
-        {
-            rangesLeft++;
-            rangesTotal++;
-            // This will reset rangesLeft, so that the number of ranges left will be less than the total ranges at the
-            // end of the method.
-            if (lastToken == null || range.contains(lastToken))
-                rangesLeft = 0;
-        }
-        return new CompactionInfo(baseCfs.metadata, OperationType.VIEW_BUILD, rangesLeft, rangesTotal, "ranges", compactionId);
-    }
-
-    public void stop()
-    {
-        isStopped = true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/MaterializedViewManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/MaterializedViewManager.java b/src/java/org/apache/cassandra/db/view/MaterializedViewManager.java
deleted file mode 100644
index 41f4ed0..0000000
--- a/src/java/org/apache/cassandra/db/view/MaterializedViewManager.java
+++ /dev/null
@@ -1,241 +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.view;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentNavigableMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.locks.Lock;
-
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.Striped;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.config.MaterializedViewDefinition;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.IMutation;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.db.commitlog.ReplayPosition;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.exceptions.OverloadedException;
-import org.apache.cassandra.exceptions.UnavailableException;
-import org.apache.cassandra.exceptions.WriteTimeoutException;
-import org.apache.cassandra.service.StorageProxy;
-import org.apache.cassandra.service.StorageService;
-
-/**
- * Manages {@link MaterializedView}'s for a single {@link ColumnFamilyStore}. All of the materialized views for that
- * table are created when this manager is initialized.
- *
- * The main purposes of the manager are to provide a single location for updates to be vetted to see whether they update
- * any views {@link MaterializedViewManager#updateAffectsView(PartitionUpdate)}, provide locks to prevent multiple
- * updates from creating incoherent updates in the view {@link MaterializedViewManager#acquireLockFor(ByteBuffer)}, and
- * to affect change on the view.
- */
-public class MaterializedViewManager
-{
-    private static final Striped<Lock> LOCKS = Striped.lazyWeakLock(DatabaseDescriptor.getConcurrentWriters() * 1024);
-    private static final boolean enableCoordinatorBatchlog = Boolean.getBoolean("cassandra.mv_enable_coordinator_batchlog");
-
-    private final ConcurrentNavigableMap<String, MaterializedView> viewsByName;
-
-    private final ColumnFamilyStore baseCfs;
-
-    public MaterializedViewManager(ColumnFamilyStore baseCfs)
-    {
-        this.viewsByName = new ConcurrentSkipListMap<>();
-
-        this.baseCfs = baseCfs;
-    }
-
-    public Iterable<MaterializedView> allViews()
-    {
-        return viewsByName.values();
-    }
-
-    public Iterable<ColumnFamilyStore> allViewsCfs()
-    {
-        List<ColumnFamilyStore> viewColumnFamilies = new ArrayList<>();
-        for (MaterializedView view : allViews())
-            viewColumnFamilies.add(view.getViewCfs());
-        return viewColumnFamilies;
-    }
-
-    public void init()
-    {
-        reload();
-    }
-
-    public void invalidate()
-    {
-        for (MaterializedView view : allViews())
-            removeMaterializedView(view.name);
-    }
-
-    public void reload()
-    {
-        Map<String, MaterializedViewDefinition> newViewsByName = new HashMap<>();
-        for (MaterializedViewDefinition definition : baseCfs.metadata.getMaterializedViews())
-        {
-            newViewsByName.put(definition.viewName, definition);
-        }
-
-        for (String viewName : viewsByName.keySet())
-        {
-            if (!newViewsByName.containsKey(viewName))
-                removeMaterializedView(viewName);
-        }
-
-        for (Map.Entry<String, MaterializedViewDefinition> entry : newViewsByName.entrySet())
-        {
-            if (!viewsByName.containsKey(entry.getKey()))
-                addMaterializedView(entry.getValue());
-        }
-
-        for (MaterializedView view : allViews())
-        {
-            view.build();
-            // We provide the new definition from the base metadata
-            view.updateDefinition(newViewsByName.get(view.name));
-        }
-    }
-
-    public void buildAllViews()
-    {
-        for (MaterializedView view : allViews())
-            view.build();
-    }
-
-    public void removeMaterializedView(String name)
-    {
-        MaterializedView view = viewsByName.remove(name);
-
-        if (view == null)
-            return;
-
-        SystemKeyspace.setMaterializedViewRemoved(baseCfs.metadata.ksName, view.name);
-    }
-
-    public void addMaterializedView(MaterializedViewDefinition definition)
-    {
-        MaterializedView view = new MaterializedView(definition, baseCfs);
-
-        viewsByName.put(definition.viewName, view);
-    }
-
-    /**
-     * Calculates and pushes updates to the views replicas. The replicas are determined by
-     * {@link MaterializedViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
-     */
-    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog)
-    {
-        List<Mutation> mutations = null;
-        TemporalRow.Set temporalRows = null;
-        for (Map.Entry<String, MaterializedView> view : viewsByName.entrySet())
-        {
-
-            temporalRows = view.getValue().getTemporalRowSet(update, temporalRows, false);
-
-            Collection<Mutation> viewMutations = view.getValue().createMutations(update, temporalRows, false);
-            if (viewMutations != null && !viewMutations.isEmpty())
-            {
-                if (mutations == null)
-                    mutations = Lists.newLinkedList();
-                mutations.addAll(viewMutations);
-            }
-        }
-        if (mutations != null)
-        {
-            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog);
-        }
-    }
-
-    public boolean updateAffectsView(PartitionUpdate upd)
-    {
-        for (MaterializedView view : allViews())
-        {
-            if (view.updateAffectsView(upd))
-                return true;
-        }
-        return false;
-    }
-
-    public static Lock acquireLockFor(ByteBuffer key)
-    {
-        Lock lock = LOCKS.get(key);
-
-        if (lock.tryLock())
-            return lock;
-
-        return null;
-    }
-
-    public static boolean updatesAffectView(Collection<? extends IMutation> mutations, boolean coordinatorBatchlog)
-    {
-        if (coordinatorBatchlog && !enableCoordinatorBatchlog)
-            return false;
-
-        for (IMutation mutation : mutations)
-        {
-            for (PartitionUpdate cf : mutation.getPartitionUpdates())
-            {
-                Keyspace keyspace = Keyspace.open(cf.metadata().ksName);
-
-                if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
-                    continue;
-
-                MaterializedViewManager viewManager = keyspace.getColumnFamilyStore(cf.metadata().cfId).materializedViewManager;
-                if (viewManager.updateAffectsView(cf))
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
-
-    public void forceBlockingFlush()
-    {
-        for (ColumnFamilyStore viewCfs : allViewsCfs())
-            viewCfs.forceBlockingFlush();
-    }
-
-    public void dumpMemtables()
-    {
-        for (ColumnFamilyStore viewCfs : allViewsCfs())
-            viewCfs.dumpMemtable();
-    }
-
-    public void truncateBlocking(long truncatedAt)
-    {
-        for (ColumnFamilyStore viewCfs : allViewsCfs())
-        {
-            ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
-            SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/MaterializedViewUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/MaterializedViewUtils.java b/src/java/org/apache/cassandra/db/view/MaterializedViewUtils.java
deleted file mode 100644
index ea81750..0000000
--- a/src/java/org/apache/cassandra/db/view/MaterializedViewUtils.java
+++ /dev/null
@@ -1,109 +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.view;
-
-import java.net.InetAddress;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.cassandra.config.DatabaseDescriptor;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.dht.Token;
-import org.apache.cassandra.locator.AbstractReplicationStrategy;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
-
-public final class MaterializedViewUtils
-{
-    private MaterializedViewUtils()
-    {
-    }
-
-    /**
-     * Calculate the natural endpoint for the view.
-     *
-     * The view natural endpoint is the endpint which has the same cardinality as this node in the replication factor.
-     * The cardinality is the number at which this node would store a piece of data, given the change in replication
-     * factor.
-     *
-     * For example, if we have the following ring:
-     *   A, T1 -> B, T2 -> C, T3 -> A
-     *
-     * For the token T1, at RF=1, A would be included, so A's cardinality for T1 is 1. For the token T1, at RF=2, B would
-     * be included, so B's cardinality for token T1 is 2. For token T3, at RF = 2, A would be included, so A's cardinality
-     * for T3 is 2.
-     *
-     * For a view whose base token is T1 and whose view token is T3, the pairings between the nodes would be:
-     *  A writes to C (A's cardinality is 1 for T1, and C's cardinality is 1 for T3)
-     *  B writes to A (B's cardinality is 2 for T1, and A's cardinality is 2 for T3)
-     *  C writes to B (C's cardinality is 3 for T1, and B's cardinality is 3 for T3)
-     *
-     * @throws RuntimeException if this method is called using a base token which does not belong to this replica
-     */
-    public static InetAddress getViewNaturalEndpoint(String keyspaceName, Token baseToken, Token viewToken)
-    {
-        AbstractReplicationStrategy replicationStrategy = Keyspace.open(keyspaceName).getReplicationStrategy();
-
-        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
-        List<InetAddress> localBaseEndpoints = new ArrayList<>();
-        List<InetAddress> localViewEndpoints = new ArrayList<>();
-        for (InetAddress baseEndpoint : replicationStrategy.getNaturalEndpoints(baseToken))
-        {
-            if (DatabaseDescriptor.getEndpointSnitch().getDatacenter(baseEndpoint).equals(localDataCenter))
-                localBaseEndpoints.add(baseEndpoint);
-        }
-
-        for (InetAddress viewEndpoint : replicationStrategy.getNaturalEndpoints(viewToken))
-        {
-            // If we are a base endpoint which is also a view replica, we use ourselves as our view replica
-            if (viewEndpoint.equals(FBUtilities.getBroadcastAddress()))
-                return viewEndpoint;
-
-            // We have to remove any endpoint which is shared between the base and the view, as it will select itself
-            // and throw off the counts otherwise.
-            if (localBaseEndpoints.contains(viewEndpoint))
-                localBaseEndpoints.remove(viewEndpoint);
-            else if (DatabaseDescriptor.getEndpointSnitch().getDatacenter(viewEndpoint).equals(localDataCenter))
-                localViewEndpoints.add(viewEndpoint);
-        }
-
-        // The replication strategy will be the same for the base and the view, as they must belong to the same keyspace.
-        // Since the same replication strategy is used, the same placement should be used and we should get the same
-        // number of replicas for all of the tokens in the ring.
-        assert localBaseEndpoints.size() == localViewEndpoints.size() : "Replication strategy should have the same number of endpoints for the base and the view";
-        int baseIdx = localBaseEndpoints.indexOf(FBUtilities.getBroadcastAddress());
-
-        if (baseIdx < 0)
-        {
-
-            if (StorageService.instance.getTokenMetadata().pendingEndpointsFor(viewToken, keyspaceName).size() > 0)
-            {
-                //Since there are pending endpoints we are going to store hints this in the batchlog regardless.
-                //So we can pretend we are the views endpoint.
-
-                return FBUtilities.getBroadcastAddress();
-            }
-
-            throw new RuntimeException("Trying to get the view natural endpoint on a non-data replica");
-        }
-
-
-        return localViewEndpoints.get(baseIdx);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/View.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/View.java b/src/java/org/apache/cassandra/db/view/View.java
new file mode 100644
index 0000000..7bcb592
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/View.java
@@ -0,0 +1,682 @@
+/*
+ * 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.view;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Set;
+import java.util.UUID;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.db.AbstractReadCommandBuilder.SinglePartitionSliceBuilder;
+import org.apache.cassandra.db.CBuilder;
+import org.apache.cassandra.db.Clustering;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.DeletionInfo;
+import org.apache.cassandra.db.DeletionTime;
+import org.apache.cassandra.db.LivenessInfo;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.RangeTombstone;
+import org.apache.cassandra.db.ReadCommand;
+import org.apache.cassandra.db.ReadOrderGroup;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.Slice;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.partitions.AbstractBTreePartition;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.BTreeRow;
+import org.apache.cassandra.db.rows.Cell;
+import org.apache.cassandra.db.rows.ColumnData;
+import org.apache.cassandra.db.rows.ComplexColumnData;
+import org.apache.cassandra.db.rows.Row;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.schema.KeyspaceMetadata;
+import org.apache.cassandra.service.pager.QueryPager;
+
+/**
+ * A View copies data from a base table into a view table which can be queried independently from the
+ * base. Every update which targets the base table must be fed through the {@link ViewManager} to ensure
+ * that if a view needs to be updated, the updates are properly created and fed into the view.
+ *
+ * This class does the job of translating the base row to the view row.
+ *
+ * It handles reading existing state and figuring out what tombstones need to be generated.
+ *
+ * {@link View#createMutations(AbstractBTreePartition, TemporalRow.Set, boolean)} is the "main method"
+ *
+ */
+public class View
+{
+    /**
+     * The columns should all be updated together, so we use this object as group.
+     */
+    private static class Columns
+    {
+        //These are the base column definitions in terms of the *views* partitioning.
+        //Meaning we can see (for example) the partition key of the view contains a clustering key
+        //from the base table.
+        public final List<ColumnDefinition> partitionDefs;
+        public final List<ColumnDefinition> primaryKeyDefs;
+        public final List<ColumnDefinition> baseComplexColumns;
+
+        private Columns(List<ColumnDefinition> partitionDefs, List<ColumnDefinition> primaryKeyDefs, List<ColumnDefinition> baseComplexColumns)
+        {
+            this.partitionDefs = partitionDefs;
+            this.primaryKeyDefs = primaryKeyDefs;
+            this.baseComplexColumns = baseComplexColumns;
+        }
+    }
+
+    public final String name;
+    private volatile ViewDefinition definition;
+
+    private final ColumnFamilyStore baseCfs;
+
+    private Columns columns;
+
+    private final boolean viewHasAllPrimaryKeys;
+    private final boolean includeAllColumns;
+    private ViewBuilder builder;
+
+    public View(ViewDefinition definition,
+                ColumnFamilyStore baseCfs)
+    {
+        this.baseCfs = baseCfs;
+
+        name = definition.viewName;
+        includeAllColumns = definition.includeAllColumns;
+
+        viewHasAllPrimaryKeys = updateDefinition(definition);
+    }
+
+    public ViewDefinition getDefinition()
+    {
+        return definition;
+    }
+
+    /**
+     * Lookup column definitions in the base table that correspond to the view columns (should be 1:1)
+     *
+     * Notify caller if all primary keys in the view are ALL primary keys in the base. We do this to simplify
+     * tombstone checks.
+     *
+     * @param columns a list of columns to lookup in the base table
+     * @param definitions lists to populate for the base table definitions
+     * @return true if all view PKs are also Base PKs
+     */
+    private boolean resolveAndAddColumns(Iterable<ColumnIdentifier> columns, List<ColumnDefinition>... definitions)
+    {
+        boolean allArePrimaryKeys = true;
+        for (ColumnIdentifier identifier : columns)
+        {
+            ColumnDefinition cdef = baseCfs.metadata.getColumnDefinition(identifier);
+            assert cdef != null : "Could not resolve column " + identifier.toString();
+
+            for (List<ColumnDefinition> list : definitions)
+            {
+                list.add(cdef);
+            }
+
+            allArePrimaryKeys = allArePrimaryKeys && cdef.isPrimaryKeyColumn();
+        }
+
+        return allArePrimaryKeys;
+    }
+
+    /**
+     * This updates the columns stored which are dependent on the base CFMetaData.
+     *
+     * @return true if the view contains only columns which are part of the base's primary key; false if there is at
+     *         least one column which is not.
+     */
+    public boolean updateDefinition(ViewDefinition definition)
+    {
+        this.definition = definition;
+
+        CFMetaData viewCfm = definition.metadata;
+        List<ColumnDefinition> partitionDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size());
+        List<ColumnDefinition> primaryKeyDefs = new ArrayList<>(viewCfm.partitionKeyColumns().size()
+                                                                + viewCfm.clusteringColumns().size());
+        List<ColumnDefinition> baseComplexColumns = new ArrayList<>();
+
+        // We only add the partition columns to the partitions list, but both partition columns and clustering
+        // columns are added to the primary keys list
+        boolean partitionAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.partitionKeyColumns(), cd -> cd.name), primaryKeyDefs, partitionDefs);
+        boolean clusteringAllPrimaryKeyColumns = resolveAndAddColumns(Iterables.transform(viewCfm.clusteringColumns(), cd -> cd.name), primaryKeyDefs);
+
+        for (ColumnDefinition cdef : baseCfs.metadata.allColumns())
+        {
+            if (cdef.isComplex())
+            {
+                baseComplexColumns.add(cdef);
+            }
+        }
+
+        this.columns = new Columns(partitionDefs, primaryKeyDefs, baseComplexColumns);
+
+        return partitionAllPrimaryKeyColumns && clusteringAllPrimaryKeyColumns;
+    }
+
+    /**
+     * Check to see if the update could possibly modify a view. Cases where the view may be updated are:
+     * <ul>
+     *     <li>View selects all columns</li>
+     *     <li>Update contains any range tombstones</li>
+     *     <li>Update touches one of the columns included in the view</li>
+     * </ul>
+     *
+     * If the update contains any range tombstones, there is a possibility that it will not touch a range that is
+     * currently included in the view.
+     *
+     * @return true if {@param partition} modifies a column included in the view
+     */
+    public boolean updateAffectsView(AbstractBTreePartition partition)
+    {
+        // If we are including all of the columns, then any update will be included
+        if (includeAllColumns)
+            return true;
+
+        // If there are range tombstones, tombstones will also need to be generated for the view
+        // This requires a query of the base rows and generating tombstones for all of those values
+        if (!partition.deletionInfo().isLive())
+            return true;
+
+        // Check each row for deletion or update
+        for (Row row : partition)
+        {
+            if (row.hasComplexDeletion())
+                return true;
+            if (!row.deletion().isLive())
+                return true;
+
+            for (ColumnData data : row)
+            {
+                if (definition.metadata.getColumnDefinition(data.column().name) != null)
+                    return true;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Creates the clustering columns for the view based on the specified row and resolver policy
+     *
+     * @param temporalRow The current row
+     * @param resolver The policy to use when selecting versions of cells use
+     * @return The clustering object to use for the view
+     */
+    private Clustering viewClustering(TemporalRow temporalRow, TemporalRow.Resolver resolver)
+    {
+        CFMetaData viewCfm = definition.metadata;
+        int numViewClustering = viewCfm.clusteringColumns().size();
+        CBuilder clustering = CBuilder.create(viewCfm.comparator);
+        for (int i = 0; i < numViewClustering; i++)
+        {
+            ColumnDefinition definition = viewCfm.clusteringColumns().get(i);
+            clustering.add(temporalRow.clusteringValue(definition, resolver));
+        }
+
+        return clustering.build();
+    }
+
+    /**
+     * @return Mutation containing a range tombstone for a base partition key and TemporalRow.
+     */
+    private PartitionUpdate createTombstone(TemporalRow temporalRow,
+                                            DecoratedKey partitionKey,
+                                            Row.Deletion deletion,
+                                            TemporalRow.Resolver resolver,
+                                            int nowInSec)
+    {
+        CFMetaData viewCfm = definition.metadata;
+        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
+        builder.newRow(viewClustering(temporalRow, resolver));
+        builder.addRowDeletion(deletion);
+        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
+    }
+
+    /**
+     * @return PartitionUpdate containing a complex tombstone for a TemporalRow, and the collection's column identifier.
+     */
+    private PartitionUpdate createComplexTombstone(TemporalRow temporalRow,
+                                                   DecoratedKey partitionKey,
+                                                   ColumnDefinition deletedColumn,
+                                                   DeletionTime deletionTime,
+                                                   TemporalRow.Resolver resolver,
+                                                   int nowInSec)
+    {
+        CFMetaData viewCfm = definition.metadata;
+        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
+        builder.newRow(viewClustering(temporalRow, resolver));
+        builder.addComplexDeletion(deletedColumn, deletionTime);
+        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
+    }
+
+    /**
+     * @return View's DecoratedKey or null, if one of the view's primary key components has an invalid resolution from
+     *         the TemporalRow and its Resolver
+     */
+    private DecoratedKey viewPartitionKey(TemporalRow temporalRow, TemporalRow.Resolver resolver)
+    {
+        List<ColumnDefinition> partitionDefs = this.columns.partitionDefs;
+        Object[] partitionKey = new Object[partitionDefs.size()];
+
+        for (int i = 0; i < partitionKey.length; i++)
+        {
+            ByteBuffer value = temporalRow.clusteringValue(partitionDefs.get(i), resolver);
+
+            if (value == null)
+                return null;
+
+            partitionKey[i] = value;
+        }
+
+        CFMetaData metadata = definition.metadata;
+        return metadata.decorateKey(CFMetaData.serializePartitionKey(metadata
+                                                                     .getKeyValidatorAsClusteringComparator()
+                                                                     .make(partitionKey)));
+    }
+
+    /**
+     * @return mutation which contains the tombstone for the referenced TemporalRow, or null if not necessary.
+     * TemporalRow's can reference at most one view row; there will be at most one row to be tombstoned, so only one
+     * mutation is necessary
+     */
+    private PartitionUpdate createRangeTombstoneForRow(TemporalRow temporalRow)
+    {
+        // Primary Key and Clustering columns do not generate tombstones
+        if (viewHasAllPrimaryKeys)
+            return null;
+
+        boolean hasUpdate = false;
+        List<ColumnDefinition> primaryKeyDefs = this.columns.primaryKeyDefs;
+        for (ColumnDefinition viewPartitionKeys : primaryKeyDefs)
+        {
+            if (!viewPartitionKeys.isPrimaryKeyColumn() && temporalRow.clusteringValue(viewPartitionKeys, TemporalRow.oldValueIfUpdated) != null)
+                hasUpdate = true;
+        }
+
+        if (!hasUpdate)
+            return null;
+
+        TemporalRow.Resolver resolver = TemporalRow.earliest;
+        return createTombstone(temporalRow,
+                               viewPartitionKey(temporalRow, resolver),
+                               Row.Deletion.shadowable(new DeletionTime(temporalRow.viewClusteringTimestamp(), temporalRow.nowInSec)),
+                               resolver,
+                               temporalRow.nowInSec);
+    }
+
+    /**
+     * @return Mutation which is the transformed base table mutation for the view.
+     */
+    private PartitionUpdate createUpdatesForInserts(TemporalRow temporalRow)
+    {
+        TemporalRow.Resolver resolver = TemporalRow.latest;
+
+        DecoratedKey partitionKey = viewPartitionKey(temporalRow, resolver);
+        CFMetaData viewCfm = definition.metadata;
+
+        if (partitionKey == null)
+        {
+            // Not having a partition key means we aren't updating anything
+            return null;
+        }
+
+        Row.Builder regularBuilder = BTreeRow.unsortedBuilder(temporalRow.nowInSec);
+
+        CBuilder clustering = CBuilder.create(viewCfm.comparator);
+        for (int i = 0; i < viewCfm.clusteringColumns().size(); i++)
+        {
+            clustering.add(temporalRow.clusteringValue(viewCfm.clusteringColumns().get(i), resolver));
+        }
+        regularBuilder.newRow(clustering.build());
+        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(viewCfm,
+                                                                     temporalRow.viewClusteringTimestamp(),
+                                                                     temporalRow.viewClusteringTtl(),
+                                                                     temporalRow.viewClusteringLocalDeletionTime()));
+
+        for (ColumnDefinition columnDefinition : viewCfm.allColumns())
+        {
+            if (columnDefinition.isPrimaryKeyColumn())
+                continue;
+
+            for (Cell cell : temporalRow.values(columnDefinition, resolver))
+            {
+                regularBuilder.addCell(cell);
+            }
+        }
+
+        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, regularBuilder.build());
+    }
+
+    /**
+     * @param partition Update which possibly contains deletion info for which to generate view tombstones.
+     * @return    View Tombstones which delete all of the rows which have been removed from the base table with
+     *            {@param partition}
+     */
+    private Collection<Mutation> createForDeletionInfo(TemporalRow.Set rowSet, AbstractBTreePartition partition)
+    {
+        final TemporalRow.Resolver resolver = TemporalRow.earliest;
+
+        DeletionInfo deletionInfo = partition.deletionInfo();
+
+        List<Mutation> mutations = new ArrayList<>();
+
+        // Check the complex columns to see if there are any which may have tombstones we need to create for the view
+        if (!columns.baseComplexColumns.isEmpty())
+        {
+            for (Row row : partition)
+            {
+                if (!row.hasComplexDeletion())
+                    continue;
+
+                TemporalRow temporalRow = rowSet.getClustering(row.clustering());
+
+                assert temporalRow != null;
+
+                for (ColumnDefinition definition : columns.baseComplexColumns)
+                {
+                    ComplexColumnData columnData = row.getComplexColumnData(definition);
+
+                    if (columnData != null)
+                    {
+                        DeletionTime time = columnData.complexDeletion();
+                        if (!time.isLive())
+                        {
+                            DecoratedKey targetKey = viewPartitionKey(temporalRow, resolver);
+                            if (targetKey != null)
+                                mutations.add(new Mutation(createComplexTombstone(temporalRow, targetKey, definition, time, resolver, temporalRow.nowInSec)));
+                        }
+                    }
+                }
+            }
+        }
+
+        ReadCommand command = null;
+
+        if (!deletionInfo.isLive())
+        {
+            // We have to generate tombstones for all of the affected rows, but we don't have the information in order
+            // to create them. This requires that we perform a read for the entire range that is being tombstoned, and
+            // generate a tombstone for each. This may be slow, because a single range tombstone can cover up to an
+            // entire partition of data which is not distributed on a single partition node.
+            DecoratedKey dk = rowSet.dk;
+
+            if (deletionInfo.hasRanges())
+            {
+                SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, dk);
+                Iterator<RangeTombstone> tombstones = deletionInfo.rangeIterator(false);
+                while (tombstones.hasNext())
+                {
+                    RangeTombstone tombstone = tombstones.next();
+
+                    builder.addSlice(tombstone.deletedSlice());
+                }
+
+                command = builder.build();
+            }
+            else
+            {
+                command = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata, rowSet.nowInSec, dk);
+            }
+        }
+
+        if (command == null)
+        {
+            SinglePartitionSliceBuilder builder = null;
+            for (Row row : partition)
+            {
+                if (!row.deletion().isLive())
+                {
+                    if (builder == null)
+                        builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
+                    builder.addSlice(Slice.make(row.clustering()));
+                }
+            }
+
+            if (builder != null)
+                command = builder.build();
+        }
+
+        if (command != null)
+        {
+
+            //We may have already done this work for
+            //another MV update so check
+
+            if (!rowSet.hasTombstonedExisting())
+            {
+                QueryPager pager = command.getPager(null);
+
+                // Add all of the rows which were recovered from the query to the row set
+                while (!pager.isExhausted())
+                {
+                    try (ReadOrderGroup orderGroup = pager.startOrderGroup();
+                         PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
+                    {
+                        if (!iter.hasNext())
+                            break;
+
+                        try (RowIterator rowIterator = iter.next())
+                        {
+                            while (rowIterator.hasNext())
+                            {
+                                Row row = rowIterator.next();
+                                rowSet.addRow(row, false);
+                            }
+                        }
+                    }
+                }
+
+                //Incase we fetched nothing, avoid re checking on another MV update
+                rowSet.setTombstonedExisting();
+            }
+
+            // If the temporal row has been deleted by the deletion info, we generate the corresponding range tombstone
+            // for the view.
+            for (TemporalRow temporalRow : rowSet)
+            {
+                DeletionTime deletionTime = temporalRow.deletionTime(partition);
+                if (!deletionTime.isLive())
+                {
+                    DecoratedKey value = viewPartitionKey(temporalRow, resolver);
+                    if (value != null)
+                    {
+                        PartitionUpdate update = createTombstone(temporalRow, value, Row.Deletion.regular(deletionTime), resolver, temporalRow.nowInSec);
+                        if (update != null)
+                            mutations.add(new Mutation(update));
+                    }
+                }
+            }
+        }
+
+        return !mutations.isEmpty() ? mutations : null;
+    }
+
+    /**
+     * Read and update temporal rows in the set which have corresponding values stored on the local node
+     */
+    private void readLocalRows(TemporalRow.Set rowSet)
+    {
+        SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
+
+        for (TemporalRow temporalRow : rowSet)
+            builder.addSlice(temporalRow.baseSlice());
+
+        QueryPager pager = builder.build().getPager(null);
+
+        while (!pager.isExhausted())
+        {
+            try (ReadOrderGroup orderGroup = pager.startOrderGroup();
+                 PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
+            {
+                while (iter.hasNext())
+                {
+                    try (RowIterator rows = iter.next())
+                    {
+                        while (rows.hasNext())
+                        {
+                            rowSet.addRow(rows.next(), false);
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     * @return Set of rows which are contained in the partition update {@param partition}
+     */
+    private TemporalRow.Set separateRows(AbstractBTreePartition partition, Set<ColumnIdentifier> viewPrimaryKeyCols)
+    {
+
+        TemporalRow.Set rowSet = new TemporalRow.Set(baseCfs, viewPrimaryKeyCols, partition.partitionKey().getKey());
+
+        for (Row row : partition)
+            rowSet.addRow(row, true);
+
+        return rowSet;
+    }
+
+    /**
+     * Splits the partition update up and adds the existing state to each row.
+     * This data can be reused for multiple MV updates on the same base table
+     *
+     * @param partition the mutation
+     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
+     *                   since all of the update will already be present in the base table.
+     * @return The set of temoral rows contained in this update
+     */
+    public TemporalRow.Set getTemporalRowSet(AbstractBTreePartition partition, TemporalRow.Set existing, boolean isBuilding)
+    {
+        if (!updateAffectsView(partition))
+            return null;
+
+        Set<ColumnIdentifier> columns = new HashSet<>(this.columns.primaryKeyDefs.size());
+        for (ColumnDefinition def : this.columns.primaryKeyDefs)
+            columns.add(def.name);
+
+        TemporalRow.Set rowSet;
+        if (existing == null)
+        {
+            rowSet = separateRows(partition, columns);
+
+            // If we are building the view, we do not want to add old values; they will always be the same
+            if (!isBuilding)
+                readLocalRows(rowSet);
+        }
+        else
+        {
+            rowSet = existing.withNewViewPrimaryKey(columns);
+        }
+
+        return rowSet;
+    }
+
+
+    /**
+     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
+     *                   since all of the update will already be present in the base table.
+     * @return View mutations which represent the changes necessary as long as previously created mutations for the view
+     *         have been applied successfully. This is based solely on the changes that are necessary given the current
+     *         state of the base table and the newly applying partition data.
+     */
+    public Collection<Mutation> createMutations(AbstractBTreePartition partition, TemporalRow.Set rowSet, boolean isBuilding)
+    {
+        if (!updateAffectsView(partition))
+            return null;
+
+        Collection<Mutation> mutations = null;
+        for (TemporalRow temporalRow : rowSet)
+        {
+            // If we are building, there is no need to check for partition tombstones; those values will not be present
+            // in the partition data
+            if (!isBuilding)
+            {
+                PartitionUpdate partitionTombstone = createRangeTombstoneForRow(temporalRow);
+                if (partitionTombstone != null)
+                {
+                    if (mutations == null) mutations = new LinkedList<>();
+                    mutations.add(new Mutation(partitionTombstone));
+                }
+            }
+
+            PartitionUpdate insert = createUpdatesForInserts(temporalRow);
+            if (insert != null)
+            {
+                if (mutations == null) mutations = new LinkedList<>();
+                mutations.add(new Mutation(insert));
+            }
+        }
+
+        if (!isBuilding)
+        {
+            Collection<Mutation> deletion = createForDeletionInfo(rowSet, partition);
+            if (deletion != null && !deletion.isEmpty())
+            {
+                if (mutations == null) mutations = new LinkedList<>();
+                mutations.addAll(deletion);
+            }
+        }
+
+        return mutations;
+    }
+
+    public synchronized void build()
+    {
+        if (this.builder != null)
+        {
+            this.builder.stop();
+            this.builder = null;
+        }
+
+        this.builder = new ViewBuilder(baseCfs, this);
+        CompactionManager.instance.submitViewBuilder(builder);
+    }
+
+    @Nullable
+    public static CFMetaData findBaseTable(String keyspace, String viewName)
+    {
+        ViewDefinition view = Schema.instance.getView(keyspace, viewName);
+        return (view == null) ? null : Schema.instance.getCFMetaData(view.baseTableId);
+    }
+
+    public static Iterable<ViewDefinition> findAll(String keyspace, String baseTable)
+    {
+        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
+        final UUID baseId = Schema.instance.getId(keyspace, baseTable);
+        return Iterables.filter(ksm.views, view -> view.baseTableId.equals(baseId));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/ViewBuilder.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewBuilder.java b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
new file mode 100644
index 0000000..62aa332
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/ViewBuilder.java
@@ -0,0 +1,214 @@
+/*
+ * 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.view;
+
+import java.util.Collection;
+import java.util.UUID;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+
+import com.google.common.base.Function;
+import com.google.common.collect.Iterables;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.DecoratedKey;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.ReadOrderGroup;
+import org.apache.cassandra.db.SinglePartitionReadCommand;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.compaction.CompactionInfo;
+import org.apache.cassandra.db.compaction.CompactionManager;
+import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.lifecycle.SSTableSet;
+import org.apache.cassandra.db.partitions.FilteredPartition;
+import org.apache.cassandra.db.partitions.PartitionIterator;
+import org.apache.cassandra.db.rows.RowIterator;
+import org.apache.cassandra.dht.Range;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.exceptions.WriteTimeoutException;
+import org.apache.cassandra.io.sstable.ReducingKeyIterator;
+import org.apache.cassandra.io.sstable.format.SSTableReader;
+import org.apache.cassandra.service.StorageProxy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.service.pager.QueryPager;
+import org.apache.cassandra.utils.FBUtilities;
+import org.apache.cassandra.utils.NoSpamLogger;
+import org.apache.cassandra.utils.Pair;
+import org.apache.cassandra.utils.UUIDGen;
+import org.apache.cassandra.utils.concurrent.Refs;
+
+public class ViewBuilder extends CompactionInfo.Holder
+{
+    private final ColumnFamilyStore baseCfs;
+    private final View view;
+    private final UUID compactionId;
+    private volatile Token prevToken = null;
+
+    private static final Logger logger = LoggerFactory.getLogger(ViewBuilder.class);
+
+    private volatile boolean isStopped = false;
+
+    public ViewBuilder(ColumnFamilyStore baseCfs, View view)
+    {
+        this.baseCfs = baseCfs;
+        this.view = view;
+        compactionId = UUIDGen.getTimeUUID();
+    }
+
+    private void buildKey(DecoratedKey key)
+    {
+        QueryPager pager = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata, FBUtilities.nowInSeconds(), key).getPager(null);
+
+        while (!pager.isExhausted())
+        {
+           try (ReadOrderGroup orderGroup = pager.startOrderGroup();
+                PartitionIterator partitionIterator = pager.fetchPageInternal(128, orderGroup))
+           {
+               if (!partitionIterator.hasNext())
+                   return;
+
+               try (RowIterator rowIterator = partitionIterator.next())
+               {
+                   FilteredPartition partition = FilteredPartition.create(rowIterator);
+                   TemporalRow.Set temporalRows = view.getTemporalRowSet(partition, null, true);
+
+                   Collection<Mutation> mutations = view.createMutations(partition, temporalRows, true);
+
+                   if (mutations != null)
+                       StorageProxy.mutateMV(key.getKey(), mutations, true);
+               }
+           }
+        }
+    }
+
+    public void run()
+    {
+        String ksname = baseCfs.metadata.ksName, viewName = view.name;
+
+        if (SystemKeyspace.isViewBuilt(ksname, viewName))
+            return;
+
+        Iterable<Range<Token>> ranges = StorageService.instance.getLocalRanges(baseCfs.metadata.ksName);
+        final Pair<Integer, Token> buildStatus = SystemKeyspace.getViewBuildStatus(ksname, viewName);
+        Token lastToken;
+        Function<org.apache.cassandra.db.lifecycle.View, Iterable<SSTableReader>> function;
+        if (buildStatus == null)
+        {
+            baseCfs.forceBlockingFlush();
+            function = org.apache.cassandra.db.lifecycle.View.select(SSTableSet.CANONICAL);
+            int generation = Integer.MIN_VALUE;
+
+            try (Refs<SSTableReader> temp = baseCfs.selectAndReference(function).refs)
+            {
+                for (SSTableReader reader : temp)
+                {
+                    generation = Math.max(reader.descriptor.generation, generation);
+                }
+            }
+
+            SystemKeyspace.beginViewBuild(ksname, viewName, generation);
+            lastToken = null;
+        }
+        else
+        {
+            function = new Function<org.apache.cassandra.db.lifecycle.View, Iterable<SSTableReader>>()
+            {
+                @Nullable
+                public Iterable<SSTableReader> apply(org.apache.cassandra.db.lifecycle.View view)
+                {
+                    Iterable<SSTableReader> readers = org.apache.cassandra.db.lifecycle.View.select(SSTableSet.CANONICAL).apply(view);
+                    if (readers != null)
+                        return Iterables.filter(readers, ssTableReader -> ssTableReader.descriptor.generation <= buildStatus.left);
+                    return null;
+                }
+            };
+            lastToken = buildStatus.right;
+        }
+
+        prevToken = lastToken;
+        try (Refs<SSTableReader> sstables = baseCfs.selectAndReference(function).refs;
+             ReducingKeyIterator iter = new ReducingKeyIterator(sstables))
+        {
+            while (!isStopped && iter.hasNext())
+            {
+                DecoratedKey key = iter.next();
+                Token token = key.getToken();
+                if (lastToken == null || lastToken.compareTo(token) < 0)
+                {
+                    for (Range<Token> range : ranges)
+                    {
+                        if (range.contains(token))
+                        {
+                            buildKey(key);
+
+                            if (prevToken == null || prevToken.compareTo(token) != 0)
+                            {
+                                SystemKeyspace.updateViewBuildStatus(ksname, viewName, key.getToken());
+                                prevToken = token;
+                            }
+                        }
+                    }
+                    lastToken = null;
+                }
+            }
+
+            if (!isStopped)
+            SystemKeyspace.finishViewBuildStatus(ksname, viewName);
+
+        }
+        catch (Exception e)
+        {
+            final ViewBuilder builder = new ViewBuilder(baseCfs, view);
+            ScheduledExecutors.nonPeriodicTasks.schedule(() -> CompactionManager.instance.submitViewBuilder(builder),
+                                                         5,
+                                                         TimeUnit.MINUTES);
+            logger.warn("Materialized View failed to complete, sleeping 5 minutes before restarting", e);
+        }
+    }
+
+    public CompactionInfo getCompactionInfo()
+    {
+        long rangesLeft = 0, rangesTotal = 0;
+        Token lastToken = prevToken;
+
+        // This approximation is not very accurate, but since we do not have a method which allows us to calculate the
+        // percentage of a range covered by a second range, this is the best approximation that we can calculate.
+        // Instead, we just count the total number of ranges that haven't been seen by the node (we use the order of
+        // the tokens to determine whether they have been seen yet or not), and the total number of ranges that a node
+        // has.
+        for (Range<Token> range : StorageService.instance.getLocalRanges(baseCfs.keyspace.getName()))
+        {
+            rangesLeft++;
+            rangesTotal++;
+            // This will reset rangesLeft, so that the number of ranges left will be less than the total ranges at the
+            // end of the method.
+            if (lastToken == null || range.contains(lastToken))
+                rangesLeft = 0;
+        }
+        return new CompactionInfo(baseCfs.metadata, OperationType.VIEW_BUILD, rangesLeft, rangesTotal, "ranges", compactionId);
+    }
+
+    public void stop()
+    {
+        isStopped = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/ViewManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewManager.java b/src/java/org/apache/cassandra/db/view/ViewManager.java
new file mode 100644
index 0000000..2364ed1
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/ViewManager.java
@@ -0,0 +1,271 @@
+/*
+ * 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.view;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.UUID;
+import java.util.concurrent.ConcurrentNavigableMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.locks.Lock;
+
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.Striped;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.db.ColumnFamilyStore;
+import org.apache.cassandra.db.IMutation;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.db.commitlog.ReplayPosition;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.service.StorageProxy;
+
+/**
+ * Manages {@link View}'s for a single {@link ColumnFamilyStore}. All of the views for that table are created when this
+ * manager is initialized.
+ *
+ * The main purposes of the manager are to provide a single location for updates to be vetted to see whether they update
+ * any views {@link ViewManager#updatesAffectView(Collection, boolean)}, provide locks to prevent multiple
+ * updates from creating incoherent updates in the view {@link ViewManager#acquireLockFor(ByteBuffer)}, and
+ * to affect change on the view.
+ */
+public class ViewManager
+{
+    public class ForStore
+    {
+        private final ConcurrentNavigableMap<String, View> viewsByName;
+
+        public ForStore()
+        {
+            this.viewsByName = new ConcurrentSkipListMap<>();
+        }
+
+        public Iterable<View> allViews()
+        {
+            return viewsByName.values();
+        }
+
+        public Iterable<ColumnFamilyStore> allViewsCfs()
+        {
+            List<ColumnFamilyStore> viewColumnFamilies = new ArrayList<>();
+            for (View view : allViews())
+                viewColumnFamilies.add(keyspace.getColumnFamilyStore(view.getDefinition().viewName));
+            return viewColumnFamilies;
+        }
+
+        public void forceBlockingFlush()
+        {
+            for (ColumnFamilyStore viewCfs : allViewsCfs())
+                viewCfs.forceBlockingFlush();
+        }
+
+        public void dumpMemtables()
+        {
+            for (ColumnFamilyStore viewCfs : allViewsCfs())
+                viewCfs.dumpMemtable();
+        }
+
+        public void truncateBlocking(long truncatedAt)
+        {
+            for (ColumnFamilyStore viewCfs : allViewsCfs())
+            {
+                ReplayPosition replayAfter = viewCfs.discardSSTables(truncatedAt);
+                SystemKeyspace.saveTruncationRecord(viewCfs, truncatedAt, replayAfter);
+            }
+        }
+
+        public void addView(View view)
+        {
+            viewsByName.put(view.name, view);
+        }
+
+        public void removeView(String name)
+        {
+            viewsByName.remove(name);
+        }
+    }
+
+    private static final Striped<Lock> LOCKS = Striped.lazyWeakLock(DatabaseDescriptor.getConcurrentViewWriters() * 1024);
+
+    private static final boolean enableCoordinatorBatchlog = Boolean.getBoolean("cassandra.mv_enable_coordinator_batchlog");
+
+    private final ConcurrentNavigableMap<UUID, ForStore> viewManagersByStore;
+    private final ConcurrentNavigableMap<String, View> viewsByName;
+    private final Keyspace keyspace;
+
+    public ViewManager(Keyspace keyspace)
+    {
+        this.viewManagersByStore = new ConcurrentSkipListMap<>();
+        this.viewsByName = new ConcurrentSkipListMap<>();
+        this.keyspace = keyspace;
+    }
+
+    /**
+     * Calculates and pushes updates to the views replicas. The replicas are determined by
+     * {@link ViewUtils#getViewNaturalEndpoint(String, Token, Token)}.
+     */
+    public void pushViewReplicaUpdates(PartitionUpdate update, boolean writeCommitLog)
+    {
+        List<Mutation> mutations = null;
+        TemporalRow.Set temporalRows = null;
+        for (Map.Entry<String, View> view : viewsByName.entrySet())
+        {
+            temporalRows = view.getValue().getTemporalRowSet(update, temporalRows, false);
+
+            Collection<Mutation> viewMutations = view.getValue().createMutations(update, temporalRows, false);
+            if (viewMutations != null && !viewMutations.isEmpty())
+            {
+                if (mutations == null)
+                    mutations = Lists.newLinkedList();
+                mutations.addAll(viewMutations);
+            }
+        }
+
+        if (mutations != null)
+            StorageProxy.mutateMV(update.partitionKey().getKey(), mutations, writeCommitLog);
+    }
+
+    public boolean updatesAffectView(Collection<? extends IMutation> mutations, boolean coordinatorBatchlog)
+    {
+        if (coordinatorBatchlog && !enableCoordinatorBatchlog)
+            return false;
+
+        for (IMutation mutation : mutations)
+        {
+            for (PartitionUpdate cf : mutation.getPartitionUpdates())
+            {
+                assert keyspace.getName().equals(cf.metadata().ksName);
+
+                if (coordinatorBatchlog && keyspace.getReplicationStrategy().getReplicationFactor() == 1)
+                    continue;
+
+                for (View view : allViews())
+                {
+                    if (!cf.metadata().cfId.equals(view.getDefinition().baseTableId))
+                        continue;
+
+                    if (view.updateAffectsView(cf))
+                        return true;
+                }
+            }
+        }
+
+        return false;
+    }
+
+    public Iterable<View> allViews()
+    {
+        return viewsByName.values();
+    }
+
+    public void update(String viewName)
+    {
+        View view = viewsByName.get(viewName);
+        assert view != null : "When updating a view, it should already be in the ViewManager";
+        view.build();
+
+        // We provide the new definition from the base metadata
+        Optional<ViewDefinition> viewDefinition = keyspace.getMetadata().views.get(viewName);
+        assert viewDefinition.isPresent() : "When updating a view, it should still be in the Keyspaces views";
+        view.updateDefinition(viewDefinition.get());
+    }
+
+    public void reload()
+    {
+        Map<String, ViewDefinition> newViewsByName = new HashMap<>();
+        for (ViewDefinition definition : keyspace.getMetadata().views)
+        {
+            newViewsByName.put(definition.viewName, definition);
+        }
+
+        for (String viewName : viewsByName.keySet())
+        {
+            if (!newViewsByName.containsKey(viewName))
+                removeView(viewName);
+        }
+
+        for (Map.Entry<String, ViewDefinition> entry : newViewsByName.entrySet())
+        {
+            if (!viewsByName.containsKey(entry.getKey()))
+                addView(entry.getValue());
+        }
+
+        for (View view : allViews())
+        {
+            view.build();
+            // We provide the new definition from the base metadata
+            view.updateDefinition(newViewsByName.get(view.name));
+        }
+    }
+
+    public void addView(ViewDefinition definition)
+    {
+        View view = new View(definition, keyspace.getColumnFamilyStore(definition.baseTableId));
+        forTable(view.getDefinition().baseTableId).addView(view);
+        viewsByName.put(definition.viewName, view);
+    }
+
+    public void removeView(String name)
+    {
+        View view = viewsByName.remove(name);
+
+        if (view == null)
+            return;
+
+        forTable(view.getDefinition().baseTableId).removeView(name);
+        SystemKeyspace.setViewRemoved(keyspace.getName(), view.name);
+    }
+
+    public void buildAllViews()
+    {
+        for (View view : allViews())
+            view.build();
+    }
+
+    public ForStore forTable(UUID baseId)
+    {
+        ForStore forStore = viewManagersByStore.get(baseId);
+        if (forStore == null)
+        {
+            forStore = new ForStore();
+            ForStore previous = viewManagersByStore.put(baseId, forStore);
+            if (previous != null)
+                forStore = previous;
+        }
+        return forStore;
+    }
+
+    public static Lock acquireLockFor(ByteBuffer key)
+    {
+        Lock lock = LOCKS.get(key);
+
+        if (lock.tryLock())
+            return lock;
+
+        return null;
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/ViewUtils.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/ViewUtils.java b/src/java/org/apache/cassandra/db/view/ViewUtils.java
new file mode 100644
index 0000000..628142d
--- /dev/null
+++ b/src/java/org/apache/cassandra/db/view/ViewUtils.java
@@ -0,0 +1,109 @@
+/*
+ * 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.view;
+
+import java.net.InetAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.cassandra.config.DatabaseDescriptor;
+import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.locator.AbstractReplicationStrategy;
+import org.apache.cassandra.service.StorageService;
+import org.apache.cassandra.utils.FBUtilities;
+
+public final class ViewUtils
+{
+    private ViewUtils()
+    {
+    }
+
+    /**
+     * Calculate the natural endpoint for the view.
+     *
+     * The view natural endpoint is the endpint which has the same cardinality as this node in the replication factor.
+     * The cardinality is the number at which this node would store a piece of data, given the change in replication
+     * factor.
+     *
+     * For example, if we have the following ring:
+     *   A, T1 -> B, T2 -> C, T3 -> A
+     *
+     * For the token T1, at RF=1, A would be included, so A's cardinality for T1 is 1. For the token T1, at RF=2, B would
+     * be included, so B's cardinality for token T1 is 2. For token T3, at RF = 2, A would be included, so A's cardinality
+     * for T3 is 2.
+     *
+     * For a view whose base token is T1 and whose view token is T3, the pairings between the nodes would be:
+     *  A writes to C (A's cardinality is 1 for T1, and C's cardinality is 1 for T3)
+     *  B writes to A (B's cardinality is 2 for T1, and A's cardinality is 2 for T3)
+     *  C writes to B (C's cardinality is 3 for T1, and B's cardinality is 3 for T3)
+     *
+     * @throws RuntimeException if this method is called using a base token which does not belong to this replica
+     */
+    public static InetAddress getViewNaturalEndpoint(String keyspaceName, Token baseToken, Token viewToken)
+    {
+        AbstractReplicationStrategy replicationStrategy = Keyspace.open(keyspaceName).getReplicationStrategy();
+
+        String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
+        List<InetAddress> localBaseEndpoints = new ArrayList<>();
+        List<InetAddress> localViewEndpoints = new ArrayList<>();
+        for (InetAddress baseEndpoint : replicationStrategy.getNaturalEndpoints(baseToken))
+        {
+            if (DatabaseDescriptor.getEndpointSnitch().getDatacenter(baseEndpoint).equals(localDataCenter))
+                localBaseEndpoints.add(baseEndpoint);
+        }
+
+        for (InetAddress viewEndpoint : replicationStrategy.getNaturalEndpoints(viewToken))
+        {
+            // If we are a base endpoint which is also a view replica, we use ourselves as our view replica
+            if (viewEndpoint.equals(FBUtilities.getBroadcastAddress()))
+                return viewEndpoint;
+
+            // We have to remove any endpoint which is shared between the base and the view, as it will select itself
+            // and throw off the counts otherwise.
+            if (localBaseEndpoints.contains(viewEndpoint))
+                localBaseEndpoints.remove(viewEndpoint);
+            else if (DatabaseDescriptor.getEndpointSnitch().getDatacenter(viewEndpoint).equals(localDataCenter))
+                localViewEndpoints.add(viewEndpoint);
+        }
+
+        // The replication strategy will be the same for the base and the view, as they must belong to the same keyspace.
+        // Since the same replication strategy is used, the same placement should be used and we should get the same
+        // number of replicas for all of the tokens in the ring.
+        assert localBaseEndpoints.size() == localViewEndpoints.size() : "Replication strategy should have the same number of endpoints for the base and the view";
+        int baseIdx = localBaseEndpoints.indexOf(FBUtilities.getBroadcastAddress());
+
+        if (baseIdx < 0)
+        {
+
+            if (StorageService.instance.getTokenMetadata().pendingEndpointsFor(viewToken, keyspaceName).size() > 0)
+            {
+                //Since there are pending endpoints we are going to store hints this in the batchlog regardless.
+                //So we can pretend we are the views endpoint.
+
+                return FBUtilities.getBroadcastAddress();
+            }
+
+            throw new RuntimeException("Trying to get the view natural endpoint on a non-data replica");
+        }
+
+
+        return localViewEndpoints.get(baseIdx);
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
index 46872c1..bbf6fd6 100644
--- a/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/ByteOrderedPartitioner.java
@@ -276,7 +276,7 @@ public class ByteOrderedPartitioner implements IPartitioner
 
         for (String ks : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfmd : Schema.instance.getTables(ks))
+            for (CFMetaData cfmd : Schema.instance.getTablesAndViews(ks))
             {
                 for (Range<Token> r : sortedRanges)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
index 464ac3d..96b4ca0 100644
--- a/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
+++ b/src/java/org/apache/cassandra/dht/OrderPreservingPartitioner.java
@@ -216,7 +216,7 @@ public class OrderPreservingPartitioner implements IPartitioner
 
         for (String ks : Schema.instance.getKeyspaces())
         {
-            for (CFMetaData cfmd : Schema.instance.getTables(ks))
+            for (CFMetaData cfmd : Schema.instance.getTablesAndViews(ks))
             {
                 for (Range<Token> r : sortedRanges)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/metrics/MVWriteMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/MVWriteMetrics.java b/src/java/org/apache/cassandra/metrics/MVWriteMetrics.java
deleted file mode 100644
index 39a5574..0000000
--- a/src/java/org/apache/cassandra/metrics/MVWriteMetrics.java
+++ /dev/null
@@ -1,42 +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.metrics;
-
-import com.codahale.metrics.Counter;
-
-import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
-
-public class MVWriteMetrics extends ClientRequestMetrics
-{
-    public final Counter viewReplicasAttempted;
-    public final Counter viewReplicasSuccess;
-
-    public MVWriteMetrics(String scope) {
-        super(scope);
-        viewReplicasAttempted = Metrics.counter(factory.createMetricName("ViewReplicasAttempted"));
-        viewReplicasSuccess = Metrics.counter(factory.createMetricName("ViewReplicasSuccess"));
-    }
-
-    public void release()
-    {
-        super.release();
-        Metrics.remove(factory.createMetricName("ViewReplicasAttempted"));
-        Metrics.remove(factory.createMetricName("ViewReplicasSuccess"));
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/metrics/ViewWriteMetrics.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/metrics/ViewWriteMetrics.java b/src/java/org/apache/cassandra/metrics/ViewWriteMetrics.java
new file mode 100644
index 0000000..c99cc5c
--- /dev/null
+++ b/src/java/org/apache/cassandra/metrics/ViewWriteMetrics.java
@@ -0,0 +1,42 @@
+/*
+ * 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.metrics;
+
+import com.codahale.metrics.Counter;
+
+import static org.apache.cassandra.metrics.CassandraMetricsRegistry.Metrics;
+
+public class ViewWriteMetrics extends ClientRequestMetrics
+{
+    public final Counter viewReplicasAttempted;
+    public final Counter viewReplicasSuccess;
+
+    public ViewWriteMetrics(String scope) {
+        super(scope);
+        viewReplicasAttempted = Metrics.counter(factory.createMetricName("ViewReplicasAttempted"));
+        viewReplicasSuccess = Metrics.counter(factory.createMetricName("ViewReplicasSuccess"));
+    }
+
+    public void release()
+    {
+        super.release();
+        Metrics.remove(factory.createMetricName("ViewReplicasAttempted"));
+        Metrics.remove(factory.createMetricName("ViewReplicasSuccess"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
index 372ff6e..0a0d9d8 100644
--- a/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
+++ b/src/java/org/apache/cassandra/schema/KeyspaceMetadata.java
@@ -22,9 +22,12 @@ import java.util.Optional;
 import java.util.Set;
 
 import com.google.common.base.Objects;
+import com.google.common.collect.Iterables;
 
+import org.apache.avro.reflect.Nullable;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.exceptions.ConfigurationException;
 
 /**
@@ -35,51 +38,72 @@ public final class KeyspaceMetadata
     public final String name;
     public final KeyspaceParams params;
     public final Tables tables;
+    public final Views views;
     public final Types types;
     public final Functions functions;
 
-    private KeyspaceMetadata(String name, KeyspaceParams params, Tables tables, Types types, Functions functions)
+    private KeyspaceMetadata(String name, KeyspaceParams params, Tables tables, Views views, Types types, Functions functions)
     {
         this.name = name;
         this.params = params;
         this.tables = tables;
+        this.views = views;
         this.types = types;
         this.functions = functions;
     }
 
     public static KeyspaceMetadata create(String name, KeyspaceParams params)
     {
-        return new KeyspaceMetadata(name, params, Tables.none(), Types.none(), Functions.none());
+        return new KeyspaceMetadata(name, params, Tables.none(), Views.none(), Types.none(), Functions.none());
     }
 
     public static KeyspaceMetadata create(String name, KeyspaceParams params, Tables tables)
     {
-        return new KeyspaceMetadata(name, params, tables, Types.none(), Functions.none());
+        return new KeyspaceMetadata(name, params, tables, Views.none(), Types.none(), Functions.none());
     }
 
-    public static KeyspaceMetadata create(String name, KeyspaceParams params, Tables tables, Types types, Functions functions)
+    public static KeyspaceMetadata create(String name, KeyspaceParams params, Tables tables, Views views, Types types, Functions functions)
     {
-        return new KeyspaceMetadata(name, params, tables, types, functions);
+        return new KeyspaceMetadata(name, params, tables, views, types, functions);
     }
 
     public KeyspaceMetadata withSwapped(KeyspaceParams params)
     {
-        return new KeyspaceMetadata(name, params, tables, types, functions);
+        return new KeyspaceMetadata(name, params, tables, views, types, functions);
     }
 
-    public KeyspaceMetadata withSwapped(Tables tables)
+    public KeyspaceMetadata withSwapped(Tables regular)
     {
-        return new KeyspaceMetadata(name, params, tables, types, functions);
+        return new KeyspaceMetadata(name, params, regular, views, types, functions);
+    }
+
+    public KeyspaceMetadata withSwapped(Views views)
+    {
+        return new KeyspaceMetadata(name, params, tables, views, types, functions);
     }
 
     public KeyspaceMetadata withSwapped(Types types)
     {
-        return new KeyspaceMetadata(name, params, tables, types, functions);
+        return new KeyspaceMetadata(name, params, tables, views, types, functions);
     }
 
     public KeyspaceMetadata withSwapped(Functions functions)
     {
-        return new KeyspaceMetadata(name, params, tables, types, functions);
+        return new KeyspaceMetadata(name, params, tables, views, types, functions);
+    }
+
+    public Iterable<CFMetaData> tablesAndViews()
+    {
+        return Iterables.concat(tables, views.metadatas());
+    }
+
+    @Nullable
+    public CFMetaData getTableOrViewNullable(String tableOrViewName)
+    {
+        ViewDefinition view = views.getNullable(tableOrViewName);
+        return view == null
+             ? tables.getNullable(tableOrViewName)
+             : view.metadata;
     }
 
     public Set<String> existingIndexNames(String cfToExclude)
@@ -94,7 +118,7 @@ public final class KeyspaceMetadata
 
     public Optional<CFMetaData> findIndexedTable(String indexName)
     {
-        for (CFMetaData cfm : tables)
+        for (CFMetaData cfm : tablesAndViews())
             if (cfm.getIndexes().has(indexName))
                 return Optional.of(cfm);
 
@@ -104,7 +128,7 @@ public final class KeyspaceMetadata
     @Override
     public int hashCode()
     {
-        return Objects.hashCode(name, params, tables, functions, types);
+        return Objects.hashCode(name, params, tables, views, functions, types);
     }
 
     @Override
@@ -121,6 +145,7 @@ public final class KeyspaceMetadata
         return name.equals(other.name)
             && params.equals(other.params)
             && tables.equals(other.tables)
+            && views.equals(other.views)
             && functions.equals(other.functions)
             && types.equals(other.types);
     }
@@ -132,6 +157,7 @@ public final class KeyspaceMetadata
                       .add("name", name)
                       .add("params", params)
                       .add("tables", tables)
+                      .add("views", views)
                       .add("functions", functions)
                       .add("types", types)
                       .toString();
@@ -144,6 +170,6 @@ public final class KeyspaceMetadata
                                                            + "or contain non-alphanumeric-underscore characters (got \"%s\")",
                                                            Schema.NAME_LENGTH,
                                                            name));
-        tables.forEach(CFMetaData::validate);
+        tablesAndViews().forEach(CFMetaData::validate);
     }
 }


[6/7] cassandra git commit: Improve MV schema representation

Posted by al...@apache.org.
Improve MV schema representation

patch by Carl Yeksigian; reviewed by Aleksey Yeschenko for
CASSANDRA-9921


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

Branch: refs/heads/trunk
Commit: a3a8dbca5cc190caea36c5029d83f02977b6d149
Parents: f542a20
Author: Carl Yeksigian <ca...@apache.org>
Authored: Tue Aug 11 12:01:48 2015 -0400
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Sep 16 15:34:18 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 ...ore-3.0.0-alpha3-55db84c-SNAPSHOT-shaded.jar |  Bin 2209303 -> 0 bytes
 ...ore-3.0.0-alpha3-8bd064d-SNAPSHOT-shaded.jar |  Bin 0 -> 2218913 bytes
 ...iver-internal-only-3.0.0a2.post0-2429ba3.zip |  Bin 229078 -> 0 bytes
 ...iver-internal-only-3.0.0a2.post0-96883eb.zip |  Bin 0 -> 230630 bytes
 .../org/apache/cassandra/concurrent/Stage.java  |    4 +-
 .../cassandra/concurrent/StageManager.java      |    2 +-
 .../org/apache/cassandra/config/CFMetaData.java |   55 +-
 .../cassandra/config/DatabaseDescriptor.java    |    2 +-
 .../config/MaterializedViewDefinition.java      |   93 --
 .../org/apache/cassandra/config/Schema.java     |  111 +-
 .../apache/cassandra/config/ViewDefinition.java |  114 ++
 src/java/org/apache/cassandra/cql3/Cql.g        |   12 +-
 .../AlterMaterializedViewStatement.java         |   91 --
 .../cql3/statements/AlterTableStatement.java    |   99 +-
 .../cql3/statements/AlterTypeStatement.java     |   10 +
 .../cql3/statements/AlterViewStatement.java     |   94 ++
 .../CreateMaterializedViewStatement.java        |  286 ----
 .../cql3/statements/CreateTriggerStatement.java |    2 +-
 .../cql3/statements/CreateViewStatement.java    |  321 +++++
 .../DropMaterializedViewStatement.java          |   92 --
 .../cql3/statements/DropTableStatement.java     |   21 +-
 .../cql3/statements/DropTypeStatement.java      |    2 +-
 .../cql3/statements/DropViewStatement.java      |   93 ++
 .../cql3/statements/ModificationStatement.java  |   38 +-
 .../cql3/statements/SelectStatement.java        |    6 +-
 .../cql3/statements/TruncateStatement.java      |    2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   18 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   20 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |   72 +-
 src/java/org/apache/cassandra/db/WriteType.java |    2 +-
 .../db/compaction/CompactionManager.java        |    4 +-
 .../cassandra/db/compaction/OperationType.java  |    2 +-
 .../cassandra/db/view/MaterializedView.java     |  749 ----------
 .../db/view/MaterializedViewBuilder.java        |  215 ---
 .../db/view/MaterializedViewManager.java        |  241 ----
 .../db/view/MaterializedViewUtils.java          |  109 --
 src/java/org/apache/cassandra/db/view/View.java |  682 +++++++++
 .../apache/cassandra/db/view/ViewBuilder.java   |  214 +++
 .../apache/cassandra/db/view/ViewManager.java   |  271 ++++
 .../org/apache/cassandra/db/view/ViewUtils.java |  109 ++
 .../cassandra/dht/ByteOrderedPartitioner.java   |    2 +-
 .../dht/OrderPreservingPartitioner.java         |    2 +-
 .../cassandra/metrics/MVWriteMetrics.java       |   42 -
 .../cassandra/metrics/ViewWriteMetrics.java     |   42 +
 .../cassandra/schema/KeyspaceMetadata.java      |   52 +-
 .../cassandra/schema/MaterializedViews.java     |  149 --
 .../apache/cassandra/schema/SchemaKeyspace.java |  292 ++--
 src/java/org/apache/cassandra/schema/Views.java |  171 +++
 .../cassandra/service/CassandraDaemon.java      |   11 +-
 .../cassandra/service/MigrationListener.java    |   14 +
 .../cassandra/service/MigrationManager.java     |   62 +-
 .../apache/cassandra/service/StartupChecks.java |    2 +-
 .../apache/cassandra/service/StorageProxy.java  |   60 +-
 .../cassandra/service/StorageService.java       |   18 +-
 .../cassandra/streaming/StreamReceiveTask.java  |   15 +-
 .../cassandra/thrift/CassandraServer.java       |   19 +-
 .../cassandra/thrift/ThriftConversion.java      |    8 +-
 .../utils/NativeSSTableLoaderClient.java        |   82 +-
 .../cql3/MaterializedViewLongTest.java          |  188 ---
 .../org/apache/cassandra/cql3/ViewLongTest.java |  188 +++
 .../unit/org/apache/cassandra/SchemaLoader.java |    2 +-
 .../config/DatabaseDescriptorTest.java          |    2 +-
 .../cassandra/cql3/MaterializedViewTest.java    | 1251 ----------------
 .../org/apache/cassandra/cql3/ViewTest.java     | 1355 ++++++++++++++++++
 .../db/view/MaterializedViewUtilsTest.java      |  115 --
 .../apache/cassandra/db/view/ViewUtilsTest.java |  115 ++
 .../apache/cassandra/dht/LengthPartitioner.java |    2 +-
 .../org/apache/cassandra/hints/HintTest.java    |    2 +-
 .../schema/LegacySchemaMigratorTest.java        |    3 +
 70 files changed, 4484 insertions(+), 4041 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index c688615..b213260 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,5 @@
 3.0.0-rc1
+ * Improve MV schema representation (CASSANDRA-9921)
  * Add flag to enable/disable coordinator batchlog for MV writes (CASSANDRA-10230)
  * Update cqlsh COPY for new internal driver serialization interface (CASSANDRA-10318)
  * Give index implementations more control over rebuild operations (CASSANDRA-10312)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/lib/cassandra-driver-core-3.0.0-alpha3-55db84c-SNAPSHOT-shaded.jar
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-core-3.0.0-alpha3-55db84c-SNAPSHOT-shaded.jar b/lib/cassandra-driver-core-3.0.0-alpha3-55db84c-SNAPSHOT-shaded.jar
deleted file mode 100644
index 385d947..0000000
Binary files a/lib/cassandra-driver-core-3.0.0-alpha3-55db84c-SNAPSHOT-shaded.jar and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/lib/cassandra-driver-core-3.0.0-alpha3-8bd064d-SNAPSHOT-shaded.jar
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-core-3.0.0-alpha3-8bd064d-SNAPSHOT-shaded.jar b/lib/cassandra-driver-core-3.0.0-alpha3-8bd064d-SNAPSHOT-shaded.jar
new file mode 100644
index 0000000..fc5d2f0
Binary files /dev/null and b/lib/cassandra-driver-core-3.0.0-alpha3-8bd064d-SNAPSHOT-shaded.jar differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/lib/cassandra-driver-internal-only-3.0.0a2.post0-2429ba3.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.0.0a2.post0-2429ba3.zip b/lib/cassandra-driver-internal-only-3.0.0a2.post0-2429ba3.zip
deleted file mode 100644
index 435c5e1..0000000
Binary files a/lib/cassandra-driver-internal-only-3.0.0a2.post0-2429ba3.zip and /dev/null differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/lib/cassandra-driver-internal-only-3.0.0a2.post0-96883eb.zip
----------------------------------------------------------------------
diff --git a/lib/cassandra-driver-internal-only-3.0.0a2.post0-96883eb.zip b/lib/cassandra-driver-internal-only-3.0.0a2.post0-96883eb.zip
new file mode 100644
index 0000000..e55b4c3
Binary files /dev/null and b/lib/cassandra-driver-internal-only-3.0.0a2.post0-96883eb.zip differ

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/concurrent/Stage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/Stage.java b/src/java/org/apache/cassandra/concurrent/Stage.java
index a57587c..ccb1565 100644
--- a/src/java/org/apache/cassandra/concurrent/Stage.java
+++ b/src/java/org/apache/cassandra/concurrent/Stage.java
@@ -27,7 +27,7 @@ public enum Stage
     READ,
     MUTATION,
     COUNTER_MUTATION,
-    MATERIALIZED_VIEW_MUTATION,
+    VIEW_MUTATION,
     GOSSIP,
     REQUEST_RESPONSE,
     ANTI_ENTROPY,
@@ -61,7 +61,7 @@ public enum Stage
                 return "internal";
             case MUTATION:
             case COUNTER_MUTATION:
-            case MATERIALIZED_VIEW_MUTATION:
+            case VIEW_MUTATION:
             case READ:
             case REQUEST_RESPONSE:
             case READ_REPAIR:

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/concurrent/StageManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/concurrent/StageManager.java b/src/java/org/apache/cassandra/concurrent/StageManager.java
index ee1fbe5..a2ad892 100644
--- a/src/java/org/apache/cassandra/concurrent/StageManager.java
+++ b/src/java/org/apache/cassandra/concurrent/StageManager.java
@@ -47,7 +47,7 @@ public class StageManager
     {
         stages.put(Stage.MUTATION, multiThreadedLowSignalStage(Stage.MUTATION, getConcurrentWriters()));
         stages.put(Stage.COUNTER_MUTATION, multiThreadedLowSignalStage(Stage.COUNTER_MUTATION, getConcurrentCounterWriters()));
-        stages.put(Stage.MATERIALIZED_VIEW_MUTATION, multiThreadedLowSignalStage(Stage.MATERIALIZED_VIEW_MUTATION, getConcurrentMaterializedViewWriters()));
+        stages.put(Stage.VIEW_MUTATION, multiThreadedLowSignalStage(Stage.VIEW_MUTATION, getConcurrentViewWriters()));
         stages.put(Stage.READ, multiThreadedLowSignalStage(Stage.READ, getConcurrentReaders()));
         stages.put(Stage.REQUEST_RESPONSE, multiThreadedLowSignalStage(Stage.REQUEST_RESPONSE, FBUtilities.getAvailableProcessors()));
         stages.put(Stage.INTERNAL_RESPONSE, multiThreadedStage(Stage.INTERNAL_RESPONSE, FBUtilities.getAvailableProcessors()));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/config/CFMetaData.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/CFMetaData.java b/src/java/org/apache/cassandra/config/CFMetaData.java
index a26b19d..cf1dbbf 100644
--- a/src/java/org/apache/cassandra/config/CFMetaData.java
+++ b/src/java/org/apache/cassandra/config/CFMetaData.java
@@ -62,7 +62,7 @@ public final class CFMetaData
 {
     public enum Flag
     {
-        SUPER, COUNTER, DENSE, COMPOUND, VIEW
+        SUPER, COUNTER, DENSE, COMPOUND
     }
 
     private static final Logger logger = LoggerFactory.getLogger(CFMetaData.class);
@@ -79,7 +79,7 @@ public final class CFMetaData
     private final boolean isCompound;
     private final boolean isSuper;
     private final boolean isCounter;
-    private final boolean isMaterializedView;
+    private final boolean isView;
 
     private final boolean isIndex;
 
@@ -94,7 +94,6 @@ public final class CFMetaData
     private volatile AbstractType<?> keyValidator = BytesType.instance;
     private volatile Map<ByteBuffer, DroppedColumn> droppedColumns = new HashMap<>();
     private volatile Triggers triggers = Triggers.none();
-    private volatile MaterializedViews materializedViews = MaterializedViews.none();
     private volatile Indexes indexes = Indexes.none();
 
     /*
@@ -219,12 +218,6 @@ public final class CFMetaData
         return this;
     }
 
-    public CFMetaData materializedViews(MaterializedViews prop)
-    {
-        materializedViews = prop;
-        return this;
-    }
-
     public CFMetaData indexes(Indexes indexes)
     {
         this.indexes = indexes;
@@ -238,7 +231,7 @@ public final class CFMetaData
                        boolean isCounter,
                        boolean isDense,
                        boolean isCompound,
-                       boolean isMaterializedView,
+                       boolean isView,
                        List<ColumnDefinition> partitionKeyColumns,
                        List<ColumnDefinition> clusteringColumns,
                        PartitionColumns partitionColumns,
@@ -252,7 +245,7 @@ public final class CFMetaData
         this.isCompound = isCompound;
         this.isSuper = isSuper;
         this.isCounter = isCounter;
-        this.isMaterializedView = isMaterializedView;
+        this.isView = isView;
 
         EnumSet<Flag> flags = EnumSet.noneOf(Flag.class);
         if (isSuper)
@@ -263,8 +256,6 @@ public final class CFMetaData
             flags.add(Flag.DENSE);
         if (isCompound)
             flags.add(Flag.COMPOUND);
-        if (isMaterializedView)
-            flags.add(Flag.VIEW);
         this.flags = Sets.immutableEnumSet(flags);
 
         isIndex = cfName.contains(".");
@@ -307,16 +298,6 @@ public final class CFMetaData
             this.compactValueColumn = CompactTables.getCompactValueColumn(partitionColumns, isSuper());
     }
 
-    public MaterializedViews getMaterializedViews()
-    {
-        return materializedViews;
-    }
-
-    public boolean hasMaterializedViews()
-    {
-        return !materializedViews.isEmpty();
-    }
-
     public Indexes getIndexes()
     {
         return indexes;
@@ -329,7 +310,7 @@ public final class CFMetaData
                                     boolean isCompound,
                                     boolean isSuper,
                                     boolean isCounter,
-                                    boolean isMaterializedView,
+                                    boolean isView,
                                     List<ColumnDefinition> columns,
                                     IPartitioner partitioner)
     {
@@ -363,7 +344,7 @@ public final class CFMetaData
                               isCounter,
                               isDense,
                               isCompound,
-                              isMaterializedView,
+                              isView,
                               partitions,
                               clusterings,
                               builder.build(),
@@ -464,7 +445,7 @@ public final class CFMetaData
                                        isCounter(),
                                        isDense(),
                                        isCompound(),
-                                       isMaterializedView(),
+                                       isView(),
                                        copy(partitionKeyColumns),
                                        copy(clusteringColumns),
                                        copy(partitionColumns),
@@ -481,7 +462,7 @@ public final class CFMetaData
                                        isCounter,
                                        isDense,
                                        isCompound,
-                                       isMaterializedView,
+                                       isView,
                                        copy(partitionKeyColumns),
                                        copy(clusteringColumns),
                                        copy(partitionColumns),
@@ -511,7 +492,6 @@ public final class CFMetaData
         return newCFMD.params(oldCFMD.params)
                       .droppedColumns(new HashMap<>(oldCFMD.droppedColumns))
                       .triggers(oldCFMD.triggers)
-                      .materializedViews(oldCFMD.materializedViews)
                       .indexes(oldCFMD.indexes);
     }
 
@@ -702,7 +682,6 @@ public final class CFMetaData
             && Objects.equal(columnMetadata, other.columnMetadata)
             && Objects.equal(droppedColumns, other.droppedColumns)
             && Objects.equal(triggers, other.triggers)
-            && Objects.equal(materializedViews, other.materializedViews)
             && Objects.equal(indexes, other.indexes);
     }
 
@@ -720,7 +699,6 @@ public final class CFMetaData
             .append(columnMetadata)
             .append(droppedColumns)
             .append(triggers)
-            .append(materializedViews)
             .append(indexes)
             .toHashCode();
     }
@@ -731,7 +709,8 @@ public final class CFMetaData
      */
     public boolean reload()
     {
-        return apply(SchemaKeyspace.createTableFromName(ksName, cfName));
+        return apply(isView ? SchemaKeyspace.createViewFromName(ksName, cfName).metadata
+                            : SchemaKeyspace.createTableFromName(ksName, cfName));
     }
 
     /**
@@ -766,7 +745,6 @@ public final class CFMetaData
             droppedColumns = cfm.droppedColumns;
 
         triggers = cfm.triggers;
-        materializedViews = cfm.materializedViews;
         indexes = cfm.indexes;
 
         logger.debug("application result is {}", this);
@@ -1070,9 +1048,9 @@ public final class CFMetaData
         return isCompound;
     }
 
-    public boolean isMaterializedView()
+    public boolean isView()
     {
-        return isMaterializedView;
+        return isView;
     }
 
     public Serializers serializers()
@@ -1121,7 +1099,6 @@ public final class CFMetaData
             .append("columnMetadata", columnMetadata.values())
             .append("droppedColumns", droppedColumns)
             .append("triggers", triggers)
-            .append("materializedViews", materializedViews)
             .append("indexes", indexes)
             .toString();
     }
@@ -1134,7 +1111,7 @@ public final class CFMetaData
         private final boolean isCompound;
         private final boolean isSuper;
         private final boolean isCounter;
-        private final boolean isMaterializedView;
+        private final boolean isView;
         private IPartitioner partitioner;
 
         private UUID tableId;
@@ -1144,7 +1121,7 @@ public final class CFMetaData
         private final List<Pair<ColumnIdentifier, AbstractType>> staticColumns = new ArrayList<>();
         private final List<Pair<ColumnIdentifier, AbstractType>> regularColumns = new ArrayList<>();
 
-        private Builder(String keyspace, String table, boolean isDense, boolean isCompound, boolean isSuper, boolean isCounter, boolean isMaterializedView)
+        private Builder(String keyspace, String table, boolean isDense, boolean isCompound, boolean isSuper, boolean isCounter, boolean isView)
         {
             this.keyspace = keyspace;
             this.table = table;
@@ -1152,7 +1129,7 @@ public final class CFMetaData
             this.isCompound = isCompound;
             this.isSuper = isSuper;
             this.isCounter = isCounter;
-            this.isMaterializedView = isMaterializedView;
+            this.isView = isView;
             this.partitioner = DatabaseDescriptor.getPartitioner();
         }
 
@@ -1296,7 +1273,7 @@ public final class CFMetaData
                                   isCounter,
                                   isDense,
                                   isCompound,
-                                  isMaterializedView,
+                                  isView,
                                   partitions,
                                   clusterings,
                                   builder.build(),

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
index ed220b8..7553c92 100644
--- a/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
+++ b/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
@@ -1143,7 +1143,7 @@ public class DatabaseDescriptor
         return conf.concurrent_counter_writes;
     }
 
-    public static int getConcurrentMaterializedViewWriters()
+    public static int getConcurrentViewWriters()
     {
         return conf.concurrent_materialized_view_writes;
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/config/MaterializedViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/MaterializedViewDefinition.java b/src/java/org/apache/cassandra/config/MaterializedViewDefinition.java
deleted file mode 100644
index 90fa35c..0000000
--- a/src/java/org/apache/cassandra/config/MaterializedViewDefinition.java
+++ /dev/null
@@ -1,93 +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.config;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.cassandra.cql3.ColumnIdentifier;
-
-public class MaterializedViewDefinition
-{
-    public final String baseCfName;
-    public final String viewName;
-    // The order of partititon columns and clustering columns is important, so we cannot switch these two to sets
-    public final List<ColumnIdentifier> partitionColumns;
-    public final List<ColumnIdentifier> clusteringColumns;
-    public final Set<ColumnIdentifier> included;
-    public final boolean includeAll;
-
-    public MaterializedViewDefinition(MaterializedViewDefinition def)
-    {
-        this(def.baseCfName, def.viewName, new ArrayList<>(def.partitionColumns), new ArrayList<>(def.clusteringColumns), new HashSet<>(def.included));
-    }
-
-    /**
-     * @param baseCfName        Name of the column family from which this view is based
-     * @param viewName          Name of the view
-     * @param partitionColumns  List of all of the partition columns, in the order they are defined
-     * @param clusteringColumns List of all of the clustering columns, in the order they are defined
-     * @param included
-     */
-    public MaterializedViewDefinition(String baseCfName, String viewName, List<ColumnIdentifier> partitionColumns, List<ColumnIdentifier> clusteringColumns, Set<ColumnIdentifier> included)
-    {
-        assert partitionColumns != null && !partitionColumns.isEmpty();
-        assert included != null;
-        this.baseCfName = baseCfName;
-        this.viewName = viewName;
-        this.partitionColumns = partitionColumns;
-        this.clusteringColumns = clusteringColumns;
-        this.includeAll = included.isEmpty();
-        this.included = included;
-    }
-
-    /**
-     * @return true if the view specified by this definition will include the column, false otherwise
-     */
-    public boolean includes(ColumnIdentifier column)
-    {
-        return includeAll
-               || partitionColumns.contains(column)
-               || clusteringColumns.contains(column)
-               || included.contains(column);
-    }
-
-    /**
-     * Replace the column {@param from} with {@param to} in this materialized view definition's partition,
-     * clustering, or included columns.
-     */
-    public void renameColumn(ColumnIdentifier from, ColumnIdentifier to)
-    {
-        if (!includeAll && included.contains(from))
-        {
-            included.remove(from);
-            included.add(to);
-        }
-
-        int partitionIndex = partitionColumns.indexOf(from);
-        if (partitionIndex >= 0)
-            partitionColumns.set(partitionIndex, to);
-
-        int clusteringIndex = clusteringColumns.indexOf(from);
-        if (clusteringIndex >= 0)
-            clusteringColumns.set(clusteringIndex, to);
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/config/Schema.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/Schema.java b/src/java/org/apache/cassandra/config/Schema.java
index 7cc8394..bcde978 100644
--- a/src/java/org/apache/cassandra/config/Schema.java
+++ b/src/java/org/apache/cassandra/config/Schema.java
@@ -147,6 +147,7 @@ public class Schema
     public Schema load(KeyspaceMetadata keyspaceDef)
     {
         keyspaceDef.tables.forEach(this::load);
+        keyspaceDef.views.forEach(this::load);
         setKeyspaceMetadata(keyspaceDef);
         return this;
     }
@@ -224,8 +225,11 @@ public class Schema
     public CFMetaData getCFMetaData(String keyspaceName, String cfName)
     {
         assert keyspaceName != null;
+
         KeyspaceMetadata ksm = keyspaces.get(keyspaceName);
-        return (ksm == null) ? null : ksm.tables.getNullable(cfName);
+        return ksm == null
+             ? null
+             : ksm.getTableOrViewNullable(cfName);
     }
 
     /**
@@ -246,6 +250,13 @@ public class Schema
         return getCFMetaData(descriptor.ksname, descriptor.cfname);
     }
 
+    public ViewDefinition getView(String keyspaceName, String viewName)
+    {
+        assert keyspaceName != null;
+        KeyspaceMetadata ksm = keyspaces.get(keyspaceName);
+        return (ksm == null) ? null : ksm.views.getNullable(viewName);
+    }
+
     /**
      * Get metadata about keyspace by its name
      *
@@ -274,12 +285,12 @@ public class Schema
      *
      * @return metadata about ColumnFamilies the belong to the given keyspace
      */
-    public Tables getTables(String keyspaceName)
+    public Iterable<CFMetaData> getTablesAndViews(String keyspaceName)
     {
         assert keyspaceName != null;
         KeyspaceMetadata ksm = keyspaces.get(keyspaceName);
         assert ksm != null;
-        return ksm.tables;
+        return ksm.tablesAndViews();
     }
 
     /**
@@ -356,6 +367,24 @@ public class Schema
     }
 
     /**
+     * Load individual View Definition to the schema
+     * (to make View lookup faster)
+     *
+     * @param view The View definition to load
+     */
+    public void load(ViewDefinition view)
+    {
+        CFMetaData cfm = view.metadata;
+        Pair<String, String> key = Pair.create(cfm.ksName, cfm.cfName);
+
+        if (cfIdMap.containsKey(key))
+            throw new RuntimeException(String.format("Attempting to load already loaded view %s.%s", cfm.ksName, cfm.cfName));
+
+        logger.debug("Adding {} to cfIdMap", cfm);
+        cfIdMap.put(key, cfm.cfId);
+    }
+
+    /**
      * Used for ColumnFamily data eviction out from the schema
      *
      * @param cfm The ColumnFamily Definition to evict
@@ -365,6 +394,16 @@ public class Schema
         cfIdMap.remove(Pair.create(cfm.ksName, cfm.cfName));
     }
 
+    /**
+     * Used for View eviction from the schema
+     *
+     * @param view The view definition to evict
+     */
+    private void unload(ViewDefinition view)
+    {
+        cfIdMap.remove(Pair.create(view.ksName, view.viewName));
+    }
+
     /* Function helpers */
 
     /**
@@ -442,6 +481,7 @@ public class Schema
         {
             KeyspaceMetadata ksm = getKSMetaData(keyspaceName);
             ksm.tables.forEach(this::unload);
+            ksm.views.forEach(this::unload);
             clearKeyspaceMetadata(ksm);
         }
 
@@ -468,13 +508,13 @@ public class Schema
         KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
         String snapshotName = Keyspace.getTimestampedSnapshotName(ksName);
 
-        CompactionManager.instance.interruptCompactionFor(ksm.tables, true);
+        CompactionManager.instance.interruptCompactionFor(ksm.tablesAndViews(), true);
 
         Keyspace keyspace = Keyspace.open(ksm.name);
 
         // remove all cfs from the keyspace instance.
         List<UUID> droppedCfs = new ArrayList<>();
-        for (CFMetaData cfm : ksm.tables)
+        for (CFMetaData cfm : ksm.tablesAndViews())
         {
             ColumnFamilyStore cfs = keyspace.getColumnFamilyStore(cfm.cfName);
 
@@ -556,6 +596,67 @@ public class Schema
         CommitLog.instance.forceRecycleAllSegments(Collections.singleton(cfm.cfId));
     }
 
+    public void addView(ViewDefinition view)
+    {
+        assert getCFMetaData(view.ksName, view.viewName) == null;
+
+        update(view.ksName, ks ->
+        {
+            load(view);
+
+            // make sure it's init-ed w/ the old definitions first,
+            // since we're going to call initCf on the new one manually
+            Keyspace.open(view.ksName);
+
+            return ks.withSwapped(ks.views.with(view));
+        });
+
+        Keyspace.open(view.ksName).initCf(view.metadata.cfId, view.viewName, true);
+        Keyspace.open(view.ksName).viewManager.reload();
+        MigrationManager.instance.notifyCreateView(view);
+    }
+
+    public void updateView(String ksName, String viewName)
+    {
+        Optional<ViewDefinition> optView = getKSMetaData(ksName).views.get(viewName);
+        assert optView.isPresent();
+        ViewDefinition view = optView.get();
+        boolean columnsDidChange = view.metadata.reload();
+
+        Keyspace keyspace = Keyspace.open(view.ksName);
+        keyspace.getColumnFamilyStore(view.viewName).reload();
+        Keyspace.open(view.ksName).viewManager.update(view.viewName);
+        MigrationManager.instance.notifyUpdateView(view, columnsDidChange);
+    }
+
+    public void dropView(String ksName, String viewName)
+    {
+        KeyspaceMetadata oldKsm = getKSMetaData(ksName);
+        assert oldKsm != null;
+        ColumnFamilyStore cfs = Keyspace.open(ksName).getColumnFamilyStore(viewName);
+        assert cfs != null;
+
+        // make sure all the indexes are dropped, or else.
+        cfs.indexManager.markAllIndexesRemoved();
+
+        // reinitialize the keyspace.
+        ViewDefinition view = oldKsm.views.get(viewName).get();
+        KeyspaceMetadata newKsm = oldKsm.withSwapped(oldKsm.views.without(viewName));
+
+        unload(view);
+        setKeyspaceMetadata(newKsm);
+
+        CompactionManager.instance.interruptCompactionFor(Collections.singleton(view.metadata), true);
+
+        if (DatabaseDescriptor.isAutoSnapshot())
+            cfs.snapshot(Keyspace.getTimestampedSnapshotName(cfs.name));
+        Keyspace.open(ksName).dropCf(view.metadata.cfId);
+        Keyspace.open(ksName).viewManager.reload();
+        MigrationManager.instance.notifyDropView(view);
+
+        CommitLog.instance.forceRecycleAllSegments(Collections.singleton(view.metadata.cfId));
+    }
+
     public void addType(UserType ut)
     {
         update(ut.keyspace, ks -> ks.withSwapped(ks.types.with(ut)));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/config/ViewDefinition.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/config/ViewDefinition.java b/src/java/org/apache/cassandra/config/ViewDefinition.java
new file mode 100644
index 0000000..39695b9
--- /dev/null
+++ b/src/java/org/apache/cassandra/config/ViewDefinition.java
@@ -0,0 +1,114 @@
+/*
+ * 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.config;
+
+import java.util.Objects;
+import java.util.UUID;
+
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.commons.lang3.builder.ToStringBuilder;
+
+import org.apache.cassandra.cql3.ColumnIdentifier;
+
+public class ViewDefinition
+{
+    public final String ksName;
+    public final String viewName;
+    public final UUID baseTableId;
+    public final boolean includeAllColumns;
+    // The order of partititon columns and clustering columns is important, so we cannot switch these two to sets
+    public final CFMetaData metadata;
+
+    public ViewDefinition(ViewDefinition def)
+    {
+        this(def.ksName, def.viewName, def.baseTableId, def.includeAllColumns, def.metadata);
+    }
+
+    /**
+     * @param viewName          Name of the view
+     * @param baseTableId       Internal ID of the table which this view is based off of
+     * @param includeAllColumns Whether to include all columns or not
+     */
+    public ViewDefinition(String ksName, String viewName, UUID baseTableId, boolean includeAllColumns, CFMetaData metadata)
+    {
+        this.ksName = ksName;
+        this.viewName = viewName;
+        this.baseTableId = baseTableId;
+        this.includeAllColumns = includeAllColumns;
+        this.metadata = metadata;
+    }
+
+    /**
+     * @return true if the view specified by this definition will include the column, false otherwise
+     */
+    public boolean includes(ColumnIdentifier column)
+    {
+        return metadata.getColumnDefinition(column) != null;
+    }
+
+    public ViewDefinition copy()
+    {
+        return new ViewDefinition(ksName, viewName, baseTableId, includeAllColumns, metadata.copy());
+    }
+
+    public CFMetaData baseTableMetadata()
+    {
+        return Schema.instance.getCFMetaData(baseTableId);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        if (this == o)
+            return true;
+
+        if (!(o instanceof ViewDefinition))
+            return false;
+
+        ViewDefinition other = (ViewDefinition) o;
+        return Objects.equals(ksName, other.ksName)
+               && Objects.equals(viewName, other.viewName)
+               && Objects.equals(baseTableId, other.baseTableId)
+               && Objects.equals(includeAllColumns, other.includeAllColumns)
+               && Objects.equals(metadata, other.metadata);
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return new HashCodeBuilder(29, 1597)
+               .append(ksName)
+               .append(viewName)
+               .append(baseTableId)
+               .append(includeAllColumns)
+               .append(metadata)
+               .toHashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        return new ToStringBuilder(this)
+               .append("ksName", ksName)
+               .append("viewName", viewName)
+               .append("baseTableId", baseTableId)
+               .append("includeAllColumns", includeAllColumns)
+               .append("metadata", metadata)
+               .toString();
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/Cql.g
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/Cql.g b/src/java/org/apache/cassandra/cql3/Cql.g
index 87bec4b..f6d54f5 100644
--- a/src/java/org/apache/cassandra/cql3/Cql.g
+++ b/src/java/org/apache/cassandra/cql3/Cql.g
@@ -740,7 +740,7 @@ indexIdent returns [IndexTarget.Raw id]
  *  PRIMARY KEY (<pkColumns>)
  *  WITH <property> = <value> AND ...;
  */
-createMaterializedViewStatement returns [CreateMaterializedViewStatement expr]
+createMaterializedViewStatement returns [CreateViewStatement expr]
     @init {
         boolean ifNotExists = false;
         List<ColumnIdentifier.Raw> partitionKeys = new ArrayList<>();
@@ -753,7 +753,7 @@ createMaterializedViewStatement returns [CreateMaterializedViewStatement expr]
         '(' '(' k1=cident { partitionKeys.add(k1); } ( ',' kn=cident { partitionKeys.add(kn); } )* ')' ( ',' c1=cident { compositeKeys.add(c1); } )* ')'
     |   '(' k1=cident { partitionKeys.add(k1); } ( ',' cn=cident { compositeKeys.add(cn); } )* ')'
         )
-        { $expr = new CreateMaterializedViewStatement(cf, basecf, sclause, wclause, partitionKeys, compositeKeys, ifNotExists); }
+        { $expr = new CreateViewStatement(cf, basecf, sclause, wclause, partitionKeys, compositeKeys, ifNotExists); }
         ( K_WITH cfamProperty[expr.properties] ( K_AND cfamProperty[expr.properties] )*)?
     ;
 
@@ -820,14 +820,14 @@ alterTableStatement returns [AlterTableStatement expr]
     }
     ;
 
-alterMaterializedViewStatement returns [AlterMaterializedViewStatement expr]
+alterMaterializedViewStatement returns [AlterViewStatement expr]
     @init {
         TableAttributes attrs = new TableAttributes();
     }
     : K_ALTER K_MATERIALIZED K_VIEW name=columnFamilyName
           K_WITH properties[attrs]
     {
-        $expr = new AlterMaterializedViewStatement(name, attrs);
+        $expr = new AlterViewStatement(name, attrs);
     }
     ;
     
@@ -886,10 +886,10 @@ dropIndexStatement returns [DropIndexStatement expr]
 /**
  * DROP MATERIALIZED VIEW [IF EXISTS] <view_name>
  */
-dropMaterializedViewStatement returns [DropMaterializedViewStatement expr]
+dropMaterializedViewStatement returns [DropViewStatement expr]
     @init { boolean ifExists = false; }
     : K_DROP K_MATERIALIZED K_VIEW (K_IF K_EXISTS { ifExists = true; } )? cf=columnFamilyName
-      { $expr = new DropMaterializedViewStatement(cf, ifExists); }
+      { $expr = new DropViewStatement(cf, ifExists); }
     ;
 
 /**

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/AlterMaterializedViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterMaterializedViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterMaterializedViewStatement.java
deleted file mode 100644
index bc4ba11..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/AlterMaterializedViewStatement.java
+++ /dev/null
@@ -1,91 +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.cql3.statements;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.db.view.MaterializedView;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.schema.TableParams;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.transport.Event;
-
-import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
-
-public class AlterMaterializedViewStatement extends SchemaAlteringStatement
-{
-    private final TableAttributes attrs;
-
-    public AlterMaterializedViewStatement(CFName name, TableAttributes attrs)
-    {
-        super(name);
-        this.attrs = attrs;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        CFMetaData baseTable = MaterializedView.findBaseTable(keyspace(), columnFamily());
-        if (baseTable != null)
-            state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.ALTER);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in announceMigration()
-    }
-
-    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
-    {
-        CFMetaData meta = validateColumnFamily(keyspace(), columnFamily());
-        if (!meta.isMaterializedView())
-            throw new InvalidRequestException("Cannot use ALTER MATERIALIZED VIEW on Table");
-
-        CFMetaData cfm = meta.copy();
-
-        if (attrs == null)
-            throw new InvalidRequestException("ALTER MATERIALIZED VIEW WITH invoked, but no parameters found");
-
-        attrs.validate();
-
-        TableParams params = attrs.asAlteredTableParams(cfm.params);
-        if (params.gcGraceSeconds == 0)
-        {
-            throw new InvalidRequestException("Cannot alter gc_grace_seconds of a materialized view to 0, since this " +
-                                              "value is used to TTL undelivered updates. Setting gc_grace_seconds too " +
-                                              "low might cause undelivered updates to expire before being replayed.");
-        }
-        cfm.params(params);
-
-        MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
-        return true;
-    }
-
-    public String toString()
-    {
-        return String.format("AlterMaterializedViewStatement(name=%s)", cfName);
-    }
-
-    public Event.SchemaChange changeEvent()
-    {
-        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
index fac0c53..af9a75c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTableStatement.java
@@ -19,6 +19,8 @@ package org.apache.cassandra.cql3.statements;
 
 import java.util.*;
 
+import com.google.common.collect.Iterables;
+
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.*;
 import org.apache.cassandra.cql3.CFName;
@@ -27,6 +29,7 @@ import org.apache.cassandra.cql3.ColumnIdentifier;
 import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.CollectionType;
 import org.apache.cassandra.db.marshal.CounterColumnType;
+import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.schema.IndexMetadata;
 import org.apache.cassandra.schema.Indexes;
@@ -81,7 +84,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
     public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
     {
         CFMetaData meta = validateColumnFamily(keyspace(), columnFamily());
-        if (meta.isMaterializedView())
+        if (meta.isView())
             throw new InvalidRequestException("Cannot use ALTER TABLE on Materialized View");
 
         CFMetaData cfm = meta.copy();
@@ -95,7 +98,8 @@ public class AlterTableStatement extends SchemaAlteringStatement
             def = cfm.getColumnDefinition(columnName);
         }
 
-        List<CFMetaData> materializedViewUpdates = null;
+        List<ViewDefinition> viewUpdates = null;
+        Iterable<ViewDefinition> views = View.findAll(keyspace(), columnFamily());
 
         switch (oType)
         {
@@ -156,19 +160,20 @@ public class AlterTableStatement extends SchemaAlteringStatement
                                         ? ColumnDefinition.staticDef(cfm, columnName.bytes, type)
                                         : ColumnDefinition.regularDef(cfm, columnName.bytes, type));
 
-                // Adding a column to a table which has an include all materialized view requires the column to be added
-                // to the materialized view as well
-                for (MaterializedViewDefinition mv : cfm.getMaterializedViews())
+                // Adding a column to a table which has an include all view requires the column to be added to the view
+                // as well
+                if (!isStatic)
                 {
-                    if (mv.includeAll)
+                    for (ViewDefinition view : views)
                     {
-                        CFMetaData indexCfm = Schema.instance.getCFMetaData(keyspace(), mv.viewName).copy();
-                        indexCfm.addColumnDefinition(isStatic
-                                                     ? ColumnDefinition.staticDef(indexCfm, columnName.bytes, type)
-                                                     : ColumnDefinition.regularDef(indexCfm, columnName.bytes, type));
-                        if (materializedViewUpdates == null)
-                            materializedViewUpdates = new ArrayList<>();
-                        materializedViewUpdates.add(indexCfm);
+                        if (view.includeAllColumns)
+                        {
+                            ViewDefinition viewCopy = view.copy();
+                            viewCopy.metadata.addColumnDefinition(ColumnDefinition.regularDef(viewCopy.metadata, columnName.bytes, type));
+                            if (viewUpdates == null)
+                                viewUpdates = new ArrayList<>();
+                            viewUpdates.add(viewCopy);
+                        }
                     }
                 }
                 break;
@@ -221,17 +226,16 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 // In any case, we update the column definition
                 cfm.addOrReplaceColumnDefinition(def.withNewType(validatorType));
 
-                // We have to alter the schema of the materialized view table as well; it doesn't affect the definition however
-                for (MaterializedViewDefinition mv : cfm.getMaterializedViews())
+                // We have to alter the schema of the view table as well; it doesn't affect the definition however
+                for (ViewDefinition view : views)
                 {
-                    if (!mv.includes(columnName)) continue;
-                    // We have to use the pre-adjusted CFM, otherwise we can't resolve the Index
-                    CFMetaData indexCfm = Schema.instance.getCFMetaData(keyspace(), mv.viewName).copy();
-                    indexCfm.addOrReplaceColumnDefinition(def.withNewType(validatorType));
-
-                    if (materializedViewUpdates == null)
-                        materializedViewUpdates = new ArrayList<>();
-                    materializedViewUpdates.add(indexCfm);
+                    if (!view.includes(columnName)) continue;
+                    ViewDefinition viewCopy = view.copy();
+                    viewCopy.metadata.addOrReplaceColumnDefinition(def.withNewType(validatorType));
+
+                    if (viewUpdates == null)
+                        viewUpdates = new ArrayList<>();
+                    viewUpdates.add(viewCopy);
                 }
                 break;
 
@@ -276,20 +280,16 @@ public class AlterTableStatement extends SchemaAlteringStatement
                 }
                 cfm.indexes(allIndexes);
 
-                // If a column is dropped which is the target of a materialized view,
-                // then we need to drop the view.
-                // If a column is dropped which was selected into a materialized view,
-                // we need to drop that column from the included materialzied view table
-                // and definition.
+                // If a column is dropped which is included in a view, we don't allow the drop to take place.
                 boolean rejectAlter = false;
                 StringBuilder builder = new StringBuilder();
-                for (MaterializedViewDefinition mv : cfm.getMaterializedViews())
+                for (ViewDefinition view : views)
                 {
-                    if (!mv.includes(columnName)) continue;
+                    if (!view.includes(columnName)) continue;
                     if (rejectAlter)
                         builder.append(',');
                     rejectAlter = true;
-                    builder.append(mv.viewName);
+                    builder.append(view.viewName);
                 }
                 if (rejectAlter)
                     throw new InvalidRequestException(String.format("Cannot drop column %s, depended on by materialized views (%s.{%s})",
@@ -304,7 +304,7 @@ public class AlterTableStatement extends SchemaAlteringStatement
 
                 TableParams params = attrs.asAlteredTableParams(cfm.params);
 
-                if (cfm.hasMaterializedViews() && params.gcGraceSeconds == 0)
+                if (!Iterables.isEmpty(views) && params.gcGraceSeconds == 0)
                 {
                     throw new InvalidRequestException("Cannot alter gc_grace_seconds of the base table of a " +
                                                       "materialized view to 0, since this value is used to TTL " +
@@ -326,36 +326,31 @@ public class AlterTableStatement extends SchemaAlteringStatement
                     ColumnIdentifier to = entry.getValue().prepare(cfm);
                     cfm.renameColumn(from, to);
 
-                    // If the materialized view includes a renamed column, it must be renamed in the index table and the definition.
-                    for (MaterializedViewDefinition mv : cfm.getMaterializedViews())
+                    // If the view includes a renamed column, it must be renamed in the view table and the definition.
+                    for (ViewDefinition view : views)
                     {
-                        if (!mv.includes(from)) continue;
-
-                        CFMetaData indexCfm = Schema.instance.getCFMetaData(keyspace(), mv.viewName).copy();
-                        ColumnIdentifier indexFrom = entry.getKey().prepare(indexCfm);
-                        ColumnIdentifier indexTo = entry.getValue().prepare(indexCfm);
-                        indexCfm.renameColumn(indexFrom, indexTo);
+                        if (!view.includes(from)) continue;
 
-                        MaterializedViewDefinition mvCopy = new MaterializedViewDefinition(mv);
-                        mvCopy.renameColumn(from, to);
+                        ViewDefinition viewCopy = view.copy();
+                        ColumnIdentifier viewFrom = entry.getKey().prepare(viewCopy.metadata);
+                        ColumnIdentifier viewTo = entry.getValue().prepare(viewCopy.metadata);
+                        viewCopy.metadata.renameColumn(viewFrom, viewTo);
 
-                        cfm.materializedViews(cfm.getMaterializedViews().replace(mvCopy));
-
-                        if (materializedViewUpdates == null)
-                            materializedViewUpdates = new ArrayList<>();
-                        materializedViewUpdates.add(indexCfm);
+                        if (viewUpdates == null)
+                            viewUpdates = new ArrayList<>();
+                        viewUpdates.add(viewCopy);
                     }
                 }
                 break;
         }
 
-        if (materializedViewUpdates != null)
+        MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
+
+        if (viewUpdates != null)
         {
-            for (CFMetaData mvUpdates : materializedViewUpdates)
-                MigrationManager.announceColumnFamilyUpdate(mvUpdates, false, isLocalOnly);
+            for (ViewDefinition viewUpdate : viewUpdates)
+                MigrationManager.announceViewUpdate(viewUpdate, isLocalOnly);
         }
-
-        MigrationManager.announceColumnFamilyUpdate(cfm, false, isLocalOnly);
         return true;
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
index 4298c22..e176cec 100644
--- a/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterTypeStatement.java
@@ -115,6 +115,16 @@ public abstract class AlterTypeStatement extends SchemaAlteringStatement
                 MigrationManager.announceColumnFamilyUpdate(copy, false, isLocalOnly);
         }
 
+        for (ViewDefinition view : ksm.views)
+        {
+            ViewDefinition copy = view.copy();
+            boolean modified = false;
+            for (ColumnDefinition def : copy.metadata.allColumns())
+                modified |= updateDefinition(copy.metadata, def, toUpdate.keyspace, toUpdate.name, updated);
+            if (modified)
+                MigrationManager.announceViewUpdate(copy, isLocalOnly);
+        }
+
         // Other user types potentially using the updated type
         for (UserType ut : ksm.types)
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
new file mode 100644
index 0000000..e578c4f
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/AlterViewStatement.java
@@ -0,0 +1,94 @@
+/*
+ * 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.statements;
+
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.cql3.CFName;
+import org.apache.cassandra.db.view.View;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.TableParams;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.transport.Event;
+
+import static org.apache.cassandra.thrift.ThriftValidation.validateColumnFamily;
+
+public class AlterViewStatement extends SchemaAlteringStatement
+{
+    private final TableAttributes attrs;
+
+    public AlterViewStatement(CFName name, TableAttributes attrs)
+    {
+        super(name);
+        this.attrs = attrs;
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        CFMetaData baseTable = View.findBaseTable(keyspace(), columnFamily());
+        if (baseTable != null)
+            state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.ALTER);
+    }
+
+    public void validate(ClientState state)
+    {
+        // validated in announceMigration()
+    }
+
+    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
+    {
+        CFMetaData meta = validateColumnFamily(keyspace(), columnFamily());
+        if (!meta.isView())
+            throw new InvalidRequestException("Cannot use ALTER MATERIALIZED VIEW on Table");
+
+        ViewDefinition view = Schema.instance.getView(keyspace(), columnFamily());
+        ViewDefinition viewCopy = view.copy();
+
+        if (attrs == null)
+            throw new InvalidRequestException("ALTER MATERIALIZED VIEW WITH invoked, but no parameters found");
+
+        attrs.validate();
+
+        TableParams params = attrs.asAlteredTableParams(view.metadata.params);
+        if (params.gcGraceSeconds == 0)
+        {
+            throw new InvalidRequestException("Cannot alter gc_grace_seconds of a materialized view to 0, since this " +
+                                              "value is used to TTL undelivered updates. Setting gc_grace_seconds too " +
+                                              "low might cause undelivered updates to expire before being replayed.");
+        }
+        view.metadata.params(params);
+
+        MigrationManager.announceViewUpdate(viewCopy, isLocalOnly);
+        return true;
+    }
+
+    public String toString()
+    {
+        return String.format("AlterViewStatement(name=%s)", cfName);
+    }
+
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(Event.SchemaChange.Change.UPDATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/CreateMaterializedViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateMaterializedViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateMaterializedViewStatement.java
deleted file mode 100644
index dee0577..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/CreateMaterializedViewStatement.java
+++ /dev/null
@@ -1,286 +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.cql3.statements;
-
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.MaterializedViewDefinition;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.selection.RawSelector;
-import org.apache.cassandra.cql3.selection.Selectable;
-import org.apache.cassandra.db.view.MaterializedView;
-import org.apache.cassandra.exceptions.AlreadyExistsException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.ClientWarn;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.thrift.ThriftValidation;
-import org.apache.cassandra.transport.Event;
-
-public class CreateMaterializedViewStatement extends SchemaAlteringStatement
-{
-    private final CFName baseName;
-    private final List<RawSelector> selectClause;
-    private final List<ColumnIdentifier.Raw> notNullWhereClause;
-    private final List<ColumnIdentifier.Raw> partitionKeys;
-    private final List<ColumnIdentifier.Raw> clusteringKeys;
-    public final CFProperties properties = new CFProperties();
-    private final boolean ifNotExists;
-
-    public CreateMaterializedViewStatement(CFName viewName,
-                                           CFName baseName,
-                                           List<RawSelector> selectClause,
-                                           List<ColumnIdentifier.Raw> notNullWhereClause,
-                                           List<ColumnIdentifier.Raw> partitionKeys,
-                                           List<ColumnIdentifier.Raw> clusteringKeys,
-                                           boolean ifNotExists)
-    {
-        super(viewName);
-        this.baseName = baseName;
-        this.selectClause = selectClause;
-        this.notNullWhereClause = notNullWhereClause;
-        this.partitionKeys = partitionKeys;
-        this.clusteringKeys = clusteringKeys;
-        this.ifNotExists = ifNotExists;
-    }
-
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        if (!baseName.hasKeyspace())
-            baseName.setKeyspace(keyspace(), true);
-        state.hasColumnFamilyAccess(keyspace(), baseName.getColumnFamily(), Permission.ALTER);
-    }
-
-    public void validate(ClientState state) throws RequestValidationException
-    {
-        // We do validation in announceMigration to reduce doubling up of work
-    }
-
-    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
-    {
-        // We need to make sure that:
-        //  - primary key includes all columns in base table's primary key
-        //  - make sure that the select statement does not have anything other than columns
-        //    and their names match the base table's names
-        //  - make sure that primary key does not include any collections
-        //  - make sure there is no where clause in the select statement
-        //  - make sure there is not currently a table or view
-        //  - make sure baseTable gcGraceSeconds > 0
-
-        properties.validate();
-
-        if (properties.useCompactStorage)
-            throw new InvalidRequestException("Cannot use 'COMPACT STORAGE' when defining a materialized view");
-
-        // We enforce the keyspace because if the RF is different, the logic to wait for a
-        // specific replica would break
-        if (!baseName.getKeyspace().equals(keyspace()))
-            throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
-
-        CFMetaData cfm = ThriftValidation.validateColumnFamily(baseName.getKeyspace(), baseName.getColumnFamily());
-
-        if (cfm.isCounter())
-            throw new InvalidRequestException("Materialized views are not supported on counter tables");
-
-        if (cfm.isMaterializedView())
-            throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
-
-        if (cfm.params.gcGraceSeconds == 0)
-        {
-            throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
-                                                            "'%s' with gc_grace_seconds of 0, since this value is " +
-                                                            "used to TTL undelivered updates. Setting gc_grace_seconds" +
-                                                            " too low might cause undelivered updates to expire " +
-                                                            "before being replayed.", cfName.getColumnFamily(),
-                                                            baseName.getColumnFamily()));
-        }
-
-        Set<ColumnIdentifier> included = new HashSet<>();
-        for (RawSelector selector : selectClause)
-        {
-            Selectable.Raw selectable = selector.selectable;
-            if (selectable instanceof Selectable.WithFieldSelection.Raw)
-                throw new InvalidRequestException("Cannot select out a part of type when defining a materialized view");
-            if (selectable instanceof Selectable.WithFunction.Raw)
-                throw new InvalidRequestException("Cannot use function when defining a materialized view");
-            if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
-                throw new InvalidRequestException("Cannot use function when defining a materialized view");
-            ColumnIdentifier identifier = (ColumnIdentifier) selectable.prepare(cfm);
-            if (selector.alias != null)
-                throw new InvalidRequestException(String.format("Cannot alias column '%s' as '%s' when defining a materialized view", identifier.toString(), selector.alias.toString()));
-
-            ColumnDefinition cdef = cfm.getColumnDefinition(identifier);
-
-            if (cdef == null)
-                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
-
-            if (cdef.isStatic())
-                ClientWarn.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
-            else
-                included.add(identifier);
-        }
-
-        Set<ColumnIdentifier.Raw> targetPrimaryKeys = new HashSet<>();
-        for (ColumnIdentifier.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
-        {
-            if (!targetPrimaryKeys.add(identifier))
-                throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
-
-            ColumnDefinition cdef = cfm.getColumnDefinition(identifier.prepare(cfm));
-
-            if (cdef == null)
-                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
-
-            if (cfm.getColumnDefinition(identifier.prepare(cfm)).type.isMultiCell())
-                throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
-
-            if (cdef.isStatic())
-                throw new InvalidRequestException(String.format("Cannot use Static column '%s' in PRIMARY KEY of materialized view", identifier));
-        }
-
-        Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
-        for (ColumnDefinition definition : Iterables.concat(cfm.partitionKeyColumns(), cfm.clusteringColumns()))
-            basePrimaryKeyCols.add(definition.name);
-
-        List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
-        List<ColumnIdentifier> targetPartitionKeys = new ArrayList<>();
-        Set<ColumnIdentifier> notNullColumns = new HashSet<>();
-        if (notNullWhereClause != null)
-        {
-            for (ColumnIdentifier.Raw raw : notNullWhereClause)
-            {
-                notNullColumns.add(raw.prepare(cfm));
-            }
-        }
-
-        // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
-        boolean hasNonPKColumn = false;
-        for (ColumnIdentifier.Raw raw : partitionKeys)
-        {
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, notNullColumns);
-        }
-
-        for (ColumnIdentifier.Raw raw : clusteringKeys)
-        {
-            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, notNullColumns);
-        }
-
-        // We need to include all of the primary key colums from the base table in order to make sure that we do not
-        // overwrite values in the materialized view. We cannot support "collapsing" the base table into a smaller
-        // number of rows in the view because if we need to generate a tombstone, we have no way of knowing which value
-        // is currently being used in the view and whether or not to generate a tombstone.
-        // In order to not surprise our users, we require that they include all of the columns. We provide them with
-        // a list of all of the columns left to include.
-        boolean missingClusteringColumns = false;
-        StringBuilder columnNames = new StringBuilder();
-        for (ColumnDefinition def : cfm.allColumns())
-        {
-            if (!def.isPrimaryKeyColumn()) continue;
-
-            ColumnIdentifier identifier = def.name;
-            if (!targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier))
-            {
-                if (missingClusteringColumns)
-                    columnNames.append(',');
-                else
-                    missingClusteringColumns = true;
-                columnNames.append(identifier);
-            }
-        }
-        if (missingClusteringColumns)
-            throw new InvalidRequestException(String.format("Cannot create Materialized View %s without primary key columns from base %s (%s)",
-                                                            columnFamily(), baseName.getColumnFamily(), columnNames.toString()));
-
-        if (targetPartitionKeys.isEmpty())
-            throw new InvalidRequestException("Must select at least a column for a Materialized View");
-
-        if (targetClusteringColumns.isEmpty())
-            throw new InvalidRequestException("No columns are defined for Materialized View other than primary key");
-
-        MaterializedViewDefinition definition = new MaterializedViewDefinition(baseName.getColumnFamily(),
-                                                                               columnFamily(),
-                                                                               targetPartitionKeys,
-                                                                               targetClusteringColumns,
-                                                                               included);
-
-        CFMetaData indexCf = MaterializedView.getCFMetaData(definition, cfm, properties);
-        try
-        {
-            MigrationManager.announceNewColumnFamily(indexCf, isLocalOnly);
-        }
-        catch (AlreadyExistsException e)
-        {
-            if (ifNotExists)
-                return false;
-            throw e;
-        }
-
-        CFMetaData newCfm = cfm.copy();
-        newCfm.materializedViews(newCfm.getMaterializedViews().with(definition));
-
-        MigrationManager.announceColumnFamilyUpdate(newCfm, false, isLocalOnly);
-
-        return true;
-    }
-
-    private static boolean getColumnIdentifier(CFMetaData cfm,
-                                               Set<ColumnIdentifier> basePK,
-                                               boolean hasNonPKColumn,
-                                               ColumnIdentifier.Raw raw,
-                                               List<ColumnIdentifier> columns,
-                                               Set<ColumnIdentifier> allowedPKColumns)
-    {
-        ColumnIdentifier identifier = raw.prepare(cfm);
-
-        boolean isPk = basePK.contains(identifier);
-        if (!isPk && hasNonPKColumn)
-        {
-            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", identifier));
-        }
-
-        // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
-        // because we will never allow a single partition key to be NULL
-        boolean isSinglePartitionKey = cfm.getColumnDefinition(identifier).isPartitionKey()
-                                       && cfm.partitionKeyColumns().size() == 1;
-        if (!allowedPKColumns.remove(identifier) && !isSinglePartitionKey)
-        {
-            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", identifier));
-        }
-
-        columns.add(identifier);
-        return !isPk;
-    }
-
-    public Event.SchemaChange changeEvent()
-    {
-        return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
index 706cfea..2589622 100644
--- a/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateTriggerStatement.java
@@ -59,7 +59,7 @@ public class CreateTriggerStatement extends SchemaAlteringStatement
     public void validate(ClientState state) throws RequestValidationException
     {
         CFMetaData cfm = ThriftValidation.validateColumnFamily(keyspace(), columnFamily());
-        if (cfm.isMaterializedView())
+        if (cfm.isView())
             throw new InvalidRequestException("Cannot CREATE TRIGGER against a materialized view");
 
         try


[3/7] cassandra git commit: Improve MV schema representation

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/schema/MaterializedViews.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/MaterializedViews.java b/src/java/org/apache/cassandra/schema/MaterializedViews.java
deleted file mode 100644
index 1c55736..0000000
--- a/src/java/org/apache/cassandra/schema/MaterializedViews.java
+++ /dev/null
@@ -1,149 +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.schema;
-
-
-import java.util.Iterator;
-import java.util.Optional;
-
-import com.google.common.collect.ImmutableMap;
-
-import org.apache.cassandra.config.MaterializedViewDefinition;
-
-import static com.google.common.collect.Iterables.filter;
-
-public final class MaterializedViews implements Iterable<MaterializedViewDefinition>
-{
-    private final ImmutableMap<String, MaterializedViewDefinition> materializedViews;
-
-    private MaterializedViews(Builder builder)
-    {
-        materializedViews = builder.materializedViews.build();
-    }
-
-    public static Builder builder()
-    {
-        return new Builder();
-    }
-
-    public static MaterializedViews none()
-    {
-        return builder().build();
-    }
-
-    public Iterator<MaterializedViewDefinition> iterator()
-    {
-        return materializedViews.values().iterator();
-    }
-
-    public int size()
-    {
-        return materializedViews.size();
-    }
-
-    public boolean isEmpty()
-    {
-        return materializedViews.isEmpty();
-    }
-
-    /**
-     * Get the materialized view with the specified name
-     *
-     * @param name a non-qualified materialized view name
-     * @return an empty {@link Optional} if the materialized view name is not found; a non-empty optional of {@link MaterializedViewDefinition} otherwise
-     */
-    public Optional<MaterializedViewDefinition> get(String name)
-    {
-        return Optional.ofNullable(materializedViews.get(name));
-    }
-
-    /**
-     * Create a MaterializedViews instance with the provided materialized view added
-     */
-    public MaterializedViews with(MaterializedViewDefinition materializedView)
-    {
-        if (get(materializedView.viewName).isPresent())
-            throw new IllegalStateException(String.format("Materialized View %s already exists", materializedView.viewName));
-
-        return builder().add(this).add(materializedView).build();
-    }
-
-    /**
-     * Creates a MaterializedViews instance with the materializedView with the provided name removed
-     */
-    public MaterializedViews without(String name)
-    {
-        MaterializedViewDefinition materializedView =
-        get(name).orElseThrow(() -> new IllegalStateException(String.format("Materialized View %s doesn't exists", name)));
-
-        return builder().add(filter(this, v -> v != materializedView)).build();
-    }
-
-    /**
-     * Creates a MaterializedViews instance which contains an updated materialized view
-     */
-    public MaterializedViews replace(MaterializedViewDefinition materializedView)
-    {
-        return without(materializedView.viewName).with(materializedView);
-    }
-
-    @Override
-    public boolean equals(Object o)
-    {
-        return this == o || (o instanceof MaterializedViews && materializedViews.equals(((MaterializedViews) o).materializedViews));
-    }
-
-    @Override
-    public int hashCode()
-    {
-        return materializedViews.hashCode();
-    }
-
-    @Override
-    public String toString()
-    {
-        return materializedViews.values().toString();
-    }
-
-    public static final class Builder
-    {
-        final ImmutableMap.Builder<String, MaterializedViewDefinition> materializedViews = new ImmutableMap.Builder<>();
-
-        private Builder()
-        {
-        }
-
-        public MaterializedViews build()
-        {
-            return new MaterializedViews(this);
-        }
-
-        public Builder add(MaterializedViewDefinition materializedView)
-        {
-            materializedViews.put(materializedView.viewName, materializedView);
-            return this;
-        }
-
-        public Builder add(Iterable<MaterializedViewDefinition> materializedViews)
-        {
-            materializedViews.forEach(this::add);
-            return this;
-        }
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
index 55b841b..c922612 100644
--- a/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
+++ b/src/java/org/apache/cassandra/schema/SchemaKeyspace.java
@@ -68,14 +68,14 @@ public final class SchemaKeyspace
     public static final String COLUMNS = "columns";
     public static final String DROPPED_COLUMNS = "dropped_columns";
     public static final String TRIGGERS = "triggers";
-    public static final String MATERIALIZED_VIEWS = "materialized_views";
+    public static final String VIEWS = "views";
     public static final String TYPES = "types";
     public static final String FUNCTIONS = "functions";
     public static final String AGGREGATES = "aggregates";
     public static final String INDEXES = "indexes";
 
     public static final List<String> ALL =
-        ImmutableList.of(KEYSPACES, TABLES, COLUMNS, TRIGGERS, MATERIALIZED_VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
+        ImmutableList.of(KEYSPACES, TABLES, COLUMNS, TRIGGERS, VIEWS, TYPES, FUNCTIONS, AGGREGATES, INDEXES);
 
     private static final CFMetaData Keyspaces =
         compile(KEYSPACES,
@@ -144,17 +144,31 @@ public final class SchemaKeyspace
                 + "options frozen<map<text, text>>,"
                 + "PRIMARY KEY ((keyspace_name), table_name, trigger_name))");
 
-    private static final CFMetaData MaterializedViews =
-        compile(MATERIALIZED_VIEWS,
-                "materialized views definitions",
+    private static final CFMetaData Views =
+        compile(VIEWS,
+                "view definitions",
                 "CREATE TABLE %s ("
                 + "keyspace_name text,"
-                + "table_name text,"
                 + "view_name text,"
-                + "target_columns frozen<list<text>>,"
-                + "clustering_columns frozen<list<text>>,"
-                + "included_columns frozen<list<text>>,"
-                + "PRIMARY KEY ((keyspace_name), table_name, view_name))");
+                + "base_table_id uuid,"
+                + "base_table_name text,"
+                + "bloom_filter_fp_chance double,"
+                + "caching frozen<map<text, text>>,"
+                + "comment text,"
+                + "compaction frozen<map<text, text>>,"
+                + "compression frozen<map<text, text>>,"
+                + "dclocal_read_repair_chance double,"
+                + "default_time_to_live int,"
+                + "extensions frozen<map<text, blob>>,"
+                + "gc_grace_seconds int,"
+                + "id uuid,"
+                + "include_all_columns boolean,"
+                + "max_index_interval int,"
+                + "memtable_flush_period_in_ms int,"
+                + "min_index_interval int,"
+                + "read_repair_chance double,"
+                + "speculative_retry text,"
+                + "PRIMARY KEY ((keyspace_name), view_name))");
 
     private static final CFMetaData Indexes =
         compile(INDEXES,
@@ -210,7 +224,7 @@ public final class SchemaKeyspace
                 + "PRIMARY KEY ((keyspace_name), aggregate_name, signature))");
 
     public static final List<CFMetaData> ALL_TABLE_METADATA =
-        ImmutableList.of(Keyspaces, Tables, Columns, Triggers, DroppedColumns, MaterializedViews, Types, Functions, Aggregates, Indexes);
+        ImmutableList.of(Keyspaces, Tables, Columns, Triggers, DroppedColumns, Views, Types, Functions, Aggregates, Indexes);
 
     private static CFMetaData compile(String name, String description, String schema)
     {
@@ -265,9 +279,10 @@ public final class SchemaKeyspace
 
                     readSchemaPartitionForKeyspaceAndApply(TYPES, key,
                         types -> readSchemaPartitionForKeyspaceAndApply(TABLES, key,
-                        tables -> readSchemaPartitionForKeyspaceAndApply(FUNCTIONS, key,
+                        tables -> readSchemaPartitionForKeyspaceAndApply(VIEWS, key,
+                        views -> readSchemaPartitionForKeyspaceAndApply(FUNCTIONS, key,
                         functions -> readSchemaPartitionForKeyspaceAndApply(AGGREGATES, key,
-                        aggregates -> keyspaces.add(createKeyspaceFromSchemaPartitions(partition, tables, types, functions, aggregates)))))
+                        aggregates -> keyspaces.add(createKeyspaceFromSchemaPartitions(partition, tables, views, types, functions, aggregates))))))
                     );
                 }
             }
@@ -473,6 +488,7 @@ public final class SchemaKeyspace
         // current state of the schema
         Map<DecoratedKey, FilteredPartition> oldKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
         Map<DecoratedKey, FilteredPartition> oldColumnFamilies = readSchemaForKeyspaces(TABLES, keyspaces);
+        Map<DecoratedKey, FilteredPartition> oldViews = readSchemaForKeyspaces(VIEWS, keyspaces);
         Map<DecoratedKey, FilteredPartition> oldTypes = readSchemaForKeyspaces(TYPES, keyspaces);
         Map<DecoratedKey, FilteredPartition> oldFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
         Map<DecoratedKey, FilteredPartition> oldAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
@@ -485,12 +501,14 @@ public final class SchemaKeyspace
         // with new data applied
         Map<DecoratedKey, FilteredPartition> newKeyspaces = readSchemaForKeyspaces(KEYSPACES, keyspaces);
         Map<DecoratedKey, FilteredPartition> newColumnFamilies = readSchemaForKeyspaces(TABLES, keyspaces);
+        Map<DecoratedKey, FilteredPartition> newViews = readSchemaForKeyspaces(VIEWS, keyspaces);
         Map<DecoratedKey, FilteredPartition> newTypes = readSchemaForKeyspaces(TYPES, keyspaces);
         Map<DecoratedKey, FilteredPartition> newFunctions = readSchemaForKeyspaces(FUNCTIONS, keyspaces);
         Map<DecoratedKey, FilteredPartition> newAggregates = readSchemaForKeyspaces(AGGREGATES, keyspaces);
 
         Set<String> keyspacesToDrop = mergeKeyspaces(oldKeyspaces, newKeyspaces);
         mergeTables(oldColumnFamilies, newColumnFamilies);
+        mergeViews(oldViews, newViews);
         mergeTypes(oldTypes, newTypes);
         mergeFunctions(oldFunctions, newFunctions);
         mergeAggregates(oldAggregates, newAggregates);
@@ -546,6 +564,27 @@ public final class SchemaKeyspace
         });
     }
 
+    private static void mergeViews(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
+    {
+        diffSchema(before, after, new Differ()
+        {
+            public void onDropped(UntypedResultSet.Row oldRow)
+            {
+                Schema.instance.dropView(oldRow.getString("keyspace_name"), oldRow.getString("view_name"));
+            }
+
+            public void onAdded(UntypedResultSet.Row newRow)
+            {
+                Schema.instance.addView(createViewFromViewRow(newRow));
+            }
+
+            public void onUpdated(UntypedResultSet.Row oldRow, UntypedResultSet.Row newRow)
+            {
+                Schema.instance.updateView(newRow.getString("keyspace_name"), newRow.getString("view_name"));
+            }
+        });
+    }
+
     private static void mergeTypes(Map<DecoratedKey, FilteredPartition> before, Map<DecoratedKey, FilteredPartition> after)
     {
         diffSchema(before, after, new Differ()
@@ -697,6 +736,7 @@ public final class SchemaKeyspace
         Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
 
         keyspace.tables.forEach(table -> addTableToSchemaMutation(table, timestamp, true, mutation));
+        keyspace.views.forEach(view -> addViewToSchemaMutation(view, timestamp, true, mutation));
         keyspace.types.forEach(type -> addTypeToSchemaMutation(type, timestamp, mutation));
         keyspace.functions.udfs().forEach(udf -> addFunctionToSchemaMutation(udf, timestamp, mutation));
         keyspace.functions.udas().forEach(uda -> addAggregateToSchemaMutation(uda, timestamp, mutation));
@@ -717,6 +757,7 @@ public final class SchemaKeyspace
 
     private static KeyspaceMetadata createKeyspaceFromSchemaPartitions(RowIterator serializedParams,
                                                                        RowIterator serializedTables,
+                                                                       RowIterator serializedViews,
                                                                        RowIterator serializedTypes,
                                                                        RowIterator serializedFunctions,
                                                                        RowIterator serializedAggregates)
@@ -725,13 +766,14 @@ public final class SchemaKeyspace
 
         KeyspaceParams params = createKeyspaceParamsFromSchemaPartition(serializedParams);
         Tables tables = createTablesFromTablesPartition(serializedTables);
+        Views views = createViewsFromViewsPartition(serializedViews);
         Types types = createTypesFromPartition(serializedTypes);
 
         Collection<UDFunction> udfs = createFunctionsFromFunctionsPartition(serializedFunctions);
         Collection<UDAggregate> udas = createAggregatesFromAggregatesPartition(serializedAggregates);
         Functions functions = org.apache.cassandra.schema.Functions.builder().add(udfs).add(udas).build();
 
-        return KeyspaceMetadata.create(name, params, tables, types, functions);
+        return KeyspaceMetadata.create(name, params, tables, views, types, functions);
     }
 
     /**
@@ -849,9 +891,6 @@ public final class SchemaKeyspace
             for (TriggerMetadata trigger : table.getTriggers())
                 addTriggerToSchemaMutation(table, trigger, timestamp, mutation);
 
-            for (MaterializedViewDefinition materializedView: table.getMaterializedViews())
-                addMaterializedViewToSchemaMutation(table, materializedView, timestamp, mutation);
-
             for (IndexMetadata index : table.getIndexes())
                 addIndexToSchemaMutation(table, index, timestamp, mutation);
         }
@@ -931,22 +970,6 @@ public final class SchemaKeyspace
         for (TriggerMetadata trigger : triggerDiff.entriesOnlyOnRight().values())
             addTriggerToSchemaMutation(newTable, trigger, timestamp, mutation);
 
-        MapDifference<String, MaterializedViewDefinition> materializedViewDiff = materializedViewsDiff(oldTable.getMaterializedViews(), newTable.getMaterializedViews());
-
-        // dropped materialized views
-        for (MaterializedViewDefinition materializedView : materializedViewDiff.entriesOnlyOnLeft().values())
-            dropMaterializedViewFromSchemaMutation(oldTable, materializedView, timestamp, mutation);
-
-        // newly created materialized views
-        for (MaterializedViewDefinition materializedView : materializedViewDiff.entriesOnlyOnRight().values())
-            addMaterializedViewToSchemaMutation(newTable, materializedView, timestamp, mutation);
-
-        // updated materialized views need to be updated
-        for (MapDifference.ValueDifference<MaterializedViewDefinition> diff : materializedViewDiff.entriesDiffering().values())
-        {
-            addUpdatedMaterializedViewDefinitionToSchemaMutation(newTable, diff.rightValue(), timestamp, mutation);
-        }
-
         MapDifference<String, IndexMetadata> indexesDiff = indexesDiff(oldTable.getIndexes(),
                                                                        newTable.getIndexes());
 
@@ -989,17 +1012,6 @@ public final class SchemaKeyspace
         return Maps.difference(beforeMap, afterMap);
     }
 
-    private static MapDifference<String, MaterializedViewDefinition> materializedViewsDiff(MaterializedViews before, MaterializedViews after)
-    {
-        Map<String, MaterializedViewDefinition> beforeMap = new HashMap<>();
-        before.forEach(v -> beforeMap.put(v.viewName, v));
-
-        Map<String, MaterializedViewDefinition> afterMap = new HashMap<>();
-        after.forEach(v -> afterMap.put(v.viewName, v));
-
-        return Maps.difference(beforeMap, afterMap);
-    }
-
     public static Mutation makeDropTableMutation(KeyspaceMetadata keyspace, CFMetaData table, long timestamp)
     {
         // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
@@ -1013,9 +1025,6 @@ public final class SchemaKeyspace
         for (TriggerMetadata trigger : table.getTriggers())
             dropTriggerFromSchemaMutation(table, trigger, timestamp, mutation);
 
-        for (MaterializedViewDefinition materializedView : table.getMaterializedViews())
-            dropMaterializedViewFromSchemaMutation(table, materializedView, timestamp, mutation);
-
         for (IndexMetadata index : table.getIndexes())
             dropIndexFromSchemaMutation(table, index, timestamp, mutation);
 
@@ -1083,12 +1092,8 @@ public final class SchemaKeyspace
         Triggers triggers =
             readSchemaPartitionForTableAndApply(TRIGGERS, keyspace, table, SchemaKeyspace::createTriggersFromTriggersPartition);
 
-        MaterializedViews views =
-            readSchemaPartitionForTableAndApply(MATERIALIZED_VIEWS, keyspace, table, SchemaKeyspace::createMaterializedViewsFromMaterializedViewsPartition);
-
         CFMetaData cfm = createTableFromTableRowAndColumns(row, columns).droppedColumns(droppedColumns)
-                                                                        .triggers(triggers)
-                                                                        .materializedViews(views);
+                                                                        .triggers(triggers);
 
         // the CFMetaData itself is required to build the collection of indexes as
         // the column definitions are needed because we store only the name each
@@ -1114,7 +1119,6 @@ public final class SchemaKeyspace
         boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER);
         boolean isDense = flags.contains(CFMetaData.Flag.DENSE);
         boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND);
-        boolean isMaterializedView = flags.contains(CFMetaData.Flag.VIEW);
 
         return CFMetaData.create(keyspace,
                                  table,
@@ -1123,7 +1127,7 @@ public final class SchemaKeyspace
                                  isCompound,
                                  isSuper,
                                  isCounter,
-                                 isMaterializedView,
+                                 false,
                                  columns,
                                  DatabaseDescriptor.getPartitioner())
                          .params(createTableParamsFromRow(row));
@@ -1274,82 +1278,164 @@ public final class SchemaKeyspace
     }
 
     /*
-     * Materialized View metadata serialization/deserialization.
+     * View metadata serialization/deserialization.
      */
 
-    private static void addMaterializedViewToSchemaMutation(CFMetaData table, MaterializedViewDefinition materializedView, long timestamp, Mutation mutation)
+    public static Mutation makeCreateViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
+    {
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+        addViewToSchemaMutation(view, timestamp, true, mutation);
+        return mutation;
+    }
+
+    private static void addViewToSchemaMutation(ViewDefinition view, long timestamp, boolean includeColumns, Mutation mutation)
     {
-        RowUpdateBuilder builder = new RowUpdateBuilder(MaterializedViews, timestamp, mutation)
-            .clustering(table.cfName, materializedView.viewName);
+        RowUpdateBuilder builder = new RowUpdateBuilder(Views, timestamp, mutation)
+            .clustering(view.viewName);
+
+        CFMetaData table = view.metadata;
 
-        builder.frozenList("target_columns", materializedView.partitionColumns.stream().map(ColumnIdentifier::toString).collect(Collectors.toList()));
-        builder.frozenList("clustering_columns", materializedView.clusteringColumns.stream().map(ColumnIdentifier::toString).collect(Collectors.toList()));
-        builder.frozenList("included_columns", materializedView.included.stream().map(ColumnIdentifier::toString).collect(Collectors.toList()));
+        builder.add("include_all_columns", view.includeAllColumns)
+               .add("base_table_id", view.baseTableId)
+               .add("base_table_name", view.baseTableMetadata().cfName)
+               .add("id", table.cfId);
+
+        addTableParamsToSchemaMutation(table.params, builder);
+
+        if (includeColumns)
+        {
+            for (ColumnDefinition column : table.allColumns())
+                addColumnToSchemaMutation(table, column, timestamp, mutation);
+
+            for (CFMetaData.DroppedColumn column : table.getDroppedColumns().values())
+                addDroppedColumnToSchemaMutation(table, column, timestamp, mutation);
+        }
 
         builder.build();
     }
 
-    private static void dropMaterializedViewFromSchemaMutation(CFMetaData table, MaterializedViewDefinition materializedView, long timestamp, Mutation mutation)
+    public static Mutation makeDropViewMutation(KeyspaceMetadata keyspace, ViewDefinition view, long timestamp)
     {
-        RowUpdateBuilder.deleteRow(MaterializedViews, timestamp, mutation, table.cfName, materializedView.viewName);
+        // Include the serialized keyspace in case the target node missed a CREATE KEYSPACE migration (see CASSANDRA-5631).
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+
+        RowUpdateBuilder.deleteRow(Views, timestamp, mutation, view.viewName);
+
+        CFMetaData table = view.metadata;
+        for (ColumnDefinition column : table.allColumns())
+            dropColumnFromSchemaMutation(table, column, timestamp, mutation);
+
+        for (IndexMetadata index : table.getIndexes())
+            dropIndexFromSchemaMutation(table, index, timestamp, mutation);
+
+        return mutation;
     }
 
-    private static void addUpdatedMaterializedViewDefinitionToSchemaMutation(CFMetaData table, MaterializedViewDefinition materializedView, long timestamp, Mutation mutation)
+    public static Mutation makeUpdateViewMutation(KeyspaceMetadata keyspace,
+                                                  ViewDefinition oldView,
+                                                  ViewDefinition newView,
+                                                  long timestamp)
     {
-        addMaterializedViewToSchemaMutation(table, materializedView, timestamp, mutation);
+        Mutation mutation = makeCreateKeyspaceMutation(keyspace.name, keyspace.params, timestamp);
+
+        addViewToSchemaMutation(newView, timestamp, false, mutation);
+
+        MapDifference<ByteBuffer, ColumnDefinition> columnDiff = Maps.difference(oldView.metadata.getColumnMetadata(),
+                                                                                 newView.metadata.getColumnMetadata());
+
+        // columns that are no longer needed
+        for (ColumnDefinition column : columnDiff.entriesOnlyOnLeft().values())
+        {
+            dropColumnFromSchemaMutation(oldView.metadata, column, timestamp, mutation);
+        }
+
+        // newly added columns
+        for (ColumnDefinition column : columnDiff.entriesOnlyOnRight().values())
+            addColumnToSchemaMutation(newView.metadata, column, timestamp, mutation);
+
+        // old columns with updated attributes
+        for (ByteBuffer name : columnDiff.entriesDiffering().keySet())
+            addColumnToSchemaMutation(newView.metadata, newView.metadata.getColumnDefinition(name), timestamp, mutation);
+
+        // dropped columns
+        MapDifference<ByteBuffer, CFMetaData.DroppedColumn> droppedColumnDiff =
+        Maps.difference(oldView.metadata.getDroppedColumns(), oldView.metadata.getDroppedColumns());
+
+        // newly dropped columns
+        for (CFMetaData.DroppedColumn column : droppedColumnDiff.entriesOnlyOnRight().values())
+            addDroppedColumnToSchemaMutation(oldView.metadata, column, timestamp, mutation);
+
+        // columns added then dropped again
+        for (ByteBuffer name : droppedColumnDiff.entriesDiffering().keySet())
+            addDroppedColumnToSchemaMutation(newView.metadata, newView.metadata.getDroppedColumns().get(name), timestamp, mutation);
+
+        return mutation;
+    }
+
+    public static ViewDefinition createViewFromName(String keyspace, String view)
+    {
+        return readSchemaPartitionForTableAndApply(VIEWS, keyspace, view, partition ->
+        {
+            if (partition.isEmpty())
+                throw new RuntimeException(String.format("%s:%s not found in the schema definitions keyspace.", keyspace, view));
+
+            return createViewFromViewPartition(partition);
+        });
+    }
+
+    private static ViewDefinition createViewFromViewPartition(RowIterator partition)
+    {
+        String query = String.format("SELECT * FROM %s.%s", NAME, VIEWS);
+        return createViewFromViewRow(QueryProcessor.resultify(query, partition).one());
     }
 
     /**
-     * Deserialize materialized views from storage-level representation.
+     * Deserialize views from storage-level representation.
      *
-     * @param partition storage-level partition containing the materialized view definitions
-     * @return the list of processed MaterializedViewDefinitions
+     * @param partition storage-level partition containing the view definitions
+     * @return the list of processed ViewDefinitions
      */
-    private static MaterializedViews createMaterializedViewsFromMaterializedViewsPartition(RowIterator partition)
+    private static Views createViewsFromViewsPartition(RowIterator partition)
     {
-        MaterializedViews.Builder views = org.apache.cassandra.schema.MaterializedViews.builder();
-        String query = String.format("SELECT * FROM %s.%s", NAME, MATERIALIZED_VIEWS);
+        Views.Builder views = org.apache.cassandra.schema.Views.builder();
+        String query = String.format("SELECT * FROM %s.%s", NAME, VIEWS);
         for (UntypedResultSet.Row row : QueryProcessor.resultify(query, partition))
         {
-            MaterializedViewDefinition mv = createMaterializedViewFromMaterializedViewRow(row);
-            views.add(mv);
+            ViewDefinition view = createViewFromViewRow(row);
+            views.add(view);
         }
         return views.build();
     }
 
-    private static MaterializedViewDefinition createMaterializedViewFromMaterializedViewRow(UntypedResultSet.Row row)
+    private static ViewDefinition createViewFromViewRow(UntypedResultSet.Row row)
     {
-        String name = row.getString("view_name");
-        List<String> partitionColumnNames = row.getFrozenList("target_columns", UTF8Type.instance);
-
-        String cfName = row.getString("table_name");
-        List<String> clusteringColumnNames = row.getFrozenList("clustering_columns", UTF8Type.instance);
-
-        List<ColumnIdentifier> partitionColumns = new ArrayList<>();
-        for (String columnName : partitionColumnNames)
-        {
-            partitionColumns.add(ColumnIdentifier.getInterned(columnName, true));
-        }
-
-        List<ColumnIdentifier> clusteringColumns = new ArrayList<>();
-        for (String columnName : clusteringColumnNames)
-        {
-            clusteringColumns.add(ColumnIdentifier.getInterned(columnName, true));
-        }
+        String keyspace = row.getString("keyspace_name");
+        String view = row.getString("view_name");
+        UUID id = row.getUUID("id");
+        UUID baseTableId = row.getUUID("base_table_id");
+        boolean includeAll = row.getBoolean("include_all_columns");
 
-        List<String> includedColumnNames = row.getFrozenList("included_columns", UTF8Type.instance);
-        Set<ColumnIdentifier> includedColumns = new HashSet<>();
-        if (includedColumnNames != null)
-        {
-            for (String columnName : includedColumnNames)
-                includedColumns.add(ColumnIdentifier.getInterned(columnName, true));
-        }
+        List<ColumnDefinition> columns =
+            readSchemaPartitionForTableAndApply(COLUMNS, keyspace, view, SchemaKeyspace::createColumnsFromColumnsPartition);
 
-        return new MaterializedViewDefinition(cfName,
-                                              name,
-                                              partitionColumns,
-                                              clusteringColumns,
-                                              includedColumns);
+        Map<ByteBuffer, CFMetaData.DroppedColumn> droppedColumns =
+            readSchemaPartitionForTableAndApply(DROPPED_COLUMNS, keyspace, view, SchemaKeyspace::createDroppedColumnsFromDroppedColumnsPartition);
+
+        CFMetaData cfm = CFMetaData.create(keyspace,
+                                           view,
+                                           id,
+                                           false,
+                                           true,
+                                           false,
+                                           false,
+                                           true,
+                                           columns,
+                                           DatabaseDescriptor.getPartitioner())
+                                   .params(createTableParamsFromRow(row))
+                                   .droppedColumns(droppedColumns);
+
+        return new ViewDefinition(keyspace, view, baseTableId, includeAll, cfm);
     }
 
     /*

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/schema/Views.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/schema/Views.java b/src/java/org/apache/cassandra/schema/Views.java
new file mode 100644
index 0000000..5888b9d
--- /dev/null
+++ b/src/java/org/apache/cassandra/schema/Views.java
@@ -0,0 +1,171 @@
+/*
+ * 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.schema;
+
+
+import java.util.Iterator;
+import java.util.Optional;
+
+import javax.annotation.Nullable;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ViewDefinition;
+
+import static com.google.common.collect.Iterables.filter;
+
+public final class Views implements Iterable<ViewDefinition>
+{
+    private final ImmutableMap<String, ViewDefinition> views;
+
+    private Views(Builder builder)
+    {
+        views = builder.views.build();
+    }
+
+    public static Builder builder()
+    {
+        return new Builder();
+    }
+
+    public static Views none()
+    {
+        return builder().build();
+    }
+
+    public Iterator<ViewDefinition> iterator()
+    {
+        return views.values().iterator();
+    }
+
+    public Iterable<CFMetaData> metadatas()
+    {
+        return Iterables.transform(views.values(), view -> view.metadata);
+    }
+
+    public int size()
+    {
+        return views.size();
+    }
+
+    public boolean isEmpty()
+    {
+        return views.isEmpty();
+    }
+
+    /**
+     * Get the materialized view with the specified name
+     *
+     * @param name a non-qualified materialized view name
+     * @return an empty {@link Optional} if the materialized view name is not found; a non-empty optional of {@link ViewDefinition} otherwise
+     */
+    public Optional<ViewDefinition> get(String name)
+    {
+        return Optional.ofNullable(views.get(name));
+    }
+
+    /**
+     * Get the view with the specified name
+     *
+     * @param name a non-qualified view name
+     * @return null if the view name is not found; the found {@link ViewDefinition} otherwise
+     */
+    @Nullable
+    public ViewDefinition getNullable(String name)
+    {
+        return views.get(name);
+    }
+
+    /**
+     * Create a MaterializedViews instance with the provided materialized view added
+     */
+    public Views with(ViewDefinition view)
+    {
+        if (get(view.viewName).isPresent())
+            throw new IllegalStateException(String.format("Materialized View %s already exists", view.viewName));
+
+        return builder().add(this).add(view).build();
+    }
+
+    /**
+     * Creates a MaterializedViews instance with the materializedView with the provided name removed
+     */
+    public Views without(String name)
+    {
+        ViewDefinition materializedView =
+            get(name).orElseThrow(() -> new IllegalStateException(String.format("Materialized View %s doesn't exists", name)));
+
+        return builder().add(filter(this, v -> v != materializedView)).build();
+    }
+
+    /**
+     * Creates a MaterializedViews instance which contains an updated materialized view
+     */
+    public Views replace(ViewDefinition view, CFMetaData cfm)
+    {
+        return without(view.viewName).with(view);
+    }
+
+    @Override
+    public boolean equals(Object o)
+    {
+        return this == o || (o instanceof Views && views.equals(((Views) o).views));
+    }
+
+    @Override
+    public int hashCode()
+    {
+        return views.hashCode();
+    }
+
+    @Override
+    public String toString()
+    {
+        return views.values().toString();
+    }
+
+    public static final class Builder
+    {
+        final ImmutableMap.Builder<String, ViewDefinition> views = new ImmutableMap.Builder<>();
+
+        private Builder()
+        {
+        }
+
+        public Views build()
+        {
+            return new Views(this);
+        }
+
+
+        public Builder add(ViewDefinition view)
+        {
+            views.put(view.viewName, view);
+            return this;
+        }
+
+        public Builder add(Iterable<ViewDefinition> views)
+        {
+            views.forEach(this::add);
+            return this;
+        }
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/service/CassandraDaemon.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/CassandraDaemon.java b/src/java/org/apache/cassandra/service/CassandraDaemon.java
index 230b46a..1408a70 100644
--- a/src/java/org/apache/cassandra/service/CassandraDaemon.java
+++ b/src/java/org/apache/cassandra/service/CassandraDaemon.java
@@ -230,7 +230,7 @@ public class CassandraDaemon
             if (keyspaceName.equals(SystemKeyspace.NAME))
                 continue;
 
-            for (CFMetaData cfm : Schema.instance.getTables(keyspaceName))
+            for (CFMetaData cfm : Schema.instance.getTablesAndViews(keyspaceName))
                 ColumnFamilyStore.scrubDataDirectories(cfm);
         }
 
@@ -295,22 +295,19 @@ public class CassandraDaemon
             }
         }
 
-        Runnable indexRebuild = new Runnable()
+        Runnable viewRebuild = new Runnable()
         {
             @Override
             public void run()
             {
                 for (Keyspace keyspace : Keyspace.all())
                 {
-                    for (ColumnFamilyStore cf: keyspace.getColumnFamilyStores())
-                    {
-                        cf.materializedViewManager.buildAllViews();
-                    }
+                    keyspace.viewManager.buildAllViews();
                 }
             }
         };
 
-        ScheduledExecutors.optionalTasks.schedule(indexRebuild, StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
+        ScheduledExecutors.optionalTasks.schedule(viewRebuild, StorageService.RING_DELAY, TimeUnit.MILLISECONDS);
 
 
         SystemKeyspace.finishStartup();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/service/MigrationListener.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationListener.java b/src/java/org/apache/cassandra/service/MigrationListener.java
index 358b236..f4b3e7c 100644
--- a/src/java/org/apache/cassandra/service/MigrationListener.java
+++ b/src/java/org/apache/cassandra/service/MigrationListener.java
@@ -31,6 +31,11 @@ public abstract class MigrationListener
     {
     }
 
+    public void onCreateView(String ksName, String viewName)
+    {
+        onCreateColumnFamily(ksName, viewName);
+    }
+
     public void onCreateUserType(String ksName, String typeName)
     {
     }
@@ -51,6 +56,11 @@ public abstract class MigrationListener
     {
     }
 
+    public void onUpdateView(String ksName, String viewName, boolean columnsDidChange)
+    {
+        onUpdateColumnFamily(ksName, viewName, columnsDidChange);
+    }
+
     public void onUpdateUserType(String ksName, String typeName)
     {
     }
@@ -71,6 +81,10 @@ public abstract class MigrationListener
     {
     }
 
+    public void onDropView(String ksName, String viewName)
+    {
+    }
+
     public void onDropUserType(String ksName, String typeName)
     {
     }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/service/MigrationManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/MigrationManager.java b/src/java/org/apache/cassandra/service/MigrationManager.java
index dad6aa7..c820f18 100644
--- a/src/java/org/apache/cassandra/service/MigrationManager.java
+++ b/src/java/org/apache/cassandra/service/MigrationManager.java
@@ -31,6 +31,7 @@ import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.concurrent.StageManager;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.cql3.functions.UDAggregate;
 import org.apache.cassandra.cql3.functions.UDFunction;
 import org.apache.cassandra.db.*;
@@ -162,6 +163,12 @@ public class MigrationManager
             listener.onCreateColumnFamily(cfm.ksName, cfm.cfName);
     }
 
+    public void notifyCreateView(ViewDefinition view)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onCreateView(view.ksName, view.viewName);
+    }
+
     public void notifyCreateUserType(UserType ut)
     {
         for (MigrationListener listener : listeners)
@@ -192,6 +199,12 @@ public class MigrationManager
             listener.onUpdateColumnFamily(cfm.ksName, cfm.cfName, columnsDidChange);
     }
 
+    public void notifyUpdateView(ViewDefinition view, boolean columnsDidChange)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onUpdateView(view.ksName, view.viewName, columnsDidChange);
+    }
+
     public void notifyUpdateUserType(UserType ut)
     {
         for (MigrationListener listener : listeners)
@@ -225,6 +238,12 @@ public class MigrationManager
             listener.onDropColumnFamily(cfm.ksName, cfm.cfName);
     }
 
+    public void notifyDropView(ViewDefinition view)
+    {
+        for (MigrationListener listener : listeners)
+            listener.onDropView(view.ksName, view.viewName);
+    }
+
     public void notifyDropUserType(UserType ut)
     {
         for (MigrationListener listener : listeners)
@@ -276,13 +295,28 @@ public class MigrationManager
         KeyspaceMetadata ksm = Schema.instance.getKSMetaData(cfm.ksName);
         if (ksm == null)
             throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", cfm.cfName, cfm.ksName));
-        else if (ksm.tables.get(cfm.cfName).isPresent())
+        // If we have a table or a view which has the same name, we can't add a new one
+        else if (ksm.getTableOrViewNullable(cfm.cfName) != null)
             throw new AlreadyExistsException(cfm.ksName, cfm.cfName);
 
         logger.info(String.format("Create new table: %s", cfm));
         announce(SchemaKeyspace.makeCreateTableMutation(ksm, cfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
+    public static void announceNewView(ViewDefinition view, boolean announceLocally) throws ConfigurationException
+    {
+        view.metadata.validate();
+
+        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(view.ksName);
+        if (ksm == null)
+            throw new ConfigurationException(String.format("Cannot add table '%s' to non existing keyspace '%s'.", view.viewName, view.ksName));
+        else if (ksm.getTableOrViewNullable(view.viewName) != null)
+            throw new AlreadyExistsException(view.ksName, view.viewName);
+
+        logger.info(String.format("Create new view: %s", view));
+        announce(SchemaKeyspace.makeCreateViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
+    }
+
     public static void announceNewType(UserType newType, boolean announceLocally)
     {
         KeyspaceMetadata ksm = Schema.instance.getKSMetaData(newType.keyspace);
@@ -340,6 +374,21 @@ public class MigrationManager
         announce(SchemaKeyspace.makeUpdateTableMutation(ksm, oldCfm, cfm, FBUtilities.timestampMicros(), fromThrift), announceLocally);
     }
 
+    public static void announceViewUpdate(ViewDefinition view, boolean announceLocally) throws ConfigurationException
+    {
+        view.metadata.validate();
+
+        ViewDefinition oldView = Schema.instance.getView(view.ksName, view.viewName);
+        if (oldView == null)
+            throw new ConfigurationException(String.format("Cannot update non existing materialized view '%s' in keyspace '%s'.", view.viewName, view.ksName));
+        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(view.ksName);
+
+        oldView.metadata.validateCompatility(view.metadata);
+
+        logger.info(String.format("Update view '%s/%s' From %s To %s", view.ksName, view.viewName, oldView, view));
+        announce(SchemaKeyspace.makeUpdateViewMutation(ksm, oldView, view, FBUtilities.timestampMicros()), announceLocally);
+    }
+
     public static void announceTypeUpdate(UserType updatedType, boolean announceLocally)
     {
         announceNewType(updatedType, announceLocally);
@@ -376,6 +425,17 @@ public class MigrationManager
         announce(SchemaKeyspace.makeDropTableMutation(ksm, oldCfm, FBUtilities.timestampMicros()), announceLocally);
     }
 
+    public static void announceViewDrop(String ksName, String viewName, boolean announceLocally) throws ConfigurationException
+    {
+        ViewDefinition view = Schema.instance.getView(ksName, viewName);
+        if (view == null)
+            throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", viewName, ksName));
+        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(ksName);
+
+        logger.info(String.format("Drop table '%s/%s'", view.ksName, view.viewName));
+        announce(SchemaKeyspace.makeDropViewMutation(ksm, view, FBUtilities.timestampMicros()), announceLocally);
+    }
+
     public static void announceTypeDrop(UserType droppedType)
     {
         announceTypeDrop(droppedType, false);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/service/StartupChecks.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StartupChecks.java b/src/java/org/apache/cassandra/service/StartupChecks.java
index eca0c7e..16ff488 100644
--- a/src/java/org/apache/cassandra/service/StartupChecks.java
+++ b/src/java/org/apache/cassandra/service/StartupChecks.java
@@ -270,7 +270,7 @@ public class StartupChecks
             // we do a one-off scrub of the system keyspace first; we can't load the list of the rest of the keyspaces,
             // until system keyspace is opened.
 
-            for (CFMetaData cfm : Schema.instance.getTables(SystemKeyspace.NAME))
+            for (CFMetaData cfm : Schema.instance.getTablesAndViews(SystemKeyspace.NAME))
                 ColumnFamilyStore.scrubDataDirectories(cfm);
 
             try

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/service/StorageProxy.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageProxy.java b/src/java/org/apache/cassandra/service/StorageProxy.java
index e3b884e..d209af6 100644
--- a/src/java/org/apache/cassandra/service/StorageProxy.java
+++ b/src/java/org/apache/cassandra/service/StorageProxy.java
@@ -48,12 +48,9 @@ import org.apache.cassandra.db.filter.DataLimits;
 import org.apache.cassandra.db.filter.TombstoneOverwhelmingException;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.RowIterator;
-import org.apache.cassandra.db.view.MaterializedViewManager;
-import org.apache.cassandra.db.view.MaterializedViewUtils;
-import org.apache.cassandra.dht.AbstractBounds;
-import org.apache.cassandra.dht.Bounds;
-import org.apache.cassandra.dht.RingPosition;
-import org.apache.cassandra.dht.Token;
+import org.apache.cassandra.db.view.ViewManager;
+import org.apache.cassandra.db.view.ViewUtils;
+import org.apache.cassandra.dht.*;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.gms.FailureDetector;
 import org.apache.cassandra.gms.Gossiper;
@@ -98,7 +95,7 @@ public class StorageProxy implements StorageProxyMBean
     private static final ClientRequestMetrics writeMetrics = new ClientRequestMetrics("Write");
     private static final CASClientRequestMetrics casWriteMetrics = new CASClientRequestMetrics("CASWrite");
     private static final CASClientRequestMetrics casReadMetrics = new CASClientRequestMetrics("CASRead");
-    private static final MVWriteMetrics mvWriteMetrics = new MVWriteMetrics("MVWrite");
+    private static final ViewWriteMetrics viewWriteMetrics = new ViewWriteMetrics("ViewWrite");
 
     private static final double CONCURRENT_SUBREQUESTS_MARGIN = 0.10;
 
@@ -655,6 +652,7 @@ public class StorageProxy implements StorageProxyMBean
      * @param mutations the mutations to be applied across the replicas
      */
     public static void mutateMV(ByteBuffer dataKey, Collection<Mutation> mutations, boolean writeCommitLog)
+    throws UnavailableException, OverloadedException, WriteTimeoutException
     {
         Tracing.trace("Determining replicas for mutation");
         final String localDataCenter = DatabaseDescriptor.getEndpointSnitch().getDatacenter(FBUtilities.getBroadcastAddress());
@@ -679,15 +677,15 @@ public class StorageProxy implements StorageProxyMBean
             {
                 String keyspaceName = mutation.getKeyspaceName();
                 Token tk = mutation.key().getToken();
-                InetAddress pairedEndpoint = MaterializedViewUtils.getViewNaturalEndpoint(keyspaceName, baseToken, tk);
+                InetAddress pairedEndpoint = ViewUtils.getViewNaturalEndpoint(keyspaceName, baseToken, tk);
                 List<InetAddress> naturalEndpoints = Lists.newArrayList(pairedEndpoint);
 
-                WriteResponseHandlerWrapper wrapper = wrapMVBatchResponseHandler(mutation,
-                                                                                 consistencyLevel,
-                                                                                 consistencyLevel,
-                                                                                 naturalEndpoints,
-                                                                                 WriteType.BATCH,
-                                                                                 cleanup);
+                WriteResponseHandlerWrapper wrapper = wrapViewBatchResponseHandler(mutation,
+                                                                                   consistencyLevel,
+                                                                                   consistencyLevel,
+                                                                                   naturalEndpoints,
+                                                                                   WriteType.BATCH,
+                                                                                   cleanup);
 
                 // When local node is the endpoint and there are no pending nodes we can
                 // Just apply the mutation locally.
@@ -704,12 +702,12 @@ public class StorageProxy implements StorageProxyMBean
                                       writeCommitLog);
 
                 // now actually perform the writes and wait for them to complete
-                asyncWriteBatchedMutations(wrappers, localDataCenter, Stage.MATERIALIZED_VIEW_MUTATION);
+                asyncWriteBatchedMutations(wrappers, localDataCenter, Stage.VIEW_MUTATION);
             }
         }
         finally
         {
-            mvWriteMetrics.addNano(System.nanoTime() - startTime);
+            viewWriteMetrics.addNano(System.nanoTime() - startTime);
         }
     }
 
@@ -721,7 +719,9 @@ public class StorageProxy implements StorageProxyMBean
     {
         Collection<Mutation> augmented = TriggerExecutor.instance.execute(mutations);
 
-        boolean updatesView = MaterializedViewManager.updatesAffectView(mutations, true);
+        boolean updatesView = Keyspace.open(mutations.iterator().next().getKeyspaceName())
+                              .viewManager
+                              .updatesAffectView(mutations, true);
 
         if (augmented != null)
             mutateAtomically(augmented, consistencyLevel, updatesView);
@@ -974,14 +974,14 @@ public class StorageProxy implements StorageProxyMBean
 
     /**
      * Same as performWrites except does not initiate writes (but does perform availability checks).
-     * Keeps track of MVWriteMetrics
+     * Keeps track of ViewWriteMetrics
      */
-    private static WriteResponseHandlerWrapper wrapMVBatchResponseHandler(Mutation mutation,
-                                                                          ConsistencyLevel consistency_level,
-                                                                          ConsistencyLevel batchConsistencyLevel,
-                                                                          List<InetAddress> naturalEndpoints,
-                                                                          WriteType writeType,
-                                                                          BatchlogResponseHandler.BatchlogCleanup cleanup)
+    private static WriteResponseHandlerWrapper wrapViewBatchResponseHandler(Mutation mutation,
+                                                                            ConsistencyLevel consistency_level,
+                                                                            ConsistencyLevel batchConsistencyLevel,
+                                                                            List<InetAddress> naturalEndpoints,
+                                                                            WriteType writeType,
+                                                                            BatchlogResponseHandler.BatchlogCleanup cleanup)
     {
         Keyspace keyspace = Keyspace.open(mutation.getKeyspaceName());
         AbstractReplicationStrategy rs = keyspace.getReplicationStrategy();
@@ -989,7 +989,7 @@ public class StorageProxy implements StorageProxyMBean
         Token tk = mutation.key().getToken();
         Collection<InetAddress> pendingEndpoints = StorageService.instance.getTokenMetadata().pendingEndpointsFor(tk, keyspaceName);
         AbstractWriteResponseHandler<IMutation> writeHandler = rs.getWriteResponseHandler(naturalEndpoints, pendingEndpoints, consistency_level, null, writeType);
-        BatchlogResponseHandler<IMutation> batchHandler = new MVWriteMetricsWrapped(writeHandler, batchConsistencyLevel.blockFor(keyspace), cleanup);
+        BatchlogResponseHandler<IMutation> batchHandler = new ViewWriteMetricsWrapped(writeHandler, batchConsistencyLevel.blockFor(keyspace), cleanup);
         return new WriteResponseHandlerWrapper(batchHandler, mutation);
     }
 
@@ -2282,20 +2282,20 @@ public class StorageProxy implements StorageProxyMBean
     }
 
     /**
-     * This class captures metrics for materialized views writes.
+     * This class captures metrics for views writes.
      */
-    private static class MVWriteMetricsWrapped extends BatchlogResponseHandler<IMutation>
+    private static class ViewWriteMetricsWrapped extends BatchlogResponseHandler<IMutation>
     {
-        public MVWriteMetricsWrapped(AbstractWriteResponseHandler<IMutation> writeHandler, int i, BatchlogCleanup cleanup)
+        public ViewWriteMetricsWrapped(AbstractWriteResponseHandler<IMutation> writeHandler, int i, BatchlogCleanup cleanup)
         {
             super(writeHandler, i, cleanup);
-            mvWriteMetrics.viewReplicasAttempted.inc(totalEndpoints());
+            viewWriteMetrics.viewReplicasAttempted.inc(totalEndpoints());
         }
 
         public void response(MessageIn<IMutation> msg)
         {
             super.response(msg);
-            mvWriteMetrics.viewReplicasSuccess.inc();
+            viewWriteMetrics.viewReplicasSuccess.inc();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/service/StorageService.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/service/StorageService.java b/src/java/org/apache/cassandra/service/StorageService.java
index f095630..f0ad46f 100644
--- a/src/java/org/apache/cassandra/service/StorageService.java
+++ b/src/java/org/apache/cassandra/service/StorageService.java
@@ -630,12 +630,12 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
             public void runMayThrow() throws InterruptedException
             {
                 inShutdownHook = true;
-                ExecutorService materializedViewMutationStage = StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION);
+                ExecutorService viewMutationStage = StageManager.getStage(Stage.VIEW_MUTATION);
                 ExecutorService counterMutationStage = StageManager.getStage(Stage.COUNTER_MUTATION);
                 ExecutorService mutationStage = StageManager.getStage(Stage.MUTATION);
                 if (mutationStage.isShutdown()
                     && counterMutationStage.isShutdown()
-                    && materializedViewMutationStage.isShutdown())
+                    && viewMutationStage.isShutdown())
                     return; // drained already
 
                 if (daemon != null)
@@ -646,11 +646,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
                 // In-progress writes originating here could generate hints to be written, so shut down MessagingService
                 // before mutation stage, so we can get all the hints saved before shutting down
                 MessagingService.instance().shutdown();
-                materializedViewMutationStage.shutdown();
+                viewMutationStage.shutdown();
                 HintsService.instance.pauseDispatch();
                 counterMutationStage.shutdown();
                 mutationStage.shutdown();
-                materializedViewMutationStage.awaitTermination(3600, TimeUnit.SECONDS);
+                viewMutationStage.awaitTermination(3600, TimeUnit.SECONDS);
                 counterMutationStage.awaitTermination(3600, TimeUnit.SECONDS);
                 mutationStage.awaitTermination(3600, TimeUnit.SECONDS);
                 StorageProxy.instance.verifyNoHintsInProgress();
@@ -3179,7 +3179,7 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         if (ksMetaData == null)
             throw new IllegalArgumentException("Unknown keyspace '" + keyspaceName + "'");
 
-        CFMetaData cfMetaData = ksMetaData.tables.getNullable(cf);
+        CFMetaData cfMetaData = ksMetaData.getTableOrViewNullable(cf);
         if (cfMetaData == null)
             throw new IllegalArgumentException("Unknown table '" + cf + "' in keyspace '" + keyspaceName + "'");
 
@@ -3876,11 +3876,11 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         HintsService.instance.pauseDispatch();
 
         ExecutorService counterMutationStage = StageManager.getStage(Stage.COUNTER_MUTATION);
-        ExecutorService materializedViewMutationStage = StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION);
+        ExecutorService viewMutationStage = StageManager.getStage(Stage.VIEW_MUTATION);
         ExecutorService mutationStage = StageManager.getStage(Stage.MUTATION);
         if (mutationStage.isTerminated()
             && counterMutationStage.isTerminated()
-            && materializedViewMutationStage.isTerminated())
+            && viewMutationStage.isTerminated())
         {
             logger.warn("Cannot drain node (did it already happen?)");
             return;
@@ -3894,10 +3894,10 @@ public class StorageService extends NotificationBroadcasterSupport implements IE
         MessagingService.instance().shutdown();
 
         setMode(Mode.DRAINING, "clearing mutation stage", false);
-        materializedViewMutationStage.shutdown();
+        viewMutationStage.shutdown();
         counterMutationStage.shutdown();
         mutationStage.shutdown();
-        materializedViewMutationStage.awaitTermination(3600, TimeUnit.SECONDS);
+        viewMutationStage.awaitTermination(3600, TimeUnit.SECONDS);
         counterMutationStage.awaitTermination(3600, TimeUnit.SECONDS);
         mutationStage.awaitTermination(3600, TimeUnit.SECONDS);
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
index cb99654..f261954 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -21,6 +21,7 @@ import java.util.*;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import com.google.common.collect.Iterables;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -33,10 +34,10 @@ import org.apache.cassandra.db.compaction.OperationType;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.io.sstable.ISSTableScanner;
 import org.apache.cassandra.io.sstable.SSTableMultiWriter;
 import org.apache.cassandra.io.sstable.format.SSTableReader;
-import org.apache.cassandra.io.sstable.format.SSTableWriter;
 import org.apache.cassandra.utils.JVMStabilityInspector;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.concurrent.Refs;
@@ -127,7 +128,7 @@ public class StreamReceiveTask extends StreamTask
                 return;
             }
             ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
-            boolean hasMaterializedViews = cfs.materializedViewManager.allViews().iterator().hasNext();
+            boolean hasViews = !Iterables.isEmpty(View.findAll(kscf.left, kscf.right));
 
             try
             {
@@ -143,11 +144,11 @@ public class StreamReceiveTask extends StreamTask
 
                 try (Refs<SSTableReader> refs = Refs.ref(readers))
                 {
-                    //We have a special path for Materialized view.
-                    //Since the MV requires cleaning up any pre-existing state, we must put
+                    //We have a special path for views.
+                    //Since the view requires cleaning up any pre-existing state, we must put
                     //all partitions through the same write path as normal mutations.
-                    //This also ensures any 2i's are also updated
-                    if (hasMaterializedViews)
+                    //This also ensures any 2is are also updated
+                    if (hasViews)
                     {
                         for (SSTableReader reader : readers)
                         {
@@ -183,7 +184,7 @@ public class StreamReceiveTask extends StreamTask
                 {
                     //We don't keep the streamed sstables since we've applied them manually
                     //So we abort the txn and delete the streamed sstables
-                    if (hasMaterializedViews)
+                    if (hasViews)
                     {
                         cfs.forceBlockingFlush();
                         task.txn.abort();

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/thrift/CassandraServer.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/CassandraServer.java b/src/java/org/apache/cassandra/thrift/CassandraServer.java
index 207bb6f..7017bc1 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -44,6 +44,7 @@ import org.apache.cassandra.db.marshal.AbstractType;
 import org.apache.cassandra.db.marshal.TimeUUIDType;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.*;
+import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.*;
@@ -815,7 +816,7 @@ public class CassandraServer implements Cassandra.Iface
         cState.hasColumnFamilyAccess(keyspace, column_parent.column_family, Permission.MODIFY);
 
         CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family, false);
-        if (metadata.isMaterializedView())
+        if (metadata.isView())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
 
         ThriftValidation.validateKey(metadata, key);
@@ -912,7 +913,7 @@ public class CassandraServer implements Cassandra.Iface
             cState.hasColumnFamilyAccess(keyspace, column_family, Permission.SELECT);
 
             CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_family, false);
-            if (metadata.isMaterializedView())
+            if (metadata.isView())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
 
             ThriftValidation.validateKey(metadata, key);
@@ -1108,7 +1109,7 @@ public class CassandraServer implements Cassandra.Iface
                 cState.hasColumnFamilyAccess(keyspace, cfName, Permission.MODIFY);
 
                 CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfName);
-                if (metadata.isMaterializedView())
+                if (metadata.isView())
                     throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
 
                 ThriftValidation.validateKey(metadata, key);
@@ -1325,7 +1326,7 @@ public class CassandraServer implements Cassandra.Iface
         cState.hasColumnFamilyAccess(keyspace, column_path.column_family, Permission.MODIFY);
 
         CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_path.column_family, isCommutativeOp);
-        if (metadata.isMaterializedView())
+        if (metadata.isView())
             throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
 
         ThriftValidation.validateKey(metadata, key);
@@ -1903,7 +1904,7 @@ public class CassandraServer implements Cassandra.Iface
             cState.hasColumnFamilyAccess(keyspace, column_family, Permission.DROP);
 
             CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_family);
-            if (metadata.isMaterializedView())
+            if (metadata.isView())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot drop Materialized Views from Thrift");
 
             MigrationManager.announceColumnFamilyDrop(keyspace, column_family);
@@ -2015,9 +2016,9 @@ public class CassandraServer implements Cassandra.Iface
             if (oldCfm == null)
                 throw new InvalidRequestException("Could not find table definition to modify.");
 
-            if (oldCfm.isMaterializedView())
+            if (oldCfm.isView())
                 throw new InvalidRequestException("Cannot modify Materialized View table " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify Materialized View tables instead.");
-            if (!oldCfm.getMaterializedViews().isEmpty())
+            if (!Iterables.isEmpty(View.findAll(cf_def.keyspace, cf_def.name)))
                 throw new InvalidRequestException("Cannot modify table with Materialized View " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify tables with Materialized Views instead.");
 
             if (!oldCfm.isThriftCompatible())
@@ -2047,7 +2048,7 @@ public class CassandraServer implements Cassandra.Iface
             String keyspace = cState.getKeyspace();
             cState.hasColumnFamilyAccess(keyspace, cfname, Permission.MODIFY);
             CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfname, false);
-            if (metadata.isMaterializedView())
+            if (metadata.isView())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot truncate Materialized Views");
 
             if (startSessionIfRequested())
@@ -2134,7 +2135,7 @@ public class CassandraServer implements Cassandra.Iface
             cState.hasColumnFamilyAccess(keyspace, column_parent.column_family, Permission.MODIFY);
 
             CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_parent.column_family, true);
-            if (metadata.isMaterializedView())
+            if (metadata.isView())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
 
             ThriftValidation.validateKey(metadata, key);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/thrift/ThriftConversion.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/thrift/ThriftConversion.java b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
index 86cfe42..b721226 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -162,7 +162,7 @@ public class ThriftConversion
     public static KsDef toThrift(KeyspaceMetadata ksm)
     {
         List<CfDef> cfDefs = new ArrayList<>();
-        for (CFMetaData cfm : ksm.tables)
+        for (CFMetaData cfm : ksm.tables) // do not include views
             if (cfm.isThriftCompatible()) // Don't expose CF that cannot be correctly handle by thrift; see CASSANDRA-4377 for further details
                 cfDefs.add(toThrift(cfm));
 
@@ -272,8 +272,8 @@ public class ThriftConversion
                                       defaultValidator);
             }
 
-            // We do not allow Thrift materialized views, so we always set it to false
-            boolean isMaterializedView = false;
+            // We do not allow Thrift views, so we always set it to false
+            boolean isView = false;
 
             CFMetaData newCFMD = CFMetaData.create(cf_def.keyspace,
                                                    cf_def.name,
@@ -282,7 +282,7 @@ public class ThriftConversion
                                                    isCompound,
                                                    isSuper,
                                                    isCounter,
-                                                   isMaterializedView,
+                                                   isView,
                                                    defs,
                                                    DatabaseDescriptor.getPartitioner());
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
index d602076..8dff532 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -80,6 +80,8 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
             }
 
             tables.putAll(fetchTablesMetadata(keyspace, session, partitioner));
+            // We only need the CFMetaData for the views, so we only load that.
+            tables.putAll(fetchViewMetadata(keyspace, session, partitioner));
         }
     }
 
@@ -111,41 +113,61 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
         for (Row row : session.execute(query, keyspace))
         {
             String name = row.getString("table_name");
-            UUID id = row.getUUID("id");
-
-            Set<CFMetaData.Flag> flags = row.isNull("flags")
-                                       ? Collections.emptySet()
-                                       : CFMetaData.flagsFromStrings(row.getSet("flags", String.class));
-
-            boolean isSuper = flags.contains(CFMetaData.Flag.SUPER);
-            boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER);
-            boolean isDense = flags.contains(CFMetaData.Flag.DENSE);
-            boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND);
-            boolean isMaterializedView = flags.contains(CFMetaData.Flag.VIEW);
-
-            String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?",
-                                                SchemaKeyspace.NAME,
-                                                SchemaKeyspace.COLUMNS);
-
-            List<ColumnDefinition> defs = new ArrayList<>();
-            for (Row colRow : session.execute(columnsQuery, keyspace, name))
-                defs.add(createDefinitionFromRow(colRow, keyspace, name));
-
-            tables.put(name, CFMetaData.create(keyspace,
-                                               name,
-                                               id,
-                                               isDense,
-                                               isCompound,
-                                               isSuper,
-                                               isCounter,
-                                               isMaterializedView,
-                                               defs,
-                                               partitioner));
+            tables.put(name, createTableMetadata(keyspace, session, partitioner, false, row, name));
         }
 
         return tables;
     }
 
+    /*
+     * In the case where we are creating View CFMetaDatas, we
+     */
+    private static Map<String, CFMetaData> fetchViewMetadata(String keyspace, Session session, IPartitioner partitioner)
+    {
+        Map<String, CFMetaData> tables = new HashMap<>();
+        String query = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ?", SchemaKeyspace.NAME, SchemaKeyspace.VIEWS);
+
+        for (Row row : session.execute(query, keyspace))
+        {
+            String name = row.getString("view_name");
+            tables.put(name, createTableMetadata(keyspace, session, partitioner, true, row, name));
+        }
+
+        return tables;
+    }
+
+    private static CFMetaData createTableMetadata(String keyspace, Session session, IPartitioner partitioner, boolean isView, Row row, String name)
+    {
+        UUID id = row.getUUID("id");
+        Set<CFMetaData.Flag> flags = row.isNull("flags")
+                                     ? Collections.emptySet()
+                                     : CFMetaData.flagsFromStrings(row.getSet("flags", String.class));
+
+        boolean isSuper = flags.contains(CFMetaData.Flag.SUPER);
+        boolean isCounter = flags.contains(CFMetaData.Flag.COUNTER);
+        boolean isDense = flags.contains(CFMetaData.Flag.DENSE);
+        boolean isCompound = flags.contains(CFMetaData.Flag.COMPOUND);
+
+        String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?",
+                                            SchemaKeyspace.NAME,
+                                            SchemaKeyspace.COLUMNS);
+
+        List<ColumnDefinition> defs = new ArrayList<>();
+        for (Row colRow : session.execute(columnsQuery, keyspace, name))
+            defs.add(createDefinitionFromRow(colRow, keyspace, name));
+
+        return CFMetaData.create(keyspace,
+                                 name,
+                                 id,
+                                 isDense,
+                                 isCompound,
+                                 isSuper,
+                                 isCounter,
+                                 isView,
+                                 defs,
+                                 partitioner);
+    }
+
     private static ColumnDefinition createDefinitionFromRow(Row row, String keyspace, String table)
     {
         ColumnIdentifier name = ColumnIdentifier.getInterned(row.getBytes("column_name_bytes"), row.getString("column_name"));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java b/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java
deleted file mode 100644
index b833e60..0000000
--- a/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java
+++ /dev/null
@@ -1,188 +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.cql3;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CyclicBarrier;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.datastax.driver.core.Row;
-import com.datastax.driver.core.exceptions.NoHostAvailableException;
-import com.datastax.driver.core.exceptions.WriteTimeoutException;
-import org.apache.cassandra.concurrent.SEPExecutor;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.batchlog.BatchlogManager;
-import org.apache.cassandra.utils.WrappedRunnable;
-
-public class MaterializedViewLongTest extends CQLTester
-{
-    int protocolVersion = 4;
-    private final List<String> materializedViews = new ArrayList<>();
-
-    @BeforeClass
-    public static void startup()
-    {
-        requireNetwork();
-    }
-    @Before
-    public void begin()
-    {
-        materializedViews.clear();
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        for (String viewName : materializedViews)
-            executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + viewName);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        executeNet(protocolVersion, String.format(query, name));
-        // If exception is thrown, the view will not be added to the list; since it shouldn't have been created, this is
-        // the desired behavior
-        materializedViews.add(name);
-    }
-
-    @Test
-    public void testConflictResolution() throws Throwable
-    {
-        final int writers = 96;
-        final int insertsPerWriter = 50;
-        final Map<Integer, Exception> failedWrites = new ConcurrentHashMap<>();
-
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b int," +
-                    "c int," +
-                    "PRIMARY KEY (a, b))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
-
-        CyclicBarrier semaphore = new CyclicBarrier(writers);
-
-        Thread[] threads = new Thread[writers];
-        for (int i = 0; i < writers; i++)
-        {
-            final int writer = i;
-            Thread t = new Thread(new WrappedRunnable()
-            {
-                public void runMayThrow()
-                {
-                    try
-                    {
-                        int writerOffset = writer * insertsPerWriter;
-                        semaphore.await();
-                        for (int i = 0; i < insertsPerWriter; i++)
-                        {
-                            try
-                            {
-                                executeNet(protocolVersion, "INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP 1",
-                                           1,
-                                           1,
-                                           i + writerOffset);
-                            }
-                            catch (NoHostAvailableException|WriteTimeoutException e)
-                            {
-                                failedWrites.put(i + writerOffset, e);
-                            }
-                        }
-                    }
-                    catch (Throwable e)
-                    {
-                        throw new RuntimeException(e);
-                    }
-                }
-            });
-            t.start();
-            threads[i] = t;
-        }
-
-        for (int i = 0; i < writers; i++)
-            threads[i].join();
-
-        for (int i = 0; i < writers * insertsPerWriter; i++)
-        {
-            if (executeNet(protocolVersion, "SELECT COUNT(*) FROM system.batchlog").one().getLong(0) == 0)
-                break;
-            try
-            {
-                // This will throw exceptions whenever there are exceptions trying to push the materialized view values
-                // out, caused by the view becoming overwhelmed.
-                BatchlogManager.instance.startBatchlogReplay().get();
-            }
-            catch (Throwable ignore)
-            {
-
-            }
-        }
-
-        int value = executeNet(protocolVersion, "SELECT c FROM %s WHERE a = 1 AND b = 1").one().getInt("c");
-
-        List<Row> rows = executeNet(protocolVersion, "SELECT c FROM " + keyspace() + ".mv").all();
-
-        boolean containsC = false;
-        StringBuilder others = new StringBuilder();
-        StringBuilder overlappingFailedWrites = new StringBuilder();
-        for (Row row : rows)
-        {
-            int c = row.getInt("c");
-            if (c == value)
-                containsC = true;
-            else
-            {
-                if (others.length() != 0)
-                    others.append(' ');
-                others.append(c);
-                if (failedWrites.containsKey(c))
-                {
-                    if (overlappingFailedWrites.length() != 0)
-                        overlappingFailedWrites.append(' ');
-                    overlappingFailedWrites.append(c)
-                                           .append(':')
-                                           .append(failedWrites.get(c).getMessage());
-                }
-            }
-        }
-
-        if (rows.size() > 1)
-        {
-            throw new AssertionError(String.format("Expected 1 row, but found %d; %s c = %d, and (%s) of which (%s) failed to insert", rows.size(), containsC ? "found row with" : "no rows contained", value, others, overlappingFailedWrites));
-        }
-        else if (rows.isEmpty())
-        {
-            throw new AssertionError(String.format("Could not find row with c = %d", value));
-        }
-        else if (rows.size() == 1 && !containsC)
-        {
-            throw new AssertionError(String.format("Single row had c = %d, expected %d", rows.get(0).getInt("c"), value));
-        }
-    }
-}


[7/7] cassandra git commit: Merge branch 'cassandra-3.0' into trunk

Posted by al...@apache.org.
Merge branch 'cassandra-3.0' into trunk


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

Branch: refs/heads/trunk
Commit: 43ddec31b6b09a35708d45891a583c1a072399dc
Parents: 2f82a69 a3a8dbc
Author: Aleksey Yeschenko <al...@apache.org>
Authored: Wed Sep 16 15:35:25 2015 +0100
Committer: Aleksey Yeschenko <al...@apache.org>
Committed: Wed Sep 16 15:35:25 2015 +0100

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 ...ore-3.0.0-alpha3-55db84c-SNAPSHOT-shaded.jar |  Bin 2209303 -> 0 bytes
 ...ore-3.0.0-alpha3-8bd064d-SNAPSHOT-shaded.jar |  Bin 0 -> 2218913 bytes
 ...iver-internal-only-3.0.0a2.post0-2429ba3.zip |  Bin 229078 -> 0 bytes
 ...iver-internal-only-3.0.0a2.post0-96883eb.zip |  Bin 0 -> 230630 bytes
 .../org/apache/cassandra/concurrent/Stage.java  |    4 +-
 .../cassandra/concurrent/StageManager.java      |    2 +-
 .../org/apache/cassandra/config/CFMetaData.java |   55 +-
 .../cassandra/config/DatabaseDescriptor.java    |    2 +-
 .../config/MaterializedViewDefinition.java      |   93 --
 .../org/apache/cassandra/config/Schema.java     |  111 +-
 .../apache/cassandra/config/ViewDefinition.java |  114 ++
 src/java/org/apache/cassandra/cql3/Cql.g        |   12 +-
 .../AlterMaterializedViewStatement.java         |   91 --
 .../cql3/statements/AlterTableStatement.java    |   99 +-
 .../cql3/statements/AlterTypeStatement.java     |   10 +
 .../cql3/statements/AlterViewStatement.java     |   94 ++
 .../CreateMaterializedViewStatement.java        |  286 ----
 .../cql3/statements/CreateTriggerStatement.java |    2 +-
 .../cql3/statements/CreateViewStatement.java    |  321 +++++
 .../DropMaterializedViewStatement.java          |   92 --
 .../cql3/statements/DropTableStatement.java     |   21 +-
 .../cql3/statements/DropTypeStatement.java      |    2 +-
 .../cql3/statements/DropViewStatement.java      |   93 ++
 .../cql3/statements/ModificationStatement.java  |   38 +-
 .../cql3/statements/SelectStatement.java        |    6 +-
 .../cql3/statements/TruncateStatement.java      |    2 +-
 .../apache/cassandra/db/ColumnFamilyStore.java  |   18 +-
 src/java/org/apache/cassandra/db/Keyspace.java  |   20 +-
 .../org/apache/cassandra/db/SystemKeyspace.java |   72 +-
 src/java/org/apache/cassandra/db/WriteType.java |    2 +-
 .../db/compaction/CompactionManager.java        |    4 +-
 .../cassandra/db/compaction/OperationType.java  |    2 +-
 .../cassandra/db/view/MaterializedView.java     |  749 ----------
 .../db/view/MaterializedViewBuilder.java        |  215 ---
 .../db/view/MaterializedViewManager.java        |  241 ----
 .../db/view/MaterializedViewUtils.java          |  109 --
 src/java/org/apache/cassandra/db/view/View.java |  682 +++++++++
 .../apache/cassandra/db/view/ViewBuilder.java   |  214 +++
 .../apache/cassandra/db/view/ViewManager.java   |  271 ++++
 .../org/apache/cassandra/db/view/ViewUtils.java |  109 ++
 .../cassandra/dht/ByteOrderedPartitioner.java   |    2 +-
 .../dht/OrderPreservingPartitioner.java         |    2 +-
 .../cassandra/metrics/MVWriteMetrics.java       |   42 -
 .../cassandra/metrics/ViewWriteMetrics.java     |   42 +
 .../cassandra/schema/KeyspaceMetadata.java      |   52 +-
 .../cassandra/schema/MaterializedViews.java     |  149 --
 .../apache/cassandra/schema/SchemaKeyspace.java |  292 ++--
 src/java/org/apache/cassandra/schema/Views.java |  171 +++
 .../cassandra/service/CassandraDaemon.java      |   11 +-
 .../cassandra/service/MigrationListener.java    |   14 +
 .../cassandra/service/MigrationManager.java     |   62 +-
 .../apache/cassandra/service/StartupChecks.java |    2 +-
 .../apache/cassandra/service/StorageProxy.java  |   60 +-
 .../cassandra/service/StorageService.java       |   18 +-
 .../cassandra/streaming/StreamReceiveTask.java  |   15 +-
 .../cassandra/thrift/CassandraServer.java       |   19 +-
 .../cassandra/thrift/ThriftConversion.java      |    8 +-
 .../utils/NativeSSTableLoaderClient.java        |   82 +-
 .../cql3/MaterializedViewLongTest.java          |  188 ---
 .../org/apache/cassandra/cql3/ViewLongTest.java |  188 +++
 .../unit/org/apache/cassandra/SchemaLoader.java |    2 +-
 .../config/DatabaseDescriptorTest.java          |    2 +-
 .../cassandra/cql3/MaterializedViewTest.java    | 1251 ----------------
 .../org/apache/cassandra/cql3/ViewTest.java     | 1355 ++++++++++++++++++
 .../db/view/MaterializedViewUtilsTest.java      |  115 --
 .../apache/cassandra/db/view/ViewUtilsTest.java |  115 ++
 .../apache/cassandra/dht/LengthPartitioner.java |    2 +-
 .../org/apache/cassandra/hints/HintTest.java    |    2 +-
 .../schema/LegacySchemaMigratorTest.java        |    3 +
 70 files changed, 4484 insertions(+), 4041 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/43ddec31/CHANGES.txt
----------------------------------------------------------------------
diff --cc CHANGES.txt
index 051c9ad,b213260..8d368d7
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@@ -1,8 -1,5 +1,9 @@@
 +3.2
 + * Add transparent data encryption core classes (CASSANDRA-9945)
 +
 +
  3.0.0-rc1
+  * Improve MV schema representation (CASSANDRA-9921)
   * Add flag to enable/disable coordinator batchlog for MV writes (CASSANDRA-10230)
   * Update cqlsh COPY for new internal driver serialization interface (CASSANDRA-10318)
   * Give index implementations more control over rebuild operations (CASSANDRA-10312)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43ddec31/src/java/org/apache/cassandra/config/DatabaseDescriptor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/cassandra/blob/43ddec31/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------


[5/7] cassandra git commit: Improve MV schema representation

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
new file mode 100644
index 0000000..1a020ce
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/CreateViewStatement.java
@@ -0,0 +1,321 @@
+/*
+ * 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.statements;
+
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.google.common.collect.Iterables;
+
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.ColumnDefinition;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.cql3.CFName;
+import org.apache.cassandra.cql3.ColumnIdentifier;
+import org.apache.cassandra.cql3.selection.RawSelector;
+import org.apache.cassandra.cql3.selection.Selectable;
+import org.apache.cassandra.db.marshal.AbstractType;
+import org.apache.cassandra.db.marshal.ReversedType;
+import org.apache.cassandra.db.view.View;
+import org.apache.cassandra.exceptions.AlreadyExistsException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.TableParams;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.ClientWarn;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.thrift.ThriftValidation;
+import org.apache.cassandra.transport.Event;
+
+public class CreateViewStatement extends SchemaAlteringStatement
+{
+    private final CFName baseName;
+    private final List<RawSelector> selectClause;
+    private final List<ColumnIdentifier.Raw> notNullWhereClause;
+    private final List<ColumnIdentifier.Raw> partitionKeys;
+    private final List<ColumnIdentifier.Raw> clusteringKeys;
+    public final CFProperties properties = new CFProperties();
+    private final boolean ifNotExists;
+
+    public CreateViewStatement(CFName viewName,
+                               CFName baseName,
+                               List<RawSelector> selectClause,
+                               List<ColumnIdentifier.Raw> notNullWhereClause,
+                               List<ColumnIdentifier.Raw> partitionKeys,
+                               List<ColumnIdentifier.Raw> clusteringKeys,
+                               boolean ifNotExists)
+    {
+        super(viewName);
+        this.baseName = baseName;
+        this.selectClause = selectClause;
+        this.notNullWhereClause = notNullWhereClause;
+        this.partitionKeys = partitionKeys;
+        this.clusteringKeys = clusteringKeys;
+        this.ifNotExists = ifNotExists;
+    }
+
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        if (!baseName.hasKeyspace())
+            baseName.setKeyspace(keyspace(), true);
+        state.hasColumnFamilyAccess(keyspace(), baseName.getColumnFamily(), Permission.ALTER);
+    }
+
+    public void validate(ClientState state) throws RequestValidationException
+    {
+        // We do validation in announceMigration to reduce doubling up of work
+    }
+
+    private interface AddColumn {
+        void add(ColumnIdentifier identifier, AbstractType<?> type);
+    }
+
+    private void add(CFMetaData baseCfm, Iterable<ColumnIdentifier> columns, AddColumn adder)
+    {
+        for (ColumnIdentifier column : columns)
+        {
+            AbstractType<?> type = baseCfm.getColumnDefinition(column).type;
+            if (properties.definedOrdering.containsKey(column))
+            {
+                boolean desc = properties.definedOrdering.get(column);
+                if (!desc && type.isReversed())
+                {
+                    type = ((ReversedType)type).baseType;
+                }
+                else if (desc && !type.isReversed())
+                {
+                    type = ReversedType.getInstance(type);
+                }
+            }
+            adder.add(column, type);
+        }
+    }
+
+    public boolean announceMigration(boolean isLocalOnly) throws RequestValidationException
+    {
+        // We need to make sure that:
+        //  - primary key includes all columns in base table's primary key
+        //  - make sure that the select statement does not have anything other than columns
+        //    and their names match the base table's names
+        //  - make sure that primary key does not include any collections
+        //  - make sure there is no where clause in the select statement
+        //  - make sure there is not currently a table or view
+        //  - make sure baseTable gcGraceSeconds > 0
+
+        properties.validate();
+
+        if (properties.useCompactStorage)
+            throw new InvalidRequestException("Cannot use 'COMPACT STORAGE' when defining a materialized view");
+
+        // We enforce the keyspace because if the RF is different, the logic to wait for a
+        // specific replica would break
+        if (!baseName.getKeyspace().equals(keyspace()))
+            throw new InvalidRequestException("Cannot create a materialized view on a table in a separate keyspace");
+
+        CFMetaData cfm = ThriftValidation.validateColumnFamily(baseName.getKeyspace(), baseName.getColumnFamily());
+
+        if (cfm.isCounter())
+            throw new InvalidRequestException("Materialized views are not supported on counter tables");
+        if (cfm.isView())
+            throw new InvalidRequestException("Materialized views cannot be created against other materialized views");
+
+        if (cfm.params.gcGraceSeconds == 0)
+        {
+            throw new InvalidRequestException(String.format("Cannot create materialized view '%s' for base table " +
+                                                            "'%s' with gc_grace_seconds of 0, since this value is " +
+                                                            "used to TTL undelivered updates. Setting gc_grace_seconds" +
+                                                            " too low might cause undelivered updates to expire " +
+                                                            "before being replayed.", cfName.getColumnFamily(),
+                                                            baseName.getColumnFamily()));
+        }
+
+        Set<ColumnIdentifier> included = new HashSet<>();
+        for (RawSelector selector : selectClause)
+        {
+            Selectable.Raw selectable = selector.selectable;
+            if (selectable instanceof Selectable.WithFieldSelection.Raw)
+                throw new InvalidRequestException("Cannot select out a part of type when defining a materialized view");
+            if (selectable instanceof Selectable.WithFunction.Raw)
+                throw new InvalidRequestException("Cannot use function when defining a materialized view");
+            if (selectable instanceof Selectable.WritetimeOrTTL.Raw)
+                throw new InvalidRequestException("Cannot use function when defining a materialized view");
+            ColumnIdentifier identifier = (ColumnIdentifier) selectable.prepare(cfm);
+            if (selector.alias != null)
+                throw new InvalidRequestException(String.format("Cannot alias column '%s' as '%s' when defining a materialized view", identifier.toString(), selector.alias.toString()));
+
+            ColumnDefinition cdef = cfm.getColumnDefinition(identifier);
+
+            if (cdef == null)
+                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
+
+            if (cdef.isStatic())
+                ClientWarn.warn(String.format("Unable to include static column '%s' in Materialized View SELECT statement", identifier));
+            else
+                included.add(identifier);
+        }
+
+        Set<ColumnIdentifier.Raw> targetPrimaryKeys = new HashSet<>();
+        for (ColumnIdentifier.Raw identifier : Iterables.concat(partitionKeys, clusteringKeys))
+        {
+            if (!targetPrimaryKeys.add(identifier))
+                throw new InvalidRequestException("Duplicate entry found in PRIMARY KEY: "+identifier);
+
+            ColumnDefinition cdef = cfm.getColumnDefinition(identifier.prepare(cfm));
+
+            if (cdef == null)
+                throw new InvalidRequestException("Unknown column name detected in CREATE MATERIALIZED VIEW statement : "+identifier);
+
+            if (cfm.getColumnDefinition(identifier.prepare(cfm)).type.isMultiCell())
+                throw new InvalidRequestException(String.format("Cannot use MultiCell column '%s' in PRIMARY KEY of materialized view", identifier));
+
+            if (cdef.isStatic())
+                throw new InvalidRequestException(String.format("Cannot use Static column '%s' in PRIMARY KEY of materialized view", identifier));
+        }
+
+        Set<ColumnIdentifier> basePrimaryKeyCols = new HashSet<>();
+        for (ColumnDefinition definition : Iterables.concat(cfm.partitionKeyColumns(), cfm.clusteringColumns()))
+            basePrimaryKeyCols.add(definition.name);
+
+        List<ColumnIdentifier> targetClusteringColumns = new ArrayList<>();
+        List<ColumnIdentifier> targetPartitionKeys = new ArrayList<>();
+        Set<ColumnIdentifier> notNullColumns = new HashSet<>();
+        if (notNullWhereClause != null)
+        {
+            for (ColumnIdentifier.Raw raw : notNullWhereClause)
+            {
+                notNullColumns.add(raw.prepare(cfm));
+            }
+        }
+
+        // This is only used as an intermediate state; this is to catch whether multiple non-PK columns are used
+        boolean hasNonPKColumn = false;
+        for (ColumnIdentifier.Raw raw : partitionKeys)
+        {
+            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetPartitionKeys, notNullColumns);
+        }
+
+        for (ColumnIdentifier.Raw raw : clusteringKeys)
+        {
+            hasNonPKColumn = getColumnIdentifier(cfm, basePrimaryKeyCols, hasNonPKColumn, raw, targetClusteringColumns, notNullColumns);
+        }
+
+        // We need to include all of the primary key colums from the base table in order to make sure that we do not
+        // overwrite values in the view. We cannot support "collapsing" the base table into a smaller number of rows in
+        // the view because if we need to generate a tombstone, we have no way of knowing which value is currently being
+        // used in the view and whether or not to generate a tombstone. In order to not surprise our users, we require
+        // that they include all of the columns. We provide them with a list of all of the columns left to include.
+        boolean missingClusteringColumns = false;
+        StringBuilder columnNames = new StringBuilder();
+        List<ColumnIdentifier> includedColumns = new ArrayList<>();
+        for (ColumnDefinition def : cfm.allColumns())
+        {
+            ColumnIdentifier identifier = def.name;
+
+            if ((included.isEmpty() || included.contains(identifier))
+                && !targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier)
+                && !def.isStatic())
+            {
+                includedColumns.add(identifier);
+            }
+            if (!def.isPrimaryKeyColumn()) continue;
+
+            if (!targetClusteringColumns.contains(identifier) && !targetPartitionKeys.contains(identifier))
+            {
+                if (missingClusteringColumns)
+                    columnNames.append(',');
+                else
+                    missingClusteringColumns = true;
+                columnNames.append(identifier);
+            }
+        }
+        if (missingClusteringColumns)
+            throw new InvalidRequestException(String.format("Cannot create Materialized View %s without primary key columns from base %s (%s)",
+                                                            columnFamily(), baseName.getColumnFamily(), columnNames.toString()));
+
+        if (targetPartitionKeys.isEmpty())
+            throw new InvalidRequestException("Must select at least a column for a Materialized View");
+
+        if (targetClusteringColumns.isEmpty())
+            throw new InvalidRequestException("No columns are defined for Materialized View other than primary key");
+
+        CFMetaData.Builder cfmBuilder = CFMetaData.Builder.createView(keyspace(), columnFamily());
+        add(cfm, targetPartitionKeys, cfmBuilder::addPartitionKey);
+        add(cfm, targetClusteringColumns, cfmBuilder::addClusteringColumn);
+        add(cfm, includedColumns, cfmBuilder::addRegularColumn);
+        TableParams params = properties.properties.asNewTableParams();
+        CFMetaData viewCfm = cfmBuilder.build().params(params);
+        ViewDefinition definition = new ViewDefinition(keyspace(),
+                                                       columnFamily(),
+                                                       Schema.instance.getId(keyspace(), baseName.getColumnFamily()),
+                                                       included.isEmpty(),
+                                                       viewCfm);
+
+        try
+        {
+            MigrationManager.announceNewView(definition, isLocalOnly);
+        }
+        catch (AlreadyExistsException e)
+        {
+            if (ifNotExists)
+                return false;
+            throw e;
+        }
+
+        return true;
+    }
+
+    private static boolean getColumnIdentifier(CFMetaData cfm,
+                                               Set<ColumnIdentifier> basePK,
+                                               boolean hasNonPKColumn,
+                                               ColumnIdentifier.Raw raw,
+                                               List<ColumnIdentifier> columns,
+                                               Set<ColumnIdentifier> allowedPKColumns)
+    {
+        ColumnIdentifier identifier = raw.prepare(cfm);
+
+        boolean isPk = basePK.contains(identifier);
+        if (!isPk && hasNonPKColumn)
+        {
+            throw new InvalidRequestException(String.format("Cannot include more than one non-primary key column '%s' in materialized view partition key", identifier));
+        }
+
+        // We don't need to include the "IS NOT NULL" filter on a non-composite partition key
+        // because we will never allow a single partition key to be NULL
+        boolean isSinglePartitionKey = cfm.getColumnDefinition(identifier).isPartitionKey()
+                                       && cfm.partitionKeyColumns().size() == 1;
+        if (!allowedPKColumns.remove(identifier) && !isSinglePartitionKey)
+        {
+            throw new InvalidRequestException(String.format("Primary key column '%s' is required to be filtered by 'IS NOT NULL'", identifier));
+        }
+
+        columns.add(identifier);
+        return !isPk;
+    }
+
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(Event.SchemaChange.Change.CREATED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/DropMaterializedViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropMaterializedViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropMaterializedViewStatement.java
deleted file mode 100644
index 8adba45..0000000
--- a/src/java/org/apache/cassandra/cql3/statements/DropMaterializedViewStatement.java
+++ /dev/null
@@ -1,92 +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.cql3.statements;
-
-import org.apache.cassandra.auth.Permission;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.CFName;
-import org.apache.cassandra.db.view.MaterializedView;
-import org.apache.cassandra.exceptions.ConfigurationException;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
-import org.apache.cassandra.service.ClientState;
-import org.apache.cassandra.service.MigrationManager;
-import org.apache.cassandra.transport.Event;
-
-public class DropMaterializedViewStatement extends SchemaAlteringStatement
-{
-    public final boolean ifExists;
-
-    public DropMaterializedViewStatement(CFName cf, boolean ifExists)
-    {
-        super(cf);
-        this.ifExists = ifExists;
-    }
-
-    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
-    {
-        CFMetaData baseTable = MaterializedView.findBaseTable(keyspace(), columnFamily());
-        if (baseTable != null)
-            state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.ALTER);
-    }
-
-    public void validate(ClientState state)
-    {
-        // validated in findIndexedCf()
-    }
-
-    public Event.SchemaChange changeEvent()
-    {
-        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
-    }
-
-    public boolean announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
-    {
-        try
-        {
-            CFMetaData viewCfm = Schema.instance.getCFMetaData(keyspace(), columnFamily());
-            if (viewCfm == null)
-                throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", columnFamily(), keyspace()));
-            if (!viewCfm.isMaterializedView())
-                throw new ConfigurationException(String.format("Cannot drop non materialized view '%s' in keyspace '%s'", columnFamily(), keyspace()));
-
-            CFMetaData baseCfm = MaterializedView.findBaseTable(keyspace(), columnFamily());
-            if (baseCfm == null)
-            {
-                if (ifExists)
-                    throw new ConfigurationException(String.format("Cannot drop materialized view '%s' in keyspace '%s' without base CF.", columnFamily(), keyspace()));
-                else
-                    throw new InvalidRequestException(String.format("View '%s' could not be found in any of the tables of keyspace '%s'", cfName, keyspace()));
-            }
-
-            CFMetaData updatedCfm = baseCfm.copy();
-            updatedCfm.materializedViews(updatedCfm.getMaterializedViews().without(columnFamily()));
-            MigrationManager.announceColumnFamilyUpdate(updatedCfm, false, isLocalOnly);
-            MigrationManager.announceColumnFamilyDrop(keyspace(), columnFamily(), isLocalOnly);
-            return true;
-        }
-        catch (ConfigurationException e)
-        {
-            if (ifExists)
-                return false;
-            throw e;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
index 35dc947..14d89d9 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTableStatement.java
@@ -19,12 +19,13 @@ package org.apache.cassandra.cql3.statements;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.MaterializedViewDefinition;
+import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.cql3.CFName;
 import org.apache.cassandra.exceptions.ConfigurationException;
 import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.schema.KeyspaceMetadata;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.MigrationManager;
 import org.apache.cassandra.transport.Event;
@@ -61,20 +62,24 @@ public class DropTableStatement extends SchemaAlteringStatement
     {
         try
         {
-            CFMetaData cfm = Schema.instance.getCFMetaData(keyspace(), columnFamily());
+            KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace());
+            CFMetaData cfm = ksm.tables.getNullable(columnFamily());
             if (cfm != null)
             {
-                if (cfm.isMaterializedView())
+                if (cfm.isView())
                     throw new InvalidRequestException("Cannot use DROP TABLE on Materialized View");
 
                 boolean rejectDrop = false;
                 StringBuilder messageBuilder = new StringBuilder();
-                for (MaterializedViewDefinition def : cfm.getMaterializedViews())
+                for (ViewDefinition def : ksm.views)
                 {
-                    if (rejectDrop)
-                        messageBuilder.append(',');
-                    rejectDrop = true;
-                    messageBuilder.append(def.viewName);
+                    if (def.baseTableId.equals(cfm.cfId))
+                    {
+                        if (rejectDrop)
+                            messageBuilder.append(',');
+                        rejectDrop = true;
+                        messageBuilder.append(def.viewName);
+                    }
                 }
                 if (rejectDrop)
                 {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
index 75f6200..74c8c36 100644
--- a/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/DropTypeStatement.java
@@ -87,7 +87,7 @@ public class DropTypeStatement extends SchemaAlteringStatement
             if (!ut.name.equals(name.getUserTypeName()) && isUsedBy(ut))
                 throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by user type %s", name, ut.asCQL3Type()));
 
-        for (CFMetaData cfm : ksm.tables)
+        for (CFMetaData cfm : ksm.tablesAndViews())
             for (ColumnDefinition def : cfm.allColumns())
                 if (isUsedBy(def.type))
                     throw new InvalidRequestException(String.format("Cannot drop user type %s as it is still used by table %s.%s", name, cfm.ksName, cfm.cfName));

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
new file mode 100644
index 0000000..f2be370
--- /dev/null
+++ b/src/java/org/apache/cassandra/cql3/statements/DropViewStatement.java
@@ -0,0 +1,93 @@
+/*
+ * 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.statements;
+
+import org.apache.cassandra.auth.Permission;
+import org.apache.cassandra.config.CFMetaData;
+import org.apache.cassandra.config.Schema;
+import org.apache.cassandra.config.ViewDefinition;
+import org.apache.cassandra.cql3.CFName;
+import org.apache.cassandra.db.view.View;
+import org.apache.cassandra.exceptions.ConfigurationException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.service.ClientState;
+import org.apache.cassandra.service.MigrationManager;
+import org.apache.cassandra.transport.Event;
+
+public class DropViewStatement extends SchemaAlteringStatement
+{
+    public final boolean ifExists;
+
+    public DropViewStatement(CFName cf, boolean ifExists)
+    {
+        super(cf);
+        this.ifExists = ifExists;
+    }
+
+    public void checkAccess(ClientState state) throws UnauthorizedException, InvalidRequestException
+    {
+        CFMetaData baseTable = View.findBaseTable(keyspace(), columnFamily());
+        if (baseTable != null)
+            state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.ALTER);
+    }
+
+    public void validate(ClientState state)
+    {
+        // validated in findIndexedCf()
+    }
+
+    public Event.SchemaChange changeEvent()
+    {
+        return new Event.SchemaChange(Event.SchemaChange.Change.DROPPED, Event.SchemaChange.Target.TABLE, keyspace(), columnFamily());
+    }
+
+    public boolean announceMigration(boolean isLocalOnly) throws InvalidRequestException, ConfigurationException
+    {
+        try
+        {
+//            ViewDefinition view = Schema.instance.getViewDefinition(keyspace(), columnFamily());
+//            if (view == null)
+//            {
+//                if (Schema.instance.getCFMetaData(keyspace(), columnFamily()) != null)
+//                    throw new ConfigurationException(String.format("Cannot drop table '%s' in keyspace '%s'.", columnFamily(), keyspace()));
+//
+//                throw new ConfigurationException(String.format("Cannot drop non existing materialized view '%s' in keyspace '%s'.", columnFamily(), keyspace()));
+//            }
+//
+//            CFMetaData baseCfm = Schema.instance.getCFMetaData(view.baseTableId);
+//            if (baseCfm == null)
+//            {
+//                if (ifExists)
+//                    throw new ConfigurationException(String.format("Cannot drop materialized view '%s' in keyspace '%s' without base CF.", columnFamily(), keyspace()));
+//                else
+//                    throw new InvalidRequestException(String.format("View '%s' could not be found in any of the tables of keyspace '%s'", cfName, keyspace()));
+//            }
+
+            MigrationManager.announceViewDrop(keyspace(), columnFamily(), isLocalOnly);
+            return true;
+        }
+        catch (ConfigurationException e)
+        {
+            if (ifExists)
+                return false;
+            throw e;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
index 3855b6a..a04af4c 100644
--- a/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/ModificationStatement.java
@@ -21,13 +21,13 @@ import java.nio.ByteBuffer;
 import java.util.*;
 
 import com.google.common.collect.Iterables;
-
-import static org.apache.cassandra.cql3.statements.RequestValidations.checkNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import org.apache.cassandra.auth.Permission;
 import org.apache.cassandra.config.CFMetaData;
 import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.MaterializedViewDefinition;
+import org.apache.cassandra.config.ViewDefinition;
 import org.apache.cassandra.cql3.*;
 import org.apache.cassandra.cql3.ColumnIdentifier.Raw;
 import org.apache.cassandra.cql3.functions.Function;
@@ -38,10 +38,8 @@ import org.apache.cassandra.db.filter.*;
 import org.apache.cassandra.db.marshal.BooleanType;
 import org.apache.cassandra.db.partitions.*;
 import org.apache.cassandra.db.rows.RowIterator;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.exceptions.RequestExecutionException;
-import org.apache.cassandra.exceptions.RequestValidationException;
-import org.apache.cassandra.exceptions.UnauthorizedException;
+import org.apache.cassandra.db.view.View;
+import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.service.ClientState;
 import org.apache.cassandra.service.QueryState;
 import org.apache.cassandra.service.StorageProxy;
@@ -52,9 +50,9 @@ import org.apache.cassandra.triggers.TriggerExecutor;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.Pair;
 import org.apache.cassandra.utils.UUIDGen;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkFalse;
+import static org.apache.cassandra.cql3.statements.RequestValidations.checkNull;
 import static org.apache.cassandra.cql3.statements.RequestValidations.checkNotNull;
 
 /*
@@ -170,14 +168,9 @@ public abstract class ModificationStatement implements CQLStatement
         return cfm.isCounter();
     }
 
-    public boolean isMaterializedView()
-    {
-        return cfm.isMaterializedView();
-    }
-
-    public boolean hasMaterializedViews()
+    public boolean isView()
     {
-        return !cfm.getMaterializedViews().isEmpty();
+        return cfm.isView();
     }
 
     public long getTimestamp(long now, QueryOptions options) throws InvalidRequestException
@@ -203,13 +196,16 @@ public abstract class ModificationStatement implements CQLStatement
         if (hasConditions())
             state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
 
-        // MV updates need to get the current state from the table, and might update the materialized views
+        // MV updates need to get the current state from the table, and might update the views
         // Require Permission.SELECT on the base table, and Permission.MODIFY on the views
-        if (hasMaterializedViews())
+        Iterator<ViewDefinition> views = View.findAll(keyspace(), columnFamily()).iterator();
+        if (views.hasNext())
         {
             state.hasColumnFamilyAccess(keyspace(), columnFamily(), Permission.SELECT);
-            for (MaterializedViewDefinition view : cfm.getMaterializedViews())
-                state.hasColumnFamilyAccess(keyspace(), view.viewName, Permission.MODIFY);
+            do
+            {
+                state.hasColumnFamilyAccess(keyspace(), views.next().viewName, Permission.MODIFY);
+            } while (views.hasNext());
         }
 
         for (Function function : getFunctions())
@@ -221,7 +217,7 @@ public abstract class ModificationStatement implements CQLStatement
         checkFalse(hasConditions() && attrs.isTimestampSet(), "Cannot provide custom timestamp for conditional updates");
         checkFalse(isCounter() && attrs.isTimestampSet(), "Cannot provide custom timestamp for counter updates");
         checkFalse(isCounter() && attrs.isTimeToLiveSet(), "Cannot provide custom TTL for counter updates");
-        checkFalse(isMaterializedView(), "Cannot directly modify a materialized view");
+        checkFalse(isView(), "Cannot directly modify a materialized view");
     }
 
     public PartitionColumns updatedColumns()

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
index 7ad6c09..18e402b 100644
--- a/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/SelectStatement.java
@@ -44,7 +44,7 @@ import org.apache.cassandra.db.partitions.PartitionIterator;
 import org.apache.cassandra.db.rows.ComplexColumnData;
 import org.apache.cassandra.db.rows.Row;
 import org.apache.cassandra.db.rows.RowIterator;
-import org.apache.cassandra.db.view.MaterializedView;
+import org.apache.cassandra.db.view.View;
 import org.apache.cassandra.dht.AbstractBounds;
 import org.apache.cassandra.exceptions.*;
 import org.apache.cassandra.index.Index;
@@ -175,9 +175,9 @@ public class SelectStatement implements CQLStatement
 
     public void checkAccess(ClientState state) throws InvalidRequestException, UnauthorizedException
     {
-        if (cfm.isMaterializedView())
+        if (cfm.isView())
         {
-            CFMetaData baseTable = MaterializedView.findBaseTable(keyspace(), columnFamily());
+            CFMetaData baseTable = View.findBaseTable(keyspace(), columnFamily());
             if (baseTable != null)
                 state.hasColumnFamilyAccess(keyspace(), baseTable.cfName, Permission.SELECT);
         }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
index 5dd306a..66b3da0 100644
--- a/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
+++ b/src/java/org/apache/cassandra/cql3/statements/TruncateStatement.java
@@ -65,7 +65,7 @@ public class TruncateStatement extends CFStatement implements CQLStatement
         try
         {
             CFMetaData metaData = Schema.instance.getCFMetaData(keyspace(), columnFamily());
-            if (metaData.isMaterializedView())
+            if (metaData.isView())
                 throw new InvalidRequestException("Cannot TRUNCATE materialized view directly; must truncate base table instead");
 
             StorageProxy.truncateBlocking(keyspace(), columnFamily());

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
index c7d8926..0d6d801 100644
--- a/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
+++ b/src/java/org/apache/cassandra/db/ColumnFamilyStore.java
@@ -48,11 +48,11 @@ import org.apache.cassandra.db.commitlog.ReplayPosition;
 import org.apache.cassandra.db.compaction.*;
 import org.apache.cassandra.db.filter.ClusteringIndexFilter;
 import org.apache.cassandra.db.filter.DataLimits;
+import org.apache.cassandra.db.view.ViewManager;
 import org.apache.cassandra.db.lifecycle.*;
 import org.apache.cassandra.db.partitions.CachedPartition;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.CellPath;
-import org.apache.cassandra.db.view.MaterializedViewManager;
 import org.apache.cassandra.dht.*;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.exceptions.ConfigurationException;
@@ -193,7 +193,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     private final AtomicInteger fileIndexGenerator = new AtomicInteger(0);
 
     public final SecondaryIndexManager indexManager;
-    public final MaterializedViewManager materializedViewManager;
+    public final ViewManager.ForStore viewManager;
 
     /* These are locally held copies to be changed from the config during runtime */
     private volatile DefaultInteger minCompactionThreshold;
@@ -231,7 +231,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
         indexManager.reload();
 
-        materializedViewManager.reload();
         // If the CF comparator has changed, we need to change the memtable,
         // because the old one still aliases the previous comparator.
         if (data.getView().getCurrentMemtable().initialComparator != metadata.comparator)
@@ -377,7 +376,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         this.maxCompactionThreshold = new DefaultInteger(metadata.params.compaction.maxCompactionThreshold());
         this.directories = directories;
         this.indexManager = new SecondaryIndexManager(this);
-        this.materializedViewManager = new MaterializedViewManager(this);
+        this.viewManager = keyspace.viewManager.forTable(metadata.cfId);
         this.metric = new TableMetrics(this);
         fileIndexGenerator.set(generation);
         sampleLatencyNanos = DatabaseDescriptor.getReadRpcTimeout() / 2;
@@ -513,7 +512,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         data.dropSSTables();
         LifecycleTransaction.waitForDeletions();
         indexManager.invalidateAllIndexesBlocking();
-        materializedViewManager.invalidate();
 
         invalidateCaches();
     }
@@ -633,8 +631,6 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
     // must be called after all sstables are loaded since row cache merges all row versions
     public void init()
     {
-        materializedViewManager.init();
-
         if (!isRowCacheEnabled())
             return;
 
@@ -1912,7 +1908,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             // flush the CF being truncated before forcing the new segment
             forceBlockingFlush();
 
-            materializedViewManager.forceBlockingFlush();
+            viewManager.forceBlockingFlush();
 
             // sleep a little to make sure that our truncatedAt comes after any sstable
             // that was part of the flushed we forced; otherwise on a tie, it won't get deleted.
@@ -1921,7 +1917,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
         else
         {
             dumpMemtable();
-            materializedViewManager.dumpMemtables();
+            viewManager.dumpMemtables();
         }
 
         Runnable truncateRunnable = new Runnable()
@@ -1940,7 +1936,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
 
                 indexManager.truncateAllIndexesBlocking(truncatedAt);
 
-                materializedViewManager.truncateBlocking(truncatedAt);
+                viewManager.truncateBlocking(truncatedAt);
 
                 SystemKeyspace.saveTruncationRecord(ColumnFamilyStore.this, truncatedAt, replayAfter);
                 logger.debug("cleaning out row cache");
@@ -1974,7 +1970,7 @@ public class ColumnFamilyStore implements ColumnFamilyStoreMBean
             logger.debug("Cancelling in-progress compactions for {}", metadata.cfName);
 
             Iterable<ColumnFamilyStore> selfWithAuxiliaryCfs = interruptViews
-                                                               ? Iterables.concat(concatWithIndexes(), materializedViewManager.allViewsCfs())
+                                                               ? Iterables.concat(concatWithIndexes(), viewManager.allViewsCfs())
                                                                : concatWithIndexes();
 
             for (ColumnFamilyStore cfs : selfWithAuxiliaryCfs)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/Keyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/Keyspace.java b/src/java/org/apache/cassandra/db/Keyspace.java
index 4661bae..1169b45 100644
--- a/src/java/org/apache/cassandra/db/Keyspace.java
+++ b/src/java/org/apache/cassandra/db/Keyspace.java
@@ -40,7 +40,7 @@ import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.partitions.PartitionUpdate;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-import org.apache.cassandra.db.view.MaterializedViewManager;
+import org.apache.cassandra.db.view.ViewManager;
 import org.apache.cassandra.exceptions.WriteTimeoutException;
 import org.apache.cassandra.index.Index;
 import org.apache.cassandra.index.SecondaryIndexManager;
@@ -81,12 +81,13 @@ public class Keyspace
     private volatile KeyspaceMetadata metadata;
 
     //OpOrder is defined globally since we need to order writes across
-    //Keyspaces in the case of MaterializedViews (batchlog of MV mutations)
+    //Keyspaces in the case of Views (batchlog of view mutations)
     public static final OpOrder writeOrder = new OpOrder();
 
     /* ColumnFamilyStore per column family */
     private final ConcurrentMap<UUID, ColumnFamilyStore> columnFamilyStores = new ConcurrentHashMap<>();
     private volatile AbstractReplicationStrategy replicationStrategy;
+    public final ViewManager viewManager;
 
     public static final Function<String,Keyspace> keyspaceTransformer = new Function<String, Keyspace>()
     {
@@ -305,11 +306,13 @@ public class Keyspace
         createReplicationStrategy(metadata);
 
         this.metric = new KeyspaceMetrics(this);
-        for (CFMetaData cfm : metadata.tables)
+        this.viewManager = new ViewManager(this);
+        for (CFMetaData cfm : metadata.tablesAndViews())
         {
             logger.debug("Initializing {}.{}", getName(), cfm.cfName);
             initCf(cfm.cfId, cfm.cfName, loadSSTables);
         }
+        this.viewManager.reload();
     }
 
     private Keyspace(KeyspaceMetadata metadata)
@@ -317,6 +320,7 @@ public class Keyspace
         this.metadata = metadata;
         createReplicationStrategy(metadata);
         this.metric = new KeyspaceMetrics(this);
+        this.viewManager = new ViewManager(this);
     }
 
     public static Keyspace mockKS(KeyspaceMetadata metadata)
@@ -418,11 +422,11 @@ public class Keyspace
             throw new RuntimeException("Testing write failures");
 
         Lock lock = null;
-        boolean requiresViewUpdate = updateIndexes && MaterializedViewManager.updatesAffectView(Collections.singleton(mutation), false);
+        boolean requiresViewUpdate = updateIndexes && viewManager.updatesAffectView(Collections.singleton(mutation), false);
 
         if (requiresViewUpdate)
         {
-            lock = MaterializedViewManager.acquireLockFor(mutation.key().getKey());
+            lock = ViewManager.acquireLockFor(mutation.key().getKey());
 
             if (lock == null)
             {
@@ -430,11 +434,11 @@ public class Keyspace
                 {
                     logger.debug("Could not acquire lock for {}", ByteBufferUtil.bytesToHex(mutation.key().getKey()));
                     Tracing.trace("Could not acquire MV lock");
-                    throw new WriteTimeoutException(WriteType.MATERIALIZED_VIEW, ConsistencyLevel.LOCAL_ONE, 0, 1);
+                    throw new WriteTimeoutException(WriteType.VIEW, ConsistencyLevel.LOCAL_ONE, 0, 1);
                 }
                 else
                 {
-                    //This MV update can't happen right now. so rather than keep this thread busy
+                    //This view update can't happen right now. so rather than keep this thread busy
                     // we will re-apply ourself to the queue and try again later
                     StageManager.getStage(Stage.MUTATION).execute(() -> {
                         if (writeCommitLog)
@@ -472,7 +476,7 @@ public class Keyspace
                     try
                     {
                         Tracing.trace("Creating materialized view mutations from base table replica");
-                        cfs.materializedViewManager.pushViewReplicaUpdates(upd, !isClReplay);
+                        viewManager.pushViewReplicaUpdates(upd, !isClReplay);
                     }
                     catch (Throwable t)
                     {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/SystemKeyspace.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/SystemKeyspace.java b/src/java/org/apache/cassandra/db/SystemKeyspace.java
index d54ee8b..6b2585e 100644
--- a/src/java/org/apache/cassandra/db/SystemKeyspace.java
+++ b/src/java/org/apache/cassandra/db/SystemKeyspace.java
@@ -95,8 +95,8 @@ public final class SystemKeyspace
     public static final String SSTABLE_ACTIVITY = "sstable_activity";
     public static final String SIZE_ESTIMATES = "size_estimates";
     public static final String AVAILABLE_RANGES = "available_ranges";
-    public static final String MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS = "materialized_views_builds_in_progress";
-    public static final String BUILT_MATERIALIZED_VIEWS = "built_materialized_views";
+    public static final String VIEWS_BUILDS_IN_PROGRESS = "views_builds_in_progress";
+    public static final String BUILT_VIEWS = "built_views";
 
     @Deprecated public static final String LEGACY_HINTS = "hints";
     @Deprecated public static final String LEGACY_BATCHLOG = "batchlog";
@@ -246,9 +246,9 @@ public final class SystemKeyspace
                 + "ranges set<blob>,"
                 + "PRIMARY KEY ((keyspace_name)))");
 
-    private static final CFMetaData MaterializedViewsBuildsInProgress =
-        compile(MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS,
-                "materialized views builds current progress",
+    private static final CFMetaData ViewsBuildsInProgress =
+        compile(VIEWS_BUILDS_IN_PROGRESS,
+                "views builds current progress",
                 "CREATE TABLE %s ("
                 + "keyspace_name text,"
                 + "view_name text,"
@@ -256,9 +256,9 @@ public final class SystemKeyspace
                 + "generation_number int,"
                 + "PRIMARY KEY ((keyspace_name), view_name))");
 
-    private static final CFMetaData BuiltMaterializedViews =
-        compile(BUILT_MATERIALIZED_VIEWS,
-                "built materialized views",
+    private static final CFMetaData BuiltViews =
+        compile(BUILT_VIEWS,
+                "built views",
                 "CREATE TABLE %s ("
                 + "keyspace_name text,"
                 + "view_name text,"
@@ -414,7 +414,7 @@ public final class SystemKeyspace
 
     public static KeyspaceMetadata metadata()
     {
-        return KeyspaceMetadata.create(NAME, KeyspaceParams.local(), tables(), Types.none(), functions());
+        return KeyspaceMetadata.create(NAME, KeyspaceParams.local(), tables(), Views.none(), Types.none(), functions());
     }
 
     private static Tables tables()
@@ -430,8 +430,8 @@ public final class SystemKeyspace
                          SSTableActivity,
                          SizeEstimates,
                          AvailableRanges,
-                         MaterializedViewsBuildsInProgress,
-                         BuiltMaterializedViews,
+                         ViewsBuildsInProgress,
+                         BuiltViews,
                          LegacyHints,
                          LegacyBatchlog,
                          LegacyKeyspaces,
@@ -531,61 +531,61 @@ public final class SystemKeyspace
     public static boolean isViewBuilt(String keyspaceName, String viewName)
     {
         String req = "SELECT view_name FROM %s.\"%s\" WHERE keyspace_name=? AND view_name=?";
-        UntypedResultSet result = executeInternal(String.format(req, NAME, BUILT_MATERIALIZED_VIEWS), keyspaceName, viewName);
+        UntypedResultSet result = executeInternal(String.format(req, NAME, BUILT_VIEWS), keyspaceName, viewName);
         return !result.isEmpty();
     }
 
-    public static void setMaterializedViewBuilt(String keyspaceName, String viewName)
+    public static void setViewBuilt(String keyspaceName, String viewName)
     {
         String req = "INSERT INTO %s.\"%s\" (keyspace_name, view_name) VALUES (?, ?)";
-        executeInternal(String.format(req, NAME, BUILT_MATERIALIZED_VIEWS), keyspaceName, viewName);
-        forceBlockingFlush(BUILT_MATERIALIZED_VIEWS);
+        executeInternal(String.format(req, NAME, BUILT_VIEWS), keyspaceName, viewName);
+        forceBlockingFlush(BUILT_VIEWS);
     }
 
 
-    public static void setMaterializedViewRemoved(String keyspaceName, String viewName)
+    public static void setViewRemoved(String keyspaceName, String viewName)
     {
         String buildReq = "DELETE FROM %S.%s WHERE keyspace_name = ? AND view_name = ?";
-        executeInternal(String.format(buildReq, NAME, MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS), keyspaceName, viewName);
-        forceBlockingFlush(MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS);
+        executeInternal(String.format(buildReq, NAME, VIEWS_BUILDS_IN_PROGRESS), keyspaceName, viewName);
+        forceBlockingFlush(VIEWS_BUILDS_IN_PROGRESS);
 
         String builtReq = "DELETE FROM %s.\"%s\" WHERE keyspace_name = ? AND view_name = ?";
-        executeInternal(String.format(builtReq, NAME, BUILT_MATERIALIZED_VIEWS), keyspaceName, viewName);
-        forceBlockingFlush(BUILT_MATERIALIZED_VIEWS);
+        executeInternal(String.format(builtReq, NAME, BUILT_VIEWS), keyspaceName, viewName);
+        forceBlockingFlush(BUILT_VIEWS);
     }
 
-    public static void beginMaterializedViewBuild(String ksname, String viewName, int generationNumber)
+    public static void beginViewBuild(String ksname, String viewName, int generationNumber)
     {
-        executeInternal(String.format("INSERT INTO system.%s (keyspace_name, view_name, generation_number) VALUES (?, ?, ?)", MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS),
+        executeInternal(String.format("INSERT INTO system.%s (keyspace_name, view_name, generation_number) VALUES (?, ?, ?)", VIEWS_BUILDS_IN_PROGRESS),
                         ksname,
                         viewName,
                         generationNumber);
     }
 
-    public static void finishMaterializedViewBuildStatus(String ksname, String viewName)
+    public static void finishViewBuildStatus(String ksname, String viewName)
     {
         // We flush the view built first, because if we fail now, we'll restart at the last place we checkpointed
-        // materialized view build.
+        // view build.
         // If we flush the delete first, we'll have to restart from the beginning.
-        // Also, if the build succeeded, but the materialized view build failed, we will be able to skip the
-        // materialized view build check next boot.
-        setMaterializedViewBuilt(ksname, viewName);
-        forceBlockingFlush(BUILT_MATERIALIZED_VIEWS);
-        executeInternal(String.format("DELETE FROM system.%s WHERE keyspace_name = ? AND view_name = ?", MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS), ksname, viewName);
-        forceBlockingFlush(MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS);
+        // Also, if the build succeeded, but the view build failed, we will be able to skip the view build check
+        // next boot.
+        setViewBuilt(ksname, viewName);
+        forceBlockingFlush(BUILT_VIEWS);
+        executeInternal(String.format("DELETE FROM system.%s WHERE keyspace_name = ? AND view_name = ?", VIEWS_BUILDS_IN_PROGRESS), ksname, viewName);
+        forceBlockingFlush(VIEWS_BUILDS_IN_PROGRESS);
     }
 
-    public static void updateMaterializedViewBuildStatus(String ksname, String viewName, Token token)
+    public static void updateViewBuildStatus(String ksname, String viewName, Token token)
     {
         String req = "INSERT INTO system.%s (keyspace_name, view_name, last_token) VALUES (?, ?, ?)";
-        Token.TokenFactory factory = MaterializedViewsBuildsInProgress.partitioner.getTokenFactory();
-        executeInternal(String.format(req, MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS), ksname, viewName, factory.toString(token));
+        Token.TokenFactory factory = ViewsBuildsInProgress.partitioner.getTokenFactory();
+        executeInternal(String.format(req, VIEWS_BUILDS_IN_PROGRESS), ksname, viewName, factory.toString(token));
     }
 
-    public static Pair<Integer, Token> getMaterializedViewBuildStatus(String ksname, String viewName)
+    public static Pair<Integer, Token> getViewBuildStatus(String ksname, String viewName)
     {
         String req = "SELECT generation_number, last_token FROM system.%s WHERE keyspace_name = ? AND view_name = ?";
-        UntypedResultSet queryResultSet = executeInternal(String.format(req, MATERIALIZED_VIEWS_BUILDS_IN_PROGRESS), ksname, viewName);
+        UntypedResultSet queryResultSet = executeInternal(String.format(req, VIEWS_BUILDS_IN_PROGRESS), ksname, viewName);
         if (queryResultSet == null || queryResultSet.isEmpty())
             return null;
 
@@ -597,7 +597,7 @@ public final class SystemKeyspace
             generation = row.getInt("generation_number");
         if (row.has("last_key"))
         {
-            Token.TokenFactory factory = MaterializedViewsBuildsInProgress.partitioner.getTokenFactory();
+            Token.TokenFactory factory = ViewsBuildsInProgress.partitioner.getTokenFactory();
             lastKey = factory.fromString(row.getString("last_key"));
         }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/WriteType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/WriteType.java b/src/java/org/apache/cassandra/db/WriteType.java
index 20fb6a9..fdbe97d 100644
--- a/src/java/org/apache/cassandra/db/WriteType.java
+++ b/src/java/org/apache/cassandra/db/WriteType.java
@@ -25,5 +25,5 @@ public enum WriteType
     COUNTER,
     BATCH_LOG,
     CAS,
-    MATERIALIZED_VIEW;
+    VIEW;
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
index 5207f49..75d50e7 100644
--- a/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
+++ b/src/java/org/apache/cassandra/db/compaction/CompactionManager.java
@@ -45,7 +45,7 @@ import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.lifecycle.SSTableSet;
 import org.apache.cassandra.db.lifecycle.View;
 import org.apache.cassandra.db.rows.UnfilteredRowIterator;
-import org.apache.cassandra.db.view.MaterializedViewBuilder;
+import org.apache.cassandra.db.view.ViewBuilder;
 import org.apache.cassandra.dht.Bounds;
 import org.apache.cassandra.dht.Range;
 import org.apache.cassandra.dht.Token;
@@ -1334,7 +1334,7 @@ public class CompactionManager implements CompactionManagerMBean
         }
     }
 
-    public Future<?> submitMaterializedViewBuilder(final MaterializedViewBuilder builder)
+    public Future<?> submitViewBuilder(final ViewBuilder builder)
     {
         Runnable runnable = new Runnable()
         {

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/compaction/OperationType.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/compaction/OperationType.java b/src/java/org/apache/cassandra/db/compaction/OperationType.java
index f8f016c..a69622b 100644
--- a/src/java/org/apache/cassandra/db/compaction/OperationType.java
+++ b/src/java/org/apache/cassandra/db/compaction/OperationType.java
@@ -36,7 +36,7 @@ public enum OperationType
     FLUSH("Flush"),
     STREAM("Stream"),
     WRITE("Write"),
-    VIEW_BUILD("Materialized view build");
+    VIEW_BUILD("View build");
 
     public final String type;
     public final String fileName;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/src/java/org/apache/cassandra/db/view/MaterializedView.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/db/view/MaterializedView.java b/src/java/org/apache/cassandra/db/view/MaterializedView.java
deleted file mode 100644
index 52034cc..0000000
--- a/src/java/org/apache/cassandra/db/view/MaterializedView.java
+++ /dev/null
@@ -1,749 +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.view;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Set;
-
-import javax.annotation.Nullable;
-
-import com.google.common.collect.Iterables;
-
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.MaterializedViewDefinition;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.cql3.ColumnIdentifier;
-import org.apache.cassandra.cql3.statements.CFProperties;
-import org.apache.cassandra.db.AbstractReadCommandBuilder.SinglePartitionSliceBuilder;
-import org.apache.cassandra.db.CBuilder;
-import org.apache.cassandra.db.Clustering;
-import org.apache.cassandra.db.ColumnFamilyStore;
-import org.apache.cassandra.db.DecoratedKey;
-import org.apache.cassandra.db.DeletionInfo;
-import org.apache.cassandra.db.DeletionTime;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.LivenessInfo;
-import org.apache.cassandra.db.Mutation;
-import org.apache.cassandra.db.RangeTombstone;
-import org.apache.cassandra.db.ReadCommand;
-import org.apache.cassandra.db.ReadOrderGroup;
-import org.apache.cassandra.db.SinglePartitionReadCommand;
-import org.apache.cassandra.db.Slice;
-import org.apache.cassandra.db.compaction.CompactionManager;
-import org.apache.cassandra.db.partitions.AbstractBTreePartition;
-import org.apache.cassandra.db.partitions.PartitionIterator;
-import org.apache.cassandra.db.partitions.PartitionUpdate;
-import org.apache.cassandra.db.rows.BTreeRow;
-import org.apache.cassandra.db.rows.Cell;
-import org.apache.cassandra.db.rows.ColumnData;
-import org.apache.cassandra.db.rows.ComplexColumnData;
-import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.db.rows.RowIterator;
-import org.apache.cassandra.schema.KeyspaceMetadata;
-import org.apache.cassandra.service.pager.QueryPager;
-
-/**
- * A Materialized View copies data from a base table into a view table which can be queried independently from the
- * base. Every update which targets the base table must be fed through the {@link MaterializedViewManager} to ensure
- * that if a view needs to be updated, the updates are properly created and fed into the view.
- *
- * This class does the job of translating the base row to the view row.
- *
- * It handles reading existing state and figuring out what tombstones need to be generated.
- *
- * createMutations below is the "main method"
- *
- */
-public class MaterializedView
-{
-    /**
-     * The columns should all be updated together, so we use this object as group.
-     */
-    private static class MVColumns
-    {
-        //These are the base column definitions in terms of the *views* partitioning.
-        //Meaning we can see (for example) the partition key of the view contains a clustering key
-        //from the base table.
-        public final List<ColumnDefinition> partitionDefs;
-        public final List<ColumnDefinition> primaryKeyDefs;
-        public final List<ColumnDefinition> baseComplexColumns;
-
-        private MVColumns(List<ColumnDefinition> partitionDefs, List<ColumnDefinition> primaryKeyDefs, List<ColumnDefinition> baseComplexColumns)
-        {
-            this.partitionDefs = partitionDefs;
-            this.primaryKeyDefs = primaryKeyDefs;
-            this.baseComplexColumns = baseComplexColumns;
-        }
-    }
-
-    public final String name;
-
-    private final ColumnFamilyStore baseCfs;
-    private ColumnFamilyStore _viewCfs = null;
-
-    private MVColumns columns;
-
-    private final boolean viewHasAllPrimaryKeys;
-    private final boolean includeAll;
-    private MaterializedViewBuilder builder;
-
-    public MaterializedView(MaterializedViewDefinition definition,
-                            ColumnFamilyStore baseCfs)
-    {
-        this.baseCfs = baseCfs;
-
-        name = definition.viewName;
-        includeAll = definition.includeAll;
-
-        viewHasAllPrimaryKeys = updateDefinition(definition);
-    }
-
-    /**
-     * Lazily fetch the CFS instance for the view.
-     * We do this lazily to avoid initilization issues.
-     *
-     * @return The views CFS instance
-     */
-    public ColumnFamilyStore getViewCfs()
-    {
-        if (_viewCfs == null)
-            _viewCfs = Keyspace.openAndGetStore(Schema.instance.getCFMetaData(baseCfs.keyspace.getName(), name));
-
-        return _viewCfs;
-    }
-
-
-    /**
-     * Lookup column definitions in the base table that correspond to the view columns (should be 1:1)
-     *
-     * Notify caller if all primary keys in the view are ALL primary keys in the base. We do this to simplify
-     * tombstone checks.
-     *
-     * @param columns a list of columns to lookup in the base table
-     * @param definitions lists to populate for the base table definitions
-     * @return true if all view PKs are also Base PKs
-     */
-    private boolean resolveAndAddColumns(Iterable<ColumnIdentifier> columns, List<ColumnDefinition>... definitions)
-    {
-        boolean allArePrimaryKeys = true;
-        for (ColumnIdentifier identifier : columns)
-        {
-            ColumnDefinition cdef = baseCfs.metadata.getColumnDefinition(identifier);
-            assert cdef != null : "Could not resolve column " + identifier.toString();
-
-            for (List<ColumnDefinition> list : definitions)
-            {
-                list.add(cdef);
-            }
-
-            allArePrimaryKeys = allArePrimaryKeys && cdef.isPrimaryKeyColumn();
-        }
-
-        return allArePrimaryKeys;
-    }
-
-    /**
-     * This updates the columns stored which are dependent on the base CFMetaData.
-     *
-     * @return true if the view contains only columns which are part of the base's primary key; false if there is at
-     *         least one column which is not.
-     */
-    public boolean updateDefinition(MaterializedViewDefinition definition)
-    {
-        List<ColumnDefinition> partitionDefs = new ArrayList<>(definition.partitionColumns.size());
-        List<ColumnDefinition> primaryKeyDefs = new ArrayList<>(definition.partitionColumns.size()
-                                                                + definition.clusteringColumns.size());
-        List<ColumnDefinition> baseComplexColumns = new ArrayList<>();
-
-        // We only add the partition columns to the partitions list, but both partition columns and clustering
-        // columns are added to the primary keys list
-        boolean partitionAllPrimaryKeyColumns = resolveAndAddColumns(definition.partitionColumns, primaryKeyDefs, partitionDefs);
-        boolean clusteringAllPrimaryKeyColumns = resolveAndAddColumns(definition.clusteringColumns, primaryKeyDefs);
-
-        for (ColumnDefinition cdef : baseCfs.metadata.allColumns())
-        {
-            if (cdef.isComplex())
-            {
-                baseComplexColumns.add(cdef);
-            }
-        }
-
-        this.columns = new MVColumns(partitionDefs, primaryKeyDefs, baseComplexColumns);
-
-        return partitionAllPrimaryKeyColumns && clusteringAllPrimaryKeyColumns;
-    }
-
-    /**
-     * Check to see if the update could possibly modify a view. Cases where the view may be updated are:
-     * <ul>
-     *     <li>View selects all columns</li>
-     *     <li>Update contains any range tombstones</li>
-     *     <li>Update touches one of the columns included in the view</li>
-     * </ul>
-     *
-     * If the update contains any range tombstones, there is a possibility that it will not touch a range that is
-     * currently included in the view.
-     *
-     * @return true if {@param partition} modifies a column included in the view
-     */
-    public boolean updateAffectsView(AbstractBTreePartition partition)
-    {
-        // If we are including all of the columns, then any update will be included
-        if (includeAll)
-            return true;
-
-        // If there are range tombstones, tombstones will also need to be generated for the materialized view
-        // This requires a query of the base rows and generating tombstones for all of those values
-        if (!partition.deletionInfo().isLive())
-            return true;
-
-        // Check each row for deletion or update
-        for (Row row : partition)
-        {
-            if (row.hasComplexDeletion())
-                return true;
-            if (!row.deletion().isLive())
-                return true;
-
-            for (ColumnData data : row)
-            {
-                if (getViewCfs().metadata.getColumnDefinition(data.column().name) != null)
-                    return true;
-            }
-        }
-
-        return false;
-    }
-
-    /**
-     * Creates the clustering columns for the view based on the specified row and resolver policy
-     *
-     * @param temporalRow The current row
-     * @param resolver The policy to use when selecting versions of cells use
-     * @return The clustering object to use for the view
-     */
-    private Clustering viewClustering(TemporalRow temporalRow, TemporalRow.Resolver resolver)
-    {
-        CFMetaData viewCfm = getViewCfs().metadata;
-        int numViewClustering = viewCfm.clusteringColumns().size();
-        CBuilder clustering = CBuilder.create(getViewCfs().getComparator());
-        for (int i = 0; i < numViewClustering; i++)
-        {
-            ColumnDefinition definition = viewCfm.clusteringColumns().get(i);
-            clustering.add(temporalRow.clusteringValue(definition, resolver));
-        }
-
-        return clustering.build();
-    }
-
-    /**
-     * @return Mutation containing a range tombstone for a base partition key and TemporalRow.
-     */
-    private PartitionUpdate createTombstone(TemporalRow temporalRow,
-                                            DecoratedKey partitionKey,
-                                            Row.Deletion deletion,
-                                            TemporalRow.Resolver resolver,
-                                            int nowInSec)
-    {
-        CFMetaData viewCfm = getViewCfs().metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addRowDeletion(deletion);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
-    }
-
-    /**
-     * @return PartitionUpdate containing a complex tombstone for a TemporalRow, and the collection's column identifier.
-     */
-    private PartitionUpdate createComplexTombstone(TemporalRow temporalRow,
-                                                   DecoratedKey partitionKey,
-                                                   ColumnDefinition deletedColumn,
-                                                   DeletionTime deletionTime,
-                                                   TemporalRow.Resolver resolver,
-                                                   int nowInSec)
-    {
-
-        CFMetaData viewCfm = getViewCfs().metadata;
-        Row.Builder builder = BTreeRow.unsortedBuilder(nowInSec);
-        builder.newRow(viewClustering(temporalRow, resolver));
-        builder.addComplexDeletion(deletedColumn, deletionTime);
-        return PartitionUpdate.singleRowUpdate(viewCfm, partitionKey, builder.build());
-    }
-
-    /**
-     * @return View's DecoratedKey or null, if one of the view's primary key components has an invalid resolution from
-     *         the TemporalRow and its Resolver
-     */
-    private DecoratedKey viewPartitionKey(TemporalRow temporalRow, TemporalRow.Resolver resolver)
-    {
-        List<ColumnDefinition> partitionDefs = this.columns.partitionDefs;
-        Object[] partitionKey = new Object[partitionDefs.size()];
-
-        for (int i = 0; i < partitionKey.length; i++)
-        {
-            ByteBuffer value = temporalRow.clusteringValue(partitionDefs.get(i), resolver);
-
-            if (value == null)
-                return null;
-
-            partitionKey[i] = value;
-        }
-
-        CFMetaData metadata = getViewCfs().metadata;
-        return metadata.decorateKey(CFMetaData.serializePartitionKey(metadata
-                                                                     .getKeyValidatorAsClusteringComparator()
-                                                                     .make(partitionKey)));
-    }
-
-    /**
-     * @return mutation which contains the tombstone for the referenced TemporalRow, or null if not necessary.
-     * TemporalRow's can reference at most one view row; there will be at most one row to be tombstoned, so only one
-     * mutation is necessary
-     */
-    private PartitionUpdate createRangeTombstoneForRow(TemporalRow temporalRow)
-    {
-        // Primary Key and Clustering columns do not generate tombstones
-        if (viewHasAllPrimaryKeys)
-            return null;
-
-        boolean hasUpdate = false;
-        List<ColumnDefinition> primaryKeyDefs = this.columns.primaryKeyDefs;
-        for (ColumnDefinition viewPartitionKeys : primaryKeyDefs)
-        {
-            if (!viewPartitionKeys.isPrimaryKeyColumn() && temporalRow.clusteringValue(viewPartitionKeys, TemporalRow.oldValueIfUpdated) != null)
-                hasUpdate = true;
-        }
-
-        if (!hasUpdate)
-            return null;
-
-        TemporalRow.Resolver resolver = TemporalRow.earliest;
-        return createTombstone(temporalRow,
-                               viewPartitionKey(temporalRow, resolver),
-                               Row.Deletion.shadowable(new DeletionTime(temporalRow.viewClusteringTimestamp(), temporalRow.nowInSec)),
-                               resolver,
-                               temporalRow.nowInSec);
-    }
-
-    /**
-     * @return Mutation which is the transformed base table mutation for the materialized view.
-     */
-    private PartitionUpdate createUpdatesForInserts(TemporalRow temporalRow)
-    {
-        TemporalRow.Resolver resolver = TemporalRow.latest;
-
-        DecoratedKey partitionKey = viewPartitionKey(temporalRow, resolver);
-        ColumnFamilyStore viewCfs = getViewCfs();
-
-        if (partitionKey == null)
-        {
-            // Not having a partition key means we aren't updating anything
-            return null;
-        }
-
-        Row.Builder regularBuilder = BTreeRow.unsortedBuilder(temporalRow.nowInSec);
-
-        CBuilder clustering = CBuilder.create(viewCfs.getComparator());
-        for (int i = 0; i < viewCfs.metadata.clusteringColumns().size(); i++)
-        {
-            clustering.add(temporalRow.clusteringValue(viewCfs.metadata.clusteringColumns().get(i), resolver));
-        }
-        regularBuilder.newRow(clustering.build());
-        regularBuilder.addPrimaryKeyLivenessInfo(LivenessInfo.create(viewCfs.metadata,
-                                                                     temporalRow.viewClusteringTimestamp(),
-                                                                     temporalRow.viewClusteringTtl(),
-                                                                     temporalRow.viewClusteringLocalDeletionTime()));
-
-        for (ColumnDefinition columnDefinition : viewCfs.metadata.allColumns())
-        {
-            if (columnDefinition.isPrimaryKeyColumn())
-                continue;
-
-            for (Cell cell : temporalRow.values(columnDefinition, resolver))
-            {
-                regularBuilder.addCell(cell);
-            }
-        }
-
-        return PartitionUpdate.singleRowUpdate(viewCfs.metadata, partitionKey, regularBuilder.build());
-    }
-
-    /**
-     * @param partition Update which possibly contains deletion info for which to generate view tombstones.
-     * @return    View Tombstones which delete all of the rows which have been removed from the base table with
-     *            {@param partition}
-     */
-    private Collection<Mutation> createForDeletionInfo(TemporalRow.Set rowSet, AbstractBTreePartition partition)
-    {
-        final TemporalRow.Resolver resolver = TemporalRow.earliest;
-
-        DeletionInfo deletionInfo = partition.deletionInfo();
-
-        List<Mutation> mutations = new ArrayList<>();
-
-        // Check the complex columns to see if there are any which may have tombstones we need to create for the view
-        if (!columns.baseComplexColumns.isEmpty())
-        {
-            for (Row row : partition)
-            {
-                if (!row.hasComplexDeletion())
-                    continue;
-
-                TemporalRow temporalRow = rowSet.getClustering(row.clustering());
-
-                assert temporalRow != null;
-
-                for (ColumnDefinition definition : columns.baseComplexColumns)
-                {
-                    ComplexColumnData columnData = row.getComplexColumnData(definition);
-
-                    if (columnData != null)
-                    {
-                        DeletionTime time = columnData.complexDeletion();
-                        if (!time.isLive())
-                        {
-                            DecoratedKey targetKey = viewPartitionKey(temporalRow, resolver);
-                            if (targetKey != null)
-                                mutations.add(new Mutation(createComplexTombstone(temporalRow, targetKey, definition, time, resolver, temporalRow.nowInSec)));
-                        }
-                    }
-                }
-            }
-        }
-
-        ReadCommand command = null;
-
-        if (!deletionInfo.isLive())
-        {
-            // We have to generate tombstones for all of the affected rows, but we don't have the information in order
-            // to create them. This requires that we perform a read for the entire range that is being tombstoned, and
-            // generate a tombstone for each. This may be slow, because a single range tombstone can cover up to an
-            // entire partition of data which is not distributed on a single partition node.
-            DecoratedKey dk = rowSet.dk;
-
-            if (deletionInfo.hasRanges())
-            {
-                SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, dk);
-                Iterator<RangeTombstone> tombstones = deletionInfo.rangeIterator(false);
-                while (tombstones.hasNext())
-                {
-                    RangeTombstone tombstone = tombstones.next();
-
-                    builder.addSlice(tombstone.deletedSlice());
-                }
-
-                command = builder.build();
-            }
-            else
-            {
-                command = SinglePartitionReadCommand.fullPartitionRead(baseCfs.metadata, rowSet.nowInSec, dk);
-            }
-        }
-
-        if (command == null)
-        {
-            SinglePartitionSliceBuilder builder = null;
-            for (Row row : partition)
-            {
-                if (!row.deletion().isLive())
-                {
-                    if (builder == null)
-                        builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-                    builder.addSlice(Slice.make(row.clustering()));
-                }
-            }
-
-            if (builder != null)
-                command = builder.build();
-        }
-
-        if (command != null)
-        {
-
-            //We may have already done this work for
-            //another MV update so check
-
-            if (!rowSet.hasTombstonedExisting())
-            {
-                QueryPager pager = command.getPager(null);
-
-                // Add all of the rows which were recovered from the query to the row set
-                while (!pager.isExhausted())
-                {
-                    try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                         PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-                    {
-                        if (!iter.hasNext())
-                            break;
-
-                        try (RowIterator rowIterator = iter.next())
-                        {
-                            while (rowIterator.hasNext())
-                            {
-                                Row row = rowIterator.next();
-                                rowSet.addRow(row, false);
-                            }
-                        }
-                    }
-                }
-
-                //Incase we fetched nothing, avoid re checking on another MV update
-                rowSet.setTombstonedExisting();
-            }
-
-            // If the temporal row has been deleted by the deletion info, we generate the corresponding range tombstone
-            // for the view.
-            for (TemporalRow temporalRow : rowSet)
-            {
-                DeletionTime deletionTime = temporalRow.deletionTime(partition);
-                if (!deletionTime.isLive())
-                {
-                    DecoratedKey value = viewPartitionKey(temporalRow, resolver);
-                    if (value != null)
-                    {
-                        PartitionUpdate update = createTombstone(temporalRow, value, Row.Deletion.regular(deletionTime), resolver, temporalRow.nowInSec);
-                        if (update != null)
-                            mutations.add(new Mutation(update));
-                    }
-                }
-            }
-        }
-
-        return !mutations.isEmpty() ? mutations : null;
-    }
-
-    /**
-     * Read and update temporal rows in the set which have corresponding values stored on the local node
-     */
-    private void readLocalRows(TemporalRow.Set rowSet)
-    {
-        SinglePartitionSliceBuilder builder = new SinglePartitionSliceBuilder(baseCfs, rowSet.dk);
-
-        for (TemporalRow temporalRow : rowSet)
-            builder.addSlice(temporalRow.baseSlice());
-
-        QueryPager pager = builder.build().getPager(null);
-
-        while (!pager.isExhausted())
-        {
-            try (ReadOrderGroup orderGroup = pager.startOrderGroup();
-                 PartitionIterator iter = pager.fetchPageInternal(128, orderGroup))
-            {
-                while (iter.hasNext())
-                {
-                    try (RowIterator rows = iter.next())
-                    {
-                        while (rows.hasNext())
-                        {
-                            rowSet.addRow(rows.next(), false);
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * @return Set of rows which are contained in the partition update {@param partition}
-     */
-    private TemporalRow.Set separateRows(AbstractBTreePartition partition, Set<ColumnIdentifier> viewPrimaryKeyCols)
-    {
-
-        TemporalRow.Set rowSet = new TemporalRow.Set(baseCfs, viewPrimaryKeyCols, partition.partitionKey().getKey());
-
-        for (Row row : partition)
-            rowSet.addRow(row, true);
-
-        return rowSet;
-    }
-
-    /**
-     * Splits the partition update up and adds the existing state to each row.
-     * This data can be reused for multiple MV updates on the same base table
-     *
-     * @param partition the mutation
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return The set of temoral rows contained in this update
-     */
-    public TemporalRow.Set getTemporalRowSet(AbstractBTreePartition partition, TemporalRow.Set existing, boolean isBuilding)
-    {
-        if (!updateAffectsView(partition))
-            return null;
-
-        Set<ColumnIdentifier> columns = new HashSet<>(this.columns.primaryKeyDefs.size());
-        for (ColumnDefinition def : this.columns.primaryKeyDefs)
-            columns.add(def.name);
-
-        TemporalRow.Set rowSet = null;
-        if (existing == null)
-        {
-            rowSet = separateRows(partition, columns);
-
-            // If we are building the view, we do not want to add old values; they will always be the same
-            if (!isBuilding)
-                readLocalRows(rowSet);
-        }
-        else
-        {
-            rowSet = existing.withNewViewPrimaryKey(columns);
-        }
-
-        return rowSet;
-    }
-
-
-    /**
-     * @param isBuilding If the view is currently being built, we do not query the values which are already stored,
-     *                   since all of the update will already be present in the base table.
-     * @return View mutations which represent the changes necessary as long as previously created mutations for the view
-     *         have been applied successfully. This is based solely on the changes that are necessary given the current
-     *         state of the base table and the newly applying partition data.
-     */
-    public Collection<Mutation> createMutations(AbstractBTreePartition partition, TemporalRow.Set rowSet, boolean isBuilding)
-    {
-        if (!updateAffectsView(partition))
-            return null;
-
-        Collection<Mutation> mutations = null;
-        for (TemporalRow temporalRow : rowSet)
-        {
-            // If we are building, there is no need to check for partition tombstones; those values will not be present
-            // in the partition data
-            if (!isBuilding)
-            {
-                PartitionUpdate partitionTombstone = createRangeTombstoneForRow(temporalRow);
-                if (partitionTombstone != null)
-                {
-                    if (mutations == null) mutations = new LinkedList<>();
-                    mutations.add(new Mutation(partitionTombstone));
-                }
-            }
-
-            PartitionUpdate insert = createUpdatesForInserts(temporalRow);
-            if (insert != null)
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.add(new Mutation(insert));
-            }
-        }
-
-        if (!isBuilding)
-        {
-            Collection<Mutation> deletion = createForDeletionInfo(rowSet, partition);
-            if (deletion != null && !deletion.isEmpty())
-            {
-                if (mutations == null) mutations = new LinkedList<>();
-                mutations.addAll(deletion);
-            }
-        }
-
-        return mutations;
-    }
-
-    public synchronized void build()
-    {
-        if (this.builder != null)
-        {
-            this.builder.stop();
-            this.builder = null;
-        }
-
-        this.builder = new MaterializedViewBuilder(baseCfs, this);
-        CompactionManager.instance.submitMaterializedViewBuilder(builder);
-    }
-
-    @Nullable
-    public static CFMetaData findBaseTable(String keyspace, String view)
-    {
-        KeyspaceMetadata ksm = Schema.instance.getKSMetaData(keyspace);
-        if (ksm == null)
-            return null;
-
-        for (CFMetaData cfm : ksm.tables)
-            if (cfm.getMaterializedViews().get(view).isPresent())
-                return cfm;
-
-        return null;
-    }
-
-    /**
-     * @return CFMetaData which represents the definition given
-     */
-    public static CFMetaData getCFMetaData(MaterializedViewDefinition definition,
-                                           CFMetaData baseCf,
-                                           CFProperties properties)
-    {
-        CFMetaData.Builder viewBuilder = CFMetaData.Builder
-                                         .createView(baseCf.ksName, definition.viewName);
-
-        ColumnDefinition nonPkTarget = null;
-
-        for (ColumnIdentifier targetIdentifier : definition.partitionColumns)
-        {
-            ColumnDefinition target = baseCf.getColumnDefinition(targetIdentifier);
-            if (!target.isPartitionKey())
-                nonPkTarget = target;
-
-            viewBuilder.addPartitionKey(target.name, properties.getReversableType(targetIdentifier, target.type));
-        }
-
-        Collection<ColumnDefinition> included = new ArrayList<>();
-        for(ColumnIdentifier identifier : definition.included)
-        {
-            ColumnDefinition cfDef = baseCf.getColumnDefinition(identifier);
-            assert cfDef != null;
-            included.add(cfDef);
-        }
-
-        boolean includeAll = included.isEmpty();
-
-        for (ColumnIdentifier ident : definition.clusteringColumns)
-        {
-            ColumnDefinition column = baseCf.getColumnDefinition(ident);
-            viewBuilder.addClusteringColumn(ident, properties.getReversableType(ident, column.type));
-        }
-
-        for (ColumnDefinition column : baseCf.partitionColumns().regulars)
-        {
-            if (column != nonPkTarget && (includeAll || included.contains(column)))
-            {
-                viewBuilder.addRegularColumn(column.name, column.type);
-            }
-        }
-
-        //Add any extra clustering columns
-        for (ColumnDefinition column : Iterables.concat(baseCf.partitionKeyColumns(), baseCf.clusteringColumns()))
-        {
-            if ( (!definition.partitionColumns.contains(column.name) && !definition.clusteringColumns.contains(column.name)) &&
-                 (includeAll || included.contains(column)) )
-            {
-                viewBuilder.addRegularColumn(column.name, column.type);
-            }
-        }
-
-        return viewBuilder.build().params(properties.properties.asNewTableParams());
-    }
-}


[2/7] cassandra git commit: Improve MV schema representation

Posted by al...@apache.org.
http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/long/org/apache/cassandra/cql3/ViewLongTest.java
----------------------------------------------------------------------
diff --git a/test/long/org/apache/cassandra/cql3/ViewLongTest.java b/test/long/org/apache/cassandra/cql3/ViewLongTest.java
new file mode 100644
index 0000000..3808b73
--- /dev/null
+++ b/test/long/org/apache/cassandra/cql3/ViewLongTest.java
@@ -0,0 +1,188 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CyclicBarrier;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import com.datastax.driver.core.Row;
+import com.datastax.driver.core.exceptions.NoHostAvailableException;
+import com.datastax.driver.core.exceptions.WriteTimeoutException;
+import org.apache.cassandra.concurrent.SEPExecutor;
+import org.apache.cassandra.concurrent.Stage;
+import org.apache.cassandra.concurrent.StageManager;
+import org.apache.cassandra.batchlog.BatchlogManager;
+import org.apache.cassandra.utils.WrappedRunnable;
+
+public class ViewLongTest extends CQLTester
+{
+    int protocolVersion = 4;
+    private final List<String> views = new ArrayList<>();
+
+    @BeforeClass
+    public static void startup()
+    {
+        requireNetwork();
+    }
+    @Before
+    public void begin()
+    {
+        views.clear();
+    }
+
+    @After
+    public void end() throws Throwable
+    {
+        for (String viewName : views)
+            executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + viewName);
+    }
+
+    private void createView(String name, String query) throws Throwable
+    {
+        executeNet(protocolVersion, String.format(query, name));
+        // If exception is thrown, the view will not be added to the list; since it shouldn't have been created, this is
+        // the desired behavior
+        views.add(name);
+    }
+
+    @Test
+    public void testConflictResolution() throws Throwable
+    {
+        final int writers = 96;
+        final int insertsPerWriter = 50;
+        final Map<Integer, Exception> failedWrites = new ConcurrentHashMap<>();
+
+        createTable("CREATE TABLE %s (" +
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "PRIMARY KEY (a, b))");
+
+        executeNet(protocolVersion, "USE " + keyspace());
+
+        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
+
+        CyclicBarrier semaphore = new CyclicBarrier(writers);
+
+        Thread[] threads = new Thread[writers];
+        for (int i = 0; i < writers; i++)
+        {
+            final int writer = i;
+            Thread t = new Thread(new WrappedRunnable()
+            {
+                public void runMayThrow()
+                {
+                    try
+                    {
+                        int writerOffset = writer * insertsPerWriter;
+                        semaphore.await();
+                        for (int i = 0; i < insertsPerWriter; i++)
+                        {
+                            try
+                            {
+                                executeNet(protocolVersion, "INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP 1",
+                                           1,
+                                           1,
+                                           i + writerOffset);
+                            }
+                            catch (NoHostAvailableException|WriteTimeoutException e)
+                            {
+                                failedWrites.put(i + writerOffset, e);
+                            }
+                        }
+                    }
+                    catch (Throwable e)
+                    {
+                        throw new RuntimeException(e);
+                    }
+                }
+            });
+            t.start();
+            threads[i] = t;
+        }
+
+        for (int i = 0; i < writers; i++)
+            threads[i].join();
+
+        for (int i = 0; i < writers * insertsPerWriter; i++)
+        {
+            if (executeNet(protocolVersion, "SELECT COUNT(*) FROM system.batchlog").one().getLong(0) == 0)
+                break;
+            try
+            {
+                // This will throw exceptions whenever there are exceptions trying to push the view values out, caused
+                // by the view becoming overwhelmed.
+                BatchlogManager.instance.startBatchlogReplay().get();
+            }
+            catch (Throwable ignore)
+            {
+
+            }
+        }
+
+        int value = executeNet(protocolVersion, "SELECT c FROM %s WHERE a = 1 AND b = 1").one().getInt("c");
+
+        List<Row> rows = executeNet(protocolVersion, "SELECT c FROM " + keyspace() + ".mv").all();
+
+        boolean containsC = false;
+        StringBuilder others = new StringBuilder();
+        StringBuilder overlappingFailedWrites = new StringBuilder();
+        for (Row row : rows)
+        {
+            int c = row.getInt("c");
+            if (c == value)
+                containsC = true;
+            else
+            {
+                if (others.length() != 0)
+                    others.append(' ');
+                others.append(c);
+                if (failedWrites.containsKey(c))
+                {
+                    if (overlappingFailedWrites.length() != 0)
+                        overlappingFailedWrites.append(' ');
+                    overlappingFailedWrites.append(c)
+                                           .append(':')
+                                           .append(failedWrites.get(c).getMessage());
+                }
+            }
+        }
+
+        if (rows.size() > 1)
+        {
+            throw new AssertionError(String.format("Expected 1 row, but found %d; %s c = %d, and (%s) of which (%s) failed to insert", rows.size(), containsC ? "found row with" : "no rows contained", value, others, overlappingFailedWrites));
+        }
+        else if (rows.isEmpty())
+        {
+            throw new AssertionError(String.format("Could not find row with c = %d", value));
+        }
+        else if (rows.size() == 1 && !containsC)
+        {
+            throw new AssertionError(String.format("Single row had c = %d, expected %d", rows.get(0).getInt("c"), value));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/SchemaLoader.java b/test/unit/org/apache/cassandra/SchemaLoader.java
index 175e492..72bfd00 100644
--- a/test/unit/org/apache/cassandra/SchemaLoader.java
+++ b/test/unit/org/apache/cassandra/SchemaLoader.java
@@ -309,7 +309,7 @@ public class SchemaLoader
     private static void useCompression(List<KeyspaceMetadata> schema)
     {
         for (KeyspaceMetadata ksm : schema)
-            for (CFMetaData cfm : ksm.tables)
+            for (CFMetaData cfm : ksm.tablesAndViews())
                 cfm.compression(CompressionParams.snappy());
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
index 5887c9a..3a3b6ee 100644
--- a/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
+++ b/test/unit/org/apache/cassandra/config/DatabaseDescriptorTest.java
@@ -52,7 +52,7 @@ public class DatabaseDescriptorTest
         // test serialization of all defined test CFs.
         for (String keyspaceName : Schema.instance.getNonSystemKeyspaces())
         {
-            for (CFMetaData cfm : Schema.instance.getTables(keyspaceName))
+            for (CFMetaData cfm : Schema.instance.getTablesAndViews(keyspaceName))
             {
                 CFMetaData cfmDupe = ThriftConversion.fromThrift(ThriftConversion.toThrift(cfm));
                 assertNotNull(cfmDupe);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/a3a8dbca/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java b/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java
deleted file mode 100644
index f4b4a82..0000000
--- a/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java
+++ /dev/null
@@ -1,1251 +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.cql3;
-
-import java.math.BigDecimal;
-import java.math.BigInteger;
-import java.net.InetAddress;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.HashSet;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.datastax.driver.core.*;
-import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.exceptions.InvalidQueryException;
-import junit.framework.Assert;
-import org.apache.cassandra.concurrent.SEPExecutor;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.concurrent.StageManager;
-import org.apache.cassandra.config.CFMetaData;
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.config.Schema;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.serializers.SimpleDateSerializer;
-import org.apache.cassandra.serializers.TimeSerializer;
-import org.apache.cassandra.utils.ByteBufferUtil;
-import org.apache.cassandra.utils.FBUtilities;
-
-public class MaterializedViewTest extends CQLTester
-{
-    int protocolVersion = 4;
-    private final List<String> materializedViews = new ArrayList<>();
-
-    @BeforeClass
-    public static void startup()
-    {
-        requireNetwork();
-    }
-    @Before
-    public void begin()
-    {
-        materializedViews.clear();
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        for (String viewName : materializedViews)
-            executeNet(protocolVersion, "DROP MATERIALIZED VIEW " + viewName);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        executeNet(protocolVersion, String.format(query, name));
-        // If exception is thrown, the view will not be added to the list; since it shouldn't have been created, this is
-        // the desired behavior
-        materializedViews.add(name);
-    }
-
-    private void updateMV(String query, Object... params) throws Throwable
-    {
-        executeNet(protocolVersion, query, params);
-        while (!(((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getPendingTasks() == 0
-                 && ((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getActiveCount() == 0))
-        {
-            Thread.sleep(1);
-        }
-    }
-
-    @Test
-    public void testPartitionTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
-
-        updateMV("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
-        updateMV("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
-
-        Assert.assertEquals(1, execute("select * from %s").size());
-        Assert.assertEquals(1, execute("select * from view1").size());
-
-        updateMV("DELETE FROM %s WHERE k1 = 1");
-
-        Assert.assertEquals(0, execute("select * from %s").size());
-        Assert.assertEquals(0, execute("select * from view1").size());
-    }
-
-    @Test
-    public void testClusteringKeyTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
-
-        updateMV("INSERT INTO %s (k1, c1, val) VALUES (1, 2, 200)");
-        updateMV("INSERT INTO %s (k1, c1, val) VALUES (1, 3, 300)");
-
-        Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from view1").size());
-
-        updateMV("DELETE FROM %s WHERE k1 = 1 and c1 = 3");
-
-        Assert.assertEquals(1, execute("select * from %s").size());
-        Assert.assertEquals(1, execute("select * from view1").size());
-    }
-
-    @Test
-    public void testPrimaryKeyIsNotNull() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "PRIMARY KEY((k, asciival)))");
-
-        // Must include "IS NOT NULL" for primary keys
-        try
-        {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s");
-            Assert.fail("Should fail if no primary key is filtered as NOT NULL");
-        }
-        catch (Exception e)
-        {
-        }
-
-        // Must include both when the partition key is composite
-        try
-        {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
-            Assert.fail("Should fail if compound primary is not completely filtered as NOT NULL");
-        }
-        catch (Exception e)
-        {
-        }
-
-        dropTable("DROP TABLE %s");
-
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "PRIMARY KEY(k, asciival))");
-        try
-        {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s");
-            Assert.fail("Should fail if no primary key is filtered as NOT NULL");
-        }
-        catch (Exception e)
-        {
-        }
-
-        // Can omit "k IS NOT NULL" because we have a sinlge partition key
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
-    }
-
-    @Test
-    public void testAccessAndSchema() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "PRIMARY KEY((k, asciival)))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE bigintval IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL PRIMARY KEY (bigintval, k, asciival)");
-        updateMV("INSERT INTO %s(k,asciival,bigintval)VALUES(?,?,?)", 0, "foo", 1L);
-
-        try
-        {
-            updateMV("INSERT INTO mv1_test(k,asciival,bigintval) VALUES(?,?,?)", 1, "foo", 2L);
-            Assert.fail("Shouldn't be able to modify a MV directly");
-        }
-        catch (Exception e)
-        {
-        }
-
-        try
-        {
-            executeNet(protocolVersion, "ALTER TABLE mv1_test ADD foo text");
-            Assert.fail("Should not be able to use alter table with MV");
-        }
-        catch (Exception e)
-        {
-        }
-
-        try
-        {
-            executeNet(protocolVersion, "ALTER TABLE mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
-            Assert.fail("Should not be able to use alter table with MV");
-        }
-        catch (Exception e)
-        {
-        }
-
-        executeNet(protocolVersion, "ALTER MATERIALIZED VIEW mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
-
-        //Test alter add
-        executeNet(protocolVersion, "ALTER TABLE %s ADD foo text");
-        CFMetaData metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test");
-        Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("foo")));
-
-        updateMV("INSERT INTO %s(k,asciival,bigintval,foo)VALUES(?,?,?,?)", 0, "foo", 1L, "bar");
-        assertRows(execute("SELECT foo from %s"), row("bar"));
-
-        //Test alter rename
-        executeNet(protocolVersion, "ALTER TABLE %s RENAME asciival TO bar");
-
-        assertRows(execute("SELECT bar from %s"), row("foo"));
-        metadata = Schema.instance.getCFMetaData(keyspace(), "mv1_test");
-        Assert.assertNotNull(metadata.getColumnDefinition(ByteBufferUtil.bytes("bar")));
-    }
-
-
-    @Test
-    public void testStaticTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "c int, " +
-                    "sval text static, " +
-                    "val text, " +
-                    "PRIMARY KEY(k,c))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        try
-        {
-            createView("mv_static", "CREATE MATERIALIZED VIEW %%s AS SELECT * FROM %s WHERE sval IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (sval,k,c)");
-            Assert.fail("MV on static should fail");
-        }
-        catch (InvalidQueryException e)
-        {
-        }
-
-        createView("mv_static", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
-
-        for (int i = 0; i < 100; i++)
-            updateMV("INSERT into %s (k,c,sval,val)VALUES(?,?,?,?)", 0, i % 2, "bar" + i, "baz");
-
-        Assert.assertEquals(2, execute("select * from %s").size());
-
-        assertRows(execute("SELECT sval from %s"), row("bar99"), row("bar99"));
-
-        Assert.assertEquals(2, execute("select * from mv_static").size());
-
-        assertInvalid("SELECT sval from mv_static");
-    }
-
-
-    @Test
-    public void testOldTimestamps() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "c int, " +
-                    "val text, " +
-                    "PRIMARY KEY(k,c))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv_tstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
-
-        for (int i = 0; i < 100; i++)
-            updateMV("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz");
-
-        Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush();
-
-        Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv_tstest").size());
-
-        assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz"));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1));
-
-        //Make sure an old TS does nothing
-        updateMV("UPDATE %s USING TIMESTAMP 100 SET val = ? where k = ? AND c = ?", "bar", 0, 0);
-        assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("baz"));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0), row(1));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"));
-
-        //Latest TS
-        updateMV("UPDATE %s SET val = ? where k = ? AND c = ?", "bar", 0, 0);
-        assertRows(execute("SELECT val from %s where k = 0 and c = 0"), row("bar"));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"), row(0));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(1));
-    }
-
-    @Test
-    public void testCountersTable() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int PRIMARY KEY, " +
-                    "count counter)");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        try
-        {
-            createView("mv_counter", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE count IS NOT NULL AND k IS NOT NULL PRIMARY KEY (count,k)");
-            Assert.fail("MV on counter should fail");
-        }
-        catch (InvalidQueryException e)
-        {
-        }
-    }
-
-    @Test
-    public void complexTimestampUpdateTestWithFlush() throws Throwable
-    {
-        complexTimestampUpdateTest(true);
-    }
-
-    @Test
-    public void complexTimestampUpdateTestWithoutFlush() throws Throwable
-    {
-        complexTimestampUpdateTest(false);
-    }
-
-    public void complexTimestampUpdateTest(boolean flush) throws Throwable
-    {
-        createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-        Keyspace ks = Keyspace.open(keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, a, b)");
-        ks.getColumnFamilyStore("mv").disableAutoCompaction();
-
-        //Set initial values TS=0, leaving e null and verify view
-        executeNet(protocolVersion, "INSERT INTO %s (a, b, c, d) VALUES (0, 0, 1, 0) USING TIMESTAMP 0");
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
-
-        if (flush)
-            FBUtilities.waitOnFutures(ks.flush());
-
-        //update c's timestamp TS=2
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
-
-        if (flush)
-            FBUtilities.waitOnFutures(ks.flush());
-
-            //change c's value and TS=3, tombstones c=1 and adds c=0 record
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? and b = ? ", 0, 0, 0);
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0));
-
-        if(flush)
-        {
-            ks.getColumnFamilyStore("mv").forceMajorCompaction();
-            FBUtilities.waitOnFutures(ks.flush());
-        }
-
-
-        //change c's value back to 1 with TS=4, check we can see d
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
-        if (flush)
-        {
-            ks.getColumnFamilyStore("mv").forceMajorCompaction();
-            FBUtilities.waitOnFutures(ks.flush());
-        }
-
-        assertRows(execute("SELECT d,e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, null));
-
-
-            //Add e value @ TS=1
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 1 SET e = ? WHERE a = ? and b = ? ", 1, 0, 0);
-        assertRows(execute("SELECT d,e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, 1));
-
-        if (flush)
-            FBUtilities.waitOnFutures(ks.flush());
-
-
-        //Change d value @ TS=2
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET d = ? WHERE a = ? and b = ? ", 2, 0, 0);
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(2));
-
-        if (flush)
-            FBUtilities.waitOnFutures(ks.flush());
-
-
-        //Change d value @ TS=3
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? and b = ? ", 1, 0, 0);
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(1));
-
-
-        //Tombstone c
-        executeNet(protocolVersion, "DELETE FROM %s WHERE a = ? and b = ?", 0, 0);
-        assertRows(execute("SELECT d from mv"));
-
-        //Add back without D
-        executeNet(protocolVersion, "INSERT INTO %s (a, b, c) VALUES (0, 0, 1)");
-
-        //Make sure D doesn't pop back in.
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row((Object) null));
-
-
-        //New partition
-        // insert a row with timestamp 0
-        executeNet(protocolVersion, "INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP 0", 1, 0, 0, 0, 0);
-
-        // overwrite pk and e with timestamp 1, but don't overwrite d
-        executeNet(protocolVersion, "INSERT INTO %s (a, b, c, e) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 0, 0, 0);
-
-        // delete with timestamp 0 (which should only delete d)
-        executeNet(protocolVersion, "DELETE FROM %s USING TIMESTAMP 0 WHERE a = ? AND b = ?", 1, 0);
-        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 0, 1, 0),
-                   row(1, 0, 0, null, 0)
-        );
-
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? AND b = ?", 1, 1, 0);
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? AND b = ?", 0, 1, 0);
-        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 0, 1, 0),
-                   row(1, 0, 0, null, 0)
-        );
-
-        executeNet(protocolVersion, "UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? AND b = ?", 0, 1, 0);
-        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 0, 1, 0),
-                   row(1, 0, 0, 0, 0)
-        );
-
-
-    }
-
-    @Test
-    public void testBuilderWidePartition() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "c int, " +
-                    "intval int, " +
-                    "PRIMARY KEY (k, c))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-
-        for(int i = 0; i < 1024; i++)
-            execute("INSERT INTO %s (k, c, intval) VALUES (?, ?, ?)", 0, i, 0);
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, c, k)");
-
-
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv"))
-            Thread.sleep(1000);
-
-        assertRows(execute("SELECT count(*) from %s WHERE k = ?", 0), row(1024L));
-        assertRows(execute("SELECT count(*) from mv WHERE intval = ?", 0), row(1024L));
-    }
-
-    @Test
-    public void testRangeTombstone() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "textval1 text, " +
-                    "textval2 text, " +
-                    "PRIMARY KEY((k, asciival), bigintval, textval1)" +
-                    ")");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv_test1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), asciival, bigintval, textval1)");
-
-        for (int i = 0; i < 100; i++)
-            updateMV("INSERT into %s (k,asciival,bigintval,textval1,textval2)VALUES(?,?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i, "baz");
-
-        Assert.assertEquals(50, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size());
-        Assert.assertEquals(50, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size());
-
-        Assert.assertEquals(100, execute("select * from mv_test1").size());
-
-        //Check the builder works
-        createView("mv_test2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), asciival, bigintval, textval1)");
-
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test2"))
-            Thread.sleep(1000);
-
-        Assert.assertEquals(100, execute("select * from mv_test2").size());
-
-        createView("mv_test3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval2 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL AND textval1 IS NOT NULL PRIMARY KEY ((textval2, k), bigintval, textval1, asciival)");
-
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test3"))
-            Thread.sleep(1000);
-
-        Assert.assertEquals(100, execute("select * from mv_test3").size());
-        Assert.assertEquals(100, execute("select asciival from mv_test3 where textval2 = ? and k = ?", "baz", 0).size());
-
-        //Write a RT and verify the data is removed from index
-        updateMV("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval = ?", 0, "foo", 0L);
-
-        Assert.assertEquals(50, execute("select asciival from mv_test3 where textval2 = ? and k = ?", "baz", 0).size());
-    }
-
-
-    @Test
-    public void testRangeTombstone2() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "textval1 text, " +
-                    "PRIMARY KEY((k, asciival), bigintval)" +
-                    ")");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval1 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL PRIMARY KEY ((textval1, k), asciival, bigintval)");
-
-        for (int i = 0; i < 100; i++)
-            updateMV("INSERT into %s (k,asciival,bigintval,textval1)VALUES(?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i);
-
-        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size());
-        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size());
-
-
-        Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv").size());
-
-        //Write a RT and verify the data is removed from index
-        updateMV("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval = ?", 0, "foo", 0L);
-
-        Assert.assertEquals(1, execute("select * from %s").size());
-        Assert.assertEquals(1, execute("select * from mv").size());
-    }
-
-    @Test
-    public void testRangeTombstone3() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "textval1 text, " +
-                    "PRIMARY KEY((k, asciival), bigintval)" +
-                    ")");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE textval1 IS NOT NULL AND k IS NOT NULL AND asciival IS NOT NULL AND bigintval IS NOT NULL PRIMARY KEY ((textval1, k), asciival, bigintval)");
-
-        for (int i = 0; i < 100; i++)
-            updateMV("INSERT into %s (k,asciival,bigintval,textval1)VALUES(?,?,?,?)", 0, "foo", (long) i % 2, "bar" + i);
-
-        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 0").size());
-        Assert.assertEquals(1, execute("select * from %s where k = 0 and asciival = 'foo' and bigintval = 1").size());
-
-
-        Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv").size());
-
-        //Write a RT and verify the data is removed from index
-        updateMV("DELETE FROM %s WHERE k = ? AND asciival = ? and bigintval >= ?", 0, "foo", 0L);
-
-        Assert.assertEquals(0, execute("select * from %s").size());
-        Assert.assertEquals(0, execute("select * from mv").size());
-    }
-
-    @Test
-    public void testCompoundPartitionKey() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "PRIMARY KEY((k, asciival)))");
-
-        CFMetaData metadata = currentTableMetadata();
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        for (ColumnDefinition def : new HashSet<>(metadata.allColumns()))
-        {
-            try
-            {
-                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
-                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ("
-                               + def.name + ", k" + (def.name.toString().equals("asciival") ? "" : ", asciival") + ")";
-                createView("mv1_" + def.name, query);
-
-                if (def.type.isMultiCell())
-                    Assert.fail("MV on a multicell should fail " + def);
-            }
-            catch (InvalidQueryException e)
-            {
-                if (!def.type.isMultiCell() && !def.isPartitionKey())
-                    Assert.fail("MV creation failed on " + def);
-            }
-
-
-            try
-            {
-                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
-                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + " PRIMARY KEY ("
-                               + def.name + ", asciival" + (def.name.toString().equals("k") ? "" : ", k") + ")";
-                createView("mv2_" + def.name, query);
-
-                if (def.type.isMultiCell())
-                    Assert.fail("MV on a multicell should fail " + def);
-            }
-            catch (InvalidQueryException e)
-            {
-                if (!def.type.isMultiCell() && !def.isPartitionKey())
-                    Assert.fail("MV creation failed on " + def);
-            }
-
-            try
-            {
-                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
-                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), asciival)";
-                createView("mv3_" + def.name, query);
-
-                if (def.type.isMultiCell())
-                    Assert.fail("MV on a multicell should fail " + def);
-            }
-            catch (InvalidQueryException e)
-            {
-                if (!def.type.isMultiCell() && !def.isPartitionKey())
-                    Assert.fail("MV creation failed on " + def);
-            }
-
-
-            try
-            {
-                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
-                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), asciival)";
-                createView("mv3_" + def.name, query);
-
-                Assert.fail("Should fail on duplicate name");
-            }
-            catch (Exception e)
-            {
-            }
-
-            try
-            {
-                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
-                               + (def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ") + "PRIMARY KEY ((" + def.name + ", k), nonexistentcolumn)";
-                createView("mv3_" + def.name, query);
-                Assert.fail("Should fail with unknown base column");
-            }
-            catch (InvalidQueryException e)
-            {
-            }
-        }
-
-        updateMV("INSERT INTO %s (k, asciival, bigintval) VALUES (?, ?, fromJson(?))", 0, "ascii text", "123123123123");
-        updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
-        assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"), row(123123123123L));
-
-        //Check the MV
-        assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"), row(0, 123123123123L));
-        assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0), row(0, 123123123123L));
-        assertRows(execute("SELECT k from mv1_bigintval WHERE bigintval = ?", 123123123123L), row(0));
-        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 123123123123L, 0), row("ascii text"));
-
-
-        //UPDATE BASE
-        updateMV("INSERT INTO %s (k, asciival, bigintval) VALUES (?, ?, fromJson(?))", 0, "ascii text", "1");
-        assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"), row(1L));
-
-        //Check the MV
-        assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"), row(0, 1L));
-        assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0), row(0, 1L));
-        assertRows(execute("SELECT k from mv1_bigintval WHERE bigintval = ?", 123123123123L));
-        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 123123123123L, 0));
-        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 1L, 0), row("ascii text"));
-
-
-        //test truncate also truncates all MV
-        updateMV("TRUNCATE %s");
-
-        assertRows(execute("SELECT bigintval FROM %s WHERE k = ? and asciival = ?", 0, "ascii text"));
-        assertRows(execute("SELECT k, bigintval from mv1_asciival WHERE asciival = ?", "ascii text"));
-        assertRows(execute("SELECT k, bigintval from mv2_k WHERE asciival = ? and k = ?", "ascii text", 0));
-        assertRows(execute("SELECT asciival from mv3_bigintval where bigintval = ? AND k = ?", 1L, 0));
-    }
-
-    @Test
-    public void testCollections() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "intval int, " +
-                    "listval list<int>, " +
-                    "PRIMARY KEY (k))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
-
-        updateMV("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))", 0, 0, "[1, 2, 3]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
-        assertRows(execute("SELECT k, listval from mv WHERE intval = ?", 0), row(0, list(1, 2, 3)));
-
-        updateMV("INSERT INTO %s (k, intval) VALUES (?, ?)", 1, 1);
-        updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 1, "[1, 2, 3]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 1), row(1, list(1, 2, 3)));
-        assertRows(execute("SELECT k, listval from mv WHERE intval = ?", 1), row(1, list(1, 2, 3)));
-    }
-
-    @Test
-    public void testUpdate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "intval int, " +
-                    "PRIMARY KEY (k))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
-
-        updateMV("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 0);
-        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 0));
-        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 0), row(0, 0));
-
-        updateMV("INSERT INTO %s (k, intval) VALUES (?, ?)", 0, 1);
-        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 1));
-        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1));
-    }
-
-    @Test
-    public void testDecimalUpdate() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "k int, " +
-                    "decimalval decimal, " +
-                    "asciival ascii, " +
-                    "PRIMARY KEY (k))");
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND decimalval IS NOT NULL PRIMARY KEY (decimalval, k)");
-
-        updateMV("INSERT INTO %s (k, asciival) VALUES (?, ?)", 0, "ascii text");
-        updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
-        assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "123123.123123"));
-        assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "123123"), row(0, "ascii text"));
-
-        // accept strings for numbers that cannot be represented as doubles
-        updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
-        assertRows(execute("SELECT k, decimalval, asciival FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123"), "ascii text"));
-        assertRows(execute("SELECT k, asciival from mv WHERE decimalval = fromJson(?)", "\"123123.123123\""), row(0, "ascii text"));
-    }
-
-    @Test
-    public void testAllTypes() throws Throwable
-    {
-        String myType = createType("CREATE TYPE %s (a int, b uuid, c set<text>)");
-
-        createTable("CREATE TABLE %s (" +
-                    "k int PRIMARY KEY, " +
-                    "asciival ascii, " +
-                    "bigintval bigint, " +
-                    "blobval blob, " +
-                    "booleanval boolean, " +
-                    "dateval date, " +
-                    "decimalval decimal, " +
-                    "doubleval double, " +
-                    "floatval float, " +
-                    "inetval inet, " +
-                    "intval int, " +
-                    "textval text, " +
-                    "timeval time, " +
-                    "timestampval timestamp, " +
-                    "timeuuidval timeuuid, " +
-                    "uuidval uuid," +
-                    "varcharval varchar, " +
-                    "varintval varint, " +
-                    "listval list<int>, " +
-                    "frozenlistval frozen<list<int>>, " +
-                    "setval set<uuid>, " +
-                    "frozensetval frozen<set<uuid>>, " +
-                    "mapval map<ascii, int>," +
-                    "frozenmapval frozen<map<ascii, int>>," +
-                    "tupleval frozen<tuple<int, ascii, uuid>>," +
-                    "udtval frozen<" + myType + ">)");
-
-        CFMetaData metadata = currentTableMetadata();
-
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        for (ColumnDefinition def : new HashSet<>(metadata.allColumns()))
-        {
-            try
-            {
-                createView("mv_" + def.name, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL PRIMARY KEY (" + def.name + ",k)");
-
-                if (def.type.isMultiCell())
-                    Assert.fail("MV on a multicell should fail " + def);
-
-                if (def.isPartitionKey())
-                    Assert.fail("MV on partition key should fail " + def);
-            }
-            catch (InvalidQueryException e)
-            {
-                if (!def.type.isMultiCell() && !def.isPartitionKey())
-                    Assert.fail("MV creation failed on " + def);
-            }
-        }
-
-        // fromJson() can only be used when the receiver type is known
-        assertInvalidMessage("fromJson() cannot be used in the selection clause", "SELECT fromJson(asciival) FROM %s", 0, 0);
-
-        String func1 = createFunction(KEYSPACE, "int", "CREATE FUNCTION %s (a int) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return a.toString(); $$");
-        createFunctionOverload(func1, "int", "CREATE FUNCTION %s (a text) CALLED ON NULL INPUT RETURNS text LANGUAGE java AS $$ return new String(a); $$");
-
-        // ================ ascii ================
-        updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
-        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii \\\" text\"");
-        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0), row(0, "ascii \" text"));
-
-        // test that we can use fromJson() in other valid places in queries
-        assertRows(execute("SELECT asciival FROM %s WHERE k = fromJson(?)", "0"), row("ascii \" text"));
-
-        //Check the MV
-        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii text"));
-        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"), row(0, null));
-
-        updateMV("UPDATE %s SET asciival = fromJson(?) WHERE k = fromJson(?)", "\"ascii \\\" text\"", "0");
-        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"), row(0, null));
-
-        updateMV("DELETE FROM %s WHERE k = fromJson(?)", "0");
-        assertRows(execute("SELECT k, asciival FROM %s WHERE k = ?", 0));
-        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii \" text"));
-
-        updateMV("INSERT INTO %s (k, asciival) VALUES (?, fromJson(?))", 0, "\"ascii text\"");
-        assertRows(execute("SELECT k, udtval from mv_asciival WHERE asciival = ?", "ascii text"), row(0, null));
-
-        // ================ bigint ================
-        updateMV("INSERT INTO %s (k, bigintval) VALUES (?, fromJson(?))", 0, "123123123123");
-        assertRows(execute("SELECT k, bigintval FROM %s WHERE k = ?", 0), row(0, 123123123123L));
-        assertRows(execute("SELECT k, asciival from mv_bigintval WHERE bigintval = ?", 123123123123L), row(0, "ascii text"));
-
-        // ================ blob ================
-        updateMV("INSERT INTO %s (k, blobval) VALUES (?, fromJson(?))", 0, "\"0x00000001\"");
-        assertRows(execute("SELECT k, blobval FROM %s WHERE k = ?", 0), row(0, ByteBufferUtil.bytes(1)));
-        assertRows(execute("SELECT k, asciival from mv_blobval WHERE blobval = ?", ByteBufferUtil.bytes(1)), row(0, "ascii text"));
-
-        // ================ boolean ================
-        updateMV("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "true");
-        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, true));
-        assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", true), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, booleanval) VALUES (?, fromJson(?))", 0, "false");
-        assertRows(execute("SELECT k, booleanval FROM %s WHERE k = ?", 0), row(0, false));
-        assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", true));
-        assertRows(execute("SELECT k, asciival from mv_booleanval WHERE booleanval = ?", false), row(0, "ascii text"));
-
-        // ================ date ================
-        updateMV("INSERT INTO %s (k, dateval) VALUES (?, fromJson(?))", 0, "\"1987-03-23\"");
-        assertRows(execute("SELECT k, dateval FROM %s WHERE k = ?", 0), row(0, SimpleDateSerializer.dateStringToDays("1987-03-23")));
-        assertRows(execute("SELECT k, asciival from mv_dateval WHERE dateval = fromJson(?)", "\"1987-03-23\""), row(0, "ascii text"));
-
-        // ================ decimal ================
-        updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123.123123");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
-        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123.123123"), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123")));
-        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123.123123"));
-        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "123123"), row(0, "ascii text"));
-
-        // accept strings for numbers that cannot be represented as doubles
-        updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"123123.123123\"");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("123123.123123")));
-
-        updateMV("INSERT INTO %s (k, decimalval) VALUES (?, fromJson(?))", 0, "\"-1.23E-12\"");
-        assertRows(execute("SELECT k, decimalval FROM %s WHERE k = ?", 0), row(0, new BigDecimal("-1.23E-12")));
-        assertRows(execute("SELECT k, asciival from mv_decimalval WHERE decimalval = fromJson(?)", "\"-1.23E-12\""), row(0, "ascii text"));
-
-        // ================ double ================
-        updateMV("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123.123123");
-        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.123123d));
-        assertRows(execute("SELECT k, asciival from mv_doubleval WHERE doubleval = fromJson(?)", "123123.123123"), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, doubleval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, doubleval FROM %s WHERE k = ?", 0), row(0, 123123.0d));
-        assertRows(execute("SELECT k, asciival from mv_doubleval WHERE doubleval = fromJson(?)", "123123"), row(0, "ascii text"));
-
-        // ================ float ================
-        updateMV("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123.123123");
-        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.123123f));
-        assertRows(execute("SELECT k, asciival from mv_floatval WHERE floatval = fromJson(?)", "123123.123123"), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, floatval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, floatval FROM %s WHERE k = ?", 0), row(0, 123123.0f));
-        assertRows(execute("SELECT k, asciival from mv_floatval WHERE floatval = fromJson(?)", "123123"), row(0, "ascii text"));
-
-        // ================ inet ================
-        updateMV("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"127.0.0.1\"");
-        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("127.0.0.1")));
-        assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"127.0.0.1\""), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, inetval) VALUES (?, fromJson(?))", 0, "\"::1\"");
-        assertRows(execute("SELECT k, inetval FROM %s WHERE k = ?", 0), row(0, InetAddress.getByName("::1")));
-        assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"127.0.0.1\""));
-        assertRows(execute("SELECT k, asciival from mv_inetval WHERE inetval = fromJson(?)", "\"::1\""), row(0, "ascii text"));
-
-        // ================ int ================
-        updateMV("INSERT INTO %s (k, intval) VALUES (?, fromJson(?))", 0, "123123");
-        assertRows(execute("SELECT k, intval FROM %s WHERE k = ?", 0), row(0, 123123));
-        assertRows(execute("SELECT k, asciival from mv_intval WHERE intval = fromJson(?)", "123123"), row(0, "ascii text"));
-
-        // ================ text (varchar) ================
-        updateMV("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"some \\\" text\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "some \" text"));
-
-        updateMV("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"\\u2013\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "\u2013"));
-        assertRows(execute("SELECT k, asciival from mv_textval WHERE textval = fromJson(?)", "\"\\u2013\""), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, textval) VALUES (?, fromJson(?))", 0, "\"abcd\"");
-        assertRows(execute("SELECT k, textval FROM %s WHERE k = ?", 0), row(0, "abcd"));
-        assertRows(execute("SELECT k, asciival from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, "ascii text"));
-
-        // ================ time ================
-        updateMV("INSERT INTO %s (k, timeval) VALUES (?, fromJson(?))", 0, "\"07:35:07.000111222\"");
-        assertRows(execute("SELECT k, timeval FROM %s WHERE k = ?", 0), row(0, TimeSerializer.timeStringToLong("07:35:07.000111222")));
-        assertRows(execute("SELECT k, asciival from mv_timeval WHERE timeval = fromJson(?)", "\"07:35:07.000111222\""), row(0, "ascii text"));
-
-        // ================ timestamp ================
-        updateMV("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "123123123123");
-        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new Date(123123123123L)));
-        assertRows(execute("SELECT k, asciival from mv_timestampval WHERE timestampval = fromJson(?)", "123123123123"), row(0, "ascii text"));
-
-        updateMV("INSERT INTO %s (k, timestampval) VALUES (?, fromJson(?))", 0, "\"2014-01-01\"");
-        assertRows(execute("SELECT k, timestampval FROM %s WHERE k = ?", 0), row(0, new SimpleDateFormat("y-M-d").parse("2014-01-01")));
-        assertRows(execute("SELECT k, asciival from mv_timestampval WHERE timestampval = fromJson(?)", "\"2014-01-01\""), row(0, "ascii text"));
-
-        // ================ timeuuid ================
-        updateMV("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
-        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-
-        updateMV("INSERT INTO %s (k, timeuuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
-        assertRows(execute("SELECT k, timeuuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-        assertRows(execute("SELECT k, asciival from mv_timeuuidval WHERE timeuuidval = fromJson(?)", "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""), row(0, "ascii text"));
-
-        // ================ uuidval ================
-        updateMV("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6bddc89a-5644-11e4-97fc-56847afe9799\"");
-        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-
-        updateMV("INSERT INTO %s (k, uuidval) VALUES (?, fromJson(?))", 0, "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\"");
-        assertRows(execute("SELECT k, uuidval FROM %s WHERE k = ?", 0), row(0, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")));
-        assertRows(execute("SELECT k, asciival from mv_uuidval WHERE uuidval = fromJson(?)", "\"6BDDC89A-5644-11E4-97FC-56847AFE9799\""), row(0, "ascii text"));
-
-        // ================ varint ================
-        updateMV("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "123123123123");
-        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("123123123123")));
-        assertRows(execute("SELECT k, asciival from mv_varintval WHERE varintval = fromJson(?)", "123123123123"), row(0, "ascii text"));
-
-        // accept strings for numbers that cannot be represented as longs
-        updateMV("INSERT INTO %s (k, varintval) VALUES (?, fromJson(?))", 0, "\"1234567890123456789012345678901234567890\"");
-        assertRows(execute("SELECT k, varintval FROM %s WHERE k = ?", 0), row(0, new BigInteger("1234567890123456789012345678901234567890")));
-        assertRows(execute("SELECT k, asciival from mv_varintval WHERE varintval = fromJson(?)", "\"1234567890123456789012345678901234567890\""), row(0, "ascii text"));
-
-        // ================ lists ================
-        updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2, 3)));
-
-        updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[1]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1)));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1)));
-
-        updateMV("UPDATE %s SET listval = listval + fromJson(?) WHERE k = ?", "[2]", 0);
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(1, 2)));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2)));
-
-        updateMV("UPDATE %s SET listval = fromJson(?) + listval WHERE k = ?", "[0]", 0);
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 1, 2)));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 1, 2)));
-
-        updateMV("UPDATE %s SET listval[1] = fromJson(?) WHERE k = ?", "10", 0);
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 10, 2)));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 10, 2)));
-
-        updateMV("DELETE listval[1] FROM %s WHERE k = ?", 0);
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, list(0, 2)));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(0, 2)));
-
-        updateMV("INSERT INTO %s (k, listval) VALUES (?, fromJson(?))", 0, "[]");
-        assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0), row(0, null));
-        assertRows(execute("SELECT k, listval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, null));
-
-        // frozen
-        updateMV("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[1, 2, 3]");
-        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(1, 2, 3)));
-        assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(1, 2, 3)));
-        assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[1, 2, 3]"), row(0, "abcd"));
-
-        updateMV("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[3, 2, 1]");
-        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list(3, 2, 1)));
-        assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[1, 2, 3]"));
-        assertRows(execute("SELECT k, textval from mv_frozenlistval where frozenlistval = fromJson(?)", "[3, 2, 1]"), row(0, "abcd"));
-        assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list(3, 2, 1)));
-
-        updateMV("INSERT INTO %s (k, frozenlistval) VALUES (?, fromJson(?))", 0, "[]");
-        assertRows(execute("SELECT k, frozenlistval FROM %s WHERE k = ?", 0), row(0, list()));
-        assertRows(execute("SELECT k, frozenlistval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, list()));
-
-        // ================ sets ================
-        updateMV("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
-                 0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
-
-        // duplicates are okay, just like in CQL
-        updateMV("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))",
-                 0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
-
-        updateMV("UPDATE %s SET setval = setval + fromJson(?) WHERE k = ?", "[\"6bddc89a-5644-0000-97fc-56847afe9799\"]", 0);
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
-                   row(0, set(UUID.fromString("6bddc89a-5644-0000-97fc-56847afe9799"), UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, set(UUID.fromString("6bddc89a-5644-0000-97fc-56847afe9799"), UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
-
-        updateMV("UPDATE %s SET setval = setval - fromJson(?) WHERE k = ?", "[\"6bddc89a-5644-0000-97fc-56847afe9799\"]", 0);
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
-
-        updateMV("INSERT INTO %s (k, setval) VALUES (?, fromJson(?))", 0, "[]");
-        assertRows(execute("SELECT k, setval FROM %s WHERE k = ?", 0), row(0, null));
-        assertRows(execute("SELECT k, setval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, null));
-
-
-        // frozen
-        updateMV("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
-                 0, "[\"6bddc89a-5644-11e4-97fc-56847afe9798\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))))
-        );
-        assertRows(execute("SELECT k, frozensetval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, set(UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))));
-
-        updateMV("INSERT INTO %s (k, frozensetval) VALUES (?, fromJson(?))",
-                 0, "[\"6bddc89a-0000-11e4-97fc-56847afe9799\", \"6bddc89a-5644-11e4-97fc-56847afe9798\"]");
-        assertRows(execute("SELECT k, frozensetval FROM %s WHERE k = ?", 0),
-                   row(0, set(UUID.fromString("6bddc89a-0000-11e4-97fc-56847afe9799"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798"))))
-        );
-        assertRows(execute("SELECT k, frozensetval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, set(UUID.fromString("6bddc89a-0000-11e4-97fc-56847afe9799"), (UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9798")))));
-
-        // ================ maps ================
-        updateMV("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
-        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""), row(0, map("a", 1, "b", 2)));
-
-        updateMV("UPDATE %s SET mapval[?] = ?  WHERE k = ?", "c", 3, 0);
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0),
-                   row(0, map("a", 1, "b", 2, "c", 3))
-        );
-        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, map("a", 1, "b", 2, "c", 3)));
-
-        updateMV("UPDATE %s SET mapval[?] = ?  WHERE k = ?", "b", 10, 0);
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0),
-                   row(0, map("a", 1, "b", 10, "c", 3))
-        );
-        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, map("a", 1, "b", 10, "c", 3)));
-
-        updateMV("DELETE mapval[?] FROM %s WHERE k = ?", "b", 0);
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0),
-                   row(0, map("a", 1, "c", 3))
-        );
-        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, map("a", 1, "c", 3)));
-
-        updateMV("INSERT INTO %s (k, mapval) VALUES (?, fromJson(?))", 0, "{}");
-        assertRows(execute("SELECT k, mapval FROM %s WHERE k = ?", 0), row(0, null));
-        assertRows(execute("SELECT k, mapval from mv_textval WHERE textval = fromJson(?)", "\"abcd\""),
-                   row(0, null));
-
-        // frozen
-        updateMV("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": 2}");
-        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 1, "b", 2)));
-        assertRows(execute("SELECT k, textval FROM mv_frozenmapval WHERE frozenmapval = fromJson(?)", "{\"a\": 1, \"b\": 2}"), row(0, "abcd"));
-
-        updateMV("INSERT INTO %s (k, frozenmapval) VALUES (?, fromJson(?))", 0, "{\"b\": 2, \"a\": 3}");
-        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 3, "b", 2)));
-        assertRows(execute("SELECT k, frozenmapval FROM %s WHERE k = ?", 0), row(0, map("a", 3, "b", 2)));
-
-        // ================ tuples ================
-        updateMV("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, \"foobar\", \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
-                   row(0, tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
-        );
-        assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))),
-                   row(0, "abcd"));
-
-        updateMV("INSERT INTO %s (k, tupleval) VALUES (?, fromJson(?))", 0, "[1, null, \"6bddc89a-5644-11e4-97fc-56847afe9799\"]");
-        assertRows(execute("SELECT k, tupleval FROM %s WHERE k = ?", 0),
-                   row(0, tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799")))
-        );
-        assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, "foobar", UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))));
-        assertRows(execute("SELECT k, textval FROM mv_tupleval WHERE tupleval = ?", tuple(1, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"))),
-                   row(0, "abcd"));
-
-        // ================ UDTs ================
-        updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                   row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
-        );
-        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"),
-                   row(0, "abcd"));
-
-        // order of fields shouldn't matter
-        updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"a\": 1, \"c\": [\"foo\", \"bar\"]}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                   row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
-        );
-        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"),
-                   row(0, "abcd"));
-
-        // test nulls
-        updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                   row(0, null, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), set("bar", "foo"))
-        );
-        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"));
-        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"),
-                   row(0, "abcd"));
-
-        // test missing fields
-        updateMV("INSERT INTO %s (k, udtval) VALUES (?, fromJson(?))", 0, "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}");
-        assertRows(execute("SELECT k, udtval.a, udtval.b, udtval.c FROM %s WHERE k = ?", 0),
-                   row(0, 1, UUID.fromString("6bddc89a-5644-11e4-97fc-56847afe9799"), null)
-        );
-        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": null, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\", \"c\": [\"foo\", \"bar\"]}"));
-        assertRows(execute("SELECT k, textval FROM mv_udtval WHERE udtval = fromJson(?)", "{\"a\": 1, \"b\": \"6bddc89a-5644-11e4-97fc-56847afe9799\"}"),
-                   row(0, "abcd"));
-    }
-
-
-    @Test
-    public void ttlTest() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b int," +
-                    "c int," +
-                    "d int," +
-                    "PRIMARY KEY (a, b))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
-
-        updateMV("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TTL 5", 1, 1, 1, 1);
-
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
-        updateMV("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2);
-
-        Thread.sleep(TimeUnit.SECONDS.toMillis(3));
-        List<Row> results = executeNet(protocolVersion, "SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all();
-        Assert.assertEquals(1, results.size());
-        Assert.assertTrue("There should be a null result given back due to ttl expiry", results.get(0).isNull(0));
-    }
-
-    @Test
-    public void conflictingTimestampTest() throws Throwable
-    {
-        createTable("CREATE TABLE %s (" +
-                    "a int," +
-                    "b int," +
-                    "c int," +
-                    "PRIMARY KEY (a, b))");
-
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE c IS NOT NULL AND a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (c, a, b)");
-
-        for (int i = 0; i < 50; i++)
-        {
-            updateMV("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP 1", 1, 1, i);
-        }
-
-        ResultSet mvRows = executeNet(protocolVersion, "SELECT c FROM mv");
-        List<Row> rows = executeNet(protocolVersion, "SELECT c FROM %s").all();
-        Assert.assertEquals("There should be exactly one row in base", 1, rows.size());
-        int expected = rows.get(0).getInt("c");
-        assertRowsNet(protocolVersion, mvRows, row(expected));
-    }
-}