You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ja...@apache.org on 2015/07/29 22:16:15 UTC

[2/5] cassandra git commit: Materialized Views

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/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 b7db989..7311069 100644
--- a/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
+++ b/src/java/org/apache/cassandra/streaming/StreamReceiveTask.java
@@ -24,14 +24,25 @@ import java.util.UUID;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import org.apache.cassandra.concurrent.NamedThreadFactory;
 import org.apache.cassandra.config.Schema;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.lifecycle.LifecycleTransaction;
 import org.apache.cassandra.db.compaction.OperationType;
+import org.apache.cassandra.db.Mutation;
+import org.apache.cassandra.db.partitions.PartitionUpdate;
+import org.apache.cassandra.db.rows.RowIterators;
+import org.apache.cassandra.db.rows.Unfiltered;
+import org.apache.cassandra.db.rows.UnfilteredRowIterator;
+import org.apache.cassandra.db.rows.UnfilteredRowIterators;
+import org.apache.cassandra.io.sstable.ISSTableScanner;
 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;
@@ -41,6 +52,8 @@ import org.apache.cassandra.utils.concurrent.Refs;
  */
 public class StreamReceiveTask extends StreamTask
 {
+    private static final Logger logger = LoggerFactory.getLogger(StreamReceiveTask.class);
+
     private static final ExecutorService executor = Executors.newCachedThreadPool(new NamedThreadFactory("StreamReceiveTask"));
 
     // number of files to receive
@@ -120,21 +133,69 @@ public class StreamReceiveTask extends StreamTask
                 return;
             }
             ColumnFamilyStore cfs = Keyspace.open(kscf.left).getColumnFamilyStore(kscf.right);
+            boolean hasMaterializedViews = cfs.materializedViewManager.allViews().iterator().hasNext();
+
+            try
+            {
+                List<SSTableReader> readers = new ArrayList<>();
+                for (SSTableWriter writer : task.sstables)
+                {
+                    SSTableReader reader = writer.finish(true);
+                    readers.add(reader);
+                    task.txn.update(reader, false);
+                }
 
-            List<SSTableReader> readers = new ArrayList<>();
-            for (SSTableWriter writer : task.sstables)
-                readers.add(writer.finish(true));
-            task.txn.finish();
-            task.sstables.clear();
+                task.sstables.clear();
 
-            try (Refs<SSTableReader> refs = Refs.ref(readers))
+                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
+                    //all partitions through the same write path as normal mutations.
+                    //This also ensures any 2is are also updated
+                    if (hasMaterializedViews)
+                    {
+                        for (SSTableReader reader : readers)
+                        {
+                            try (ISSTableScanner scanner = reader.getScanner())
+                            {
+                                while (scanner.hasNext())
+                                {
+                                    try (UnfilteredRowIterator rowIterator = scanner.next())
+                                    {
+                                        new Mutation(PartitionUpdate.fromIterator(rowIterator)).apply();
+                                    }
+                                }
+                            }
+                        }
+                    }
+                    else
+                    {
+                        task.txn.finish();
+
+                        // add sstables and build secondary indexes
+                        cfs.addSSTables(readers);
+                        cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames());
+                    }
+                }
+                catch (Throwable t)
+                {
+                    logger.error("Error applying streamed sstable: ", t);
+
+                    JVMStabilityInspector.inspectThrowable(t);
+                }
+                finally
+                {
+                    //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)
+                        task.txn.abort();
+                }
+            }
+            finally
             {
-                // add sstables and build secondary indexes
-                cfs.addSSTables(readers);
-                cfs.indexManager.maybeBuildSecondaryIndexes(readers, cfs.indexManager.allIndexesNames());
+                task.session.taskCompleted(task);
             }
-
-            task.session.taskCompleted(task);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/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 b38f58e..0674c62 100644
--- a/src/java/org/apache/cassandra/thrift/CassandraServer.java
+++ b/src/java/org/apache/cassandra/thrift/CassandraServer.java
@@ -805,6 +805,9 @@ 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())
+            throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
+
         ThriftValidation.validateKey(metadata, key);
         ThriftValidation.validateColumnParent(metadata, column_parent);
         // SuperColumn field is usually optional, but not when we're inserting
@@ -898,6 +901,9 @@ public class CassandraServer implements Cassandra.Iface
             cState.hasColumnFamilyAccess(keyspace, column_family, Permission.SELECT);
 
             CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_family, false);
+            if (metadata.isMaterializedView())
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
+
             ThriftValidation.validateKey(metadata, key);
             if (metadata.isSuper())
                 throw new org.apache.cassandra.exceptions.InvalidRequestException("CAS does not support supercolumns");
@@ -1090,6 +1096,9 @@ public class CassandraServer implements Cassandra.Iface
                 cState.hasColumnFamilyAccess(keyspace, cfName, Permission.MODIFY);
 
                 CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfName);
+                if (metadata.isMaterializedView())
+                    throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
+
                 ThriftValidation.validateKey(metadata, key);
                 if (metadata.isCounter())
                     ThriftConversion.fromThrift(consistency_level).validateCounterForWrite(metadata);
@@ -1303,6 +1312,9 @@ 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())
+            throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
+
         ThriftValidation.validateKey(metadata, key);
         ThriftValidation.validateColumnPathOrParent(metadata, column_path);
         if (isCommutativeOp)
@@ -1872,6 +1884,11 @@ public class CassandraServer implements Cassandra.Iface
         {
             String keyspace = cState.getKeyspace();
             cState.hasColumnFamilyAccess(keyspace, column_family, Permission.DROP);
+
+            CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, column_family);
+            if (metadata.isMaterializedView())
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot drop Materialized Views from Thrift");
+
             MigrationManager.announceColumnFamilyDrop(keyspace, column_family);
             return Schema.instance.getVersion().toString();
         }
@@ -1982,6 +1999,11 @@ public class CassandraServer implements Cassandra.Iface
             if (oldCfm == null)
                 throw new InvalidRequestException("Could not find table definition to modify.");
 
+            if (oldCfm.isMaterializedView())
+                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())
+                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())
                 throw new InvalidRequestException("Cannot modify CQL3 table " + oldCfm.cfName + " as it may break the schema. You should use cqlsh to modify CQL3 tables instead.");
 
@@ -2009,6 +2031,9 @@ public class CassandraServer implements Cassandra.Iface
         {
             String keyspace = cState.getKeyspace();
             cState.hasColumnFamilyAccess(keyspace, cfname, Permission.MODIFY);
+            CFMetaData metadata = ThriftValidation.validateColumnFamily(keyspace, cfname, true);
+            if (metadata.isMaterializedView())
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot truncate Materialized Views");
 
             if (startSessionIfRequested())
             {
@@ -2094,6 +2119,9 @@ 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())
+                throw new org.apache.cassandra.exceptions.InvalidRequestException("Cannot modify Materialized Views directly");
+
             ThriftValidation.validateKey(metadata, key);
             ThriftConversion.fromThrift(consistency_level).validateCounterForWrite(metadata);
             ThriftValidation.validateColumnParent(metadata, column_parent);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/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 c9e5062..20d6eba 100644
--- a/src/java/org/apache/cassandra/thrift/ThriftConversion.java
+++ b/src/java/org/apache/cassandra/thrift/ThriftConversion.java
@@ -274,7 +274,10 @@ public class ThriftConversion
                                       subComparator,
                                       defaultValidator);
 
-            CFMetaData newCFMD = CFMetaData.create(cf_def.keyspace, cf_def.name, cfId, isDense, isCompound, isSuper, isCounter, defs);
+            // We do not allow Thrift materialized views, so we always set it to false
+            boolean isMaterializedView = false;
+
+            CFMetaData newCFMD = CFMetaData.create(cf_def.keyspace, cf_def.name, cfId, isDense, isCompound, isSuper, isCounter, isMaterializedView, defs);
 
             if (cf_def.isSetGc_grace_seconds())
                 newCFMD.gcGraceSeconds(cf_def.gc_grace_seconds);

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/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 d9c268b..5354b43 100644
--- a/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
+++ b/src/java/org/apache/cassandra/utils/NativeSSTableLoaderClient.java
@@ -118,6 +118,7 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
             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.MATERIALIZEDVIEW);
 
             String columnsQuery = String.format("SELECT * FROM %s.%s WHERE keyspace_name = ? AND table_name = ?",
                                                 SchemaKeyspace.NAME,
@@ -127,7 +128,7 @@ public class NativeSSTableLoaderClient extends SSTableLoader.Client
             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, defs));
+            tables.put(name, CFMetaData.create(keyspace, name, id, isDense, isCompound, isSuper, isCounter, isMaterializedView, defs));
         }
 
         return tables;

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/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
new file mode 100644
index 0000000..70ec451
--- /dev/null
+++ b/test/long/org/apache/cassandra/cql3/MaterializedViewLongTest.java
@@ -0,0 +1,194 @@
+/*
+ * 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.db.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 (?, ?, ?)",
+                                           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)
+            {
+
+            }
+        }
+
+        while (!(((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getPendingTasks() == 0
+                 && ((SEPExecutor) StageManager.getStage(Stage.MATERIALIZED_VIEW_MUTATION)).getActiveCount() == 0))
+        {
+            Thread.sleep(1);
+        }
+
+        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/3bdcaa33/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java b/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java
deleted file mode 100644
index 575036f..0000000
--- a/test/unit/org/apache/cassandra/AbstractReadCommandBuilder.java
+++ /dev/null
@@ -1,327 +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;
-
-import java.nio.ByteBuffer;
-import java.util.*;
-
-import org.apache.cassandra.config.ColumnDefinition;
-import org.apache.cassandra.cql3.*;
-import org.apache.cassandra.db.*;
-import org.apache.cassandra.db.rows.Row;
-import org.apache.cassandra.db.filter.*;
-import org.apache.cassandra.db.partitions.*;
-import org.apache.cassandra.db.marshal.*;
-import org.apache.cassandra.dht.*;
-import org.apache.cassandra.service.StorageService;
-import org.apache.cassandra.utils.FBUtilities;
-
-public abstract class AbstractReadCommandBuilder
-{
-    protected final ColumnFamilyStore cfs;
-    protected int nowInSeconds;
-
-    private int cqlLimit = -1;
-    private int pagingLimit = -1;
-    private boolean reversed = false;
-
-    private Set<ColumnIdentifier> columns;
-    protected final RowFilter filter = RowFilter.create();
-
-    private Slice.Bound lowerClusteringBound;
-    private Slice.Bound upperClusteringBound;
-
-    private NavigableSet<Clustering> clusterings;
-
-    // Use Util.cmd() instead of this ctor directly
-    AbstractReadCommandBuilder(ColumnFamilyStore cfs)
-    {
-        this.cfs = cfs;
-        this.nowInSeconds = FBUtilities.nowInSeconds();
-    }
-
-    public AbstractReadCommandBuilder withNowInSeconds(int nowInSec)
-    {
-        this.nowInSeconds = nowInSec;
-        return this;
-    }
-
-    public AbstractReadCommandBuilder fromIncl(Object... values)
-    {
-        assert lowerClusteringBound == null && clusterings == null;
-        this.lowerClusteringBound = Slice.Bound.create(cfs.metadata.comparator, true, true, values);
-        return this;
-    }
-
-    public AbstractReadCommandBuilder fromExcl(Object... values)
-    {
-        assert lowerClusteringBound == null && clusterings == null;
-        this.lowerClusteringBound = Slice.Bound.create(cfs.metadata.comparator, true, false, values);
-        return this;
-    }
-
-    public AbstractReadCommandBuilder toIncl(Object... values)
-    {
-        assert upperClusteringBound == null && clusterings == null;
-        this.upperClusteringBound = Slice.Bound.create(cfs.metadata.comparator, false, true, values);
-        return this;
-    }
-
-    public AbstractReadCommandBuilder toExcl(Object... values)
-    {
-        assert upperClusteringBound == null && clusterings == null;
-        this.upperClusteringBound = Slice.Bound.create(cfs.metadata.comparator, false, false, values);
-        return this;
-    }
-
-    public AbstractReadCommandBuilder includeRow(Object... values)
-    {
-        assert lowerClusteringBound == null && upperClusteringBound == null;
-
-        if (this.clusterings == null)
-            this.clusterings = new TreeSet<>(cfs.metadata.comparator);
-
-        this.clusterings.add(cfs.metadata.comparator.make(values));
-        return this;
-    }
-
-    public AbstractReadCommandBuilder reverse()
-    {
-        this.reversed = true;
-        return this;
-    }
-
-    public AbstractReadCommandBuilder withLimit(int newLimit)
-    {
-        this.cqlLimit = newLimit;
-        return this;
-    }
-
-    public AbstractReadCommandBuilder withPagingLimit(int newLimit)
-    {
-        this.pagingLimit = newLimit;
-        return this;
-    }
-
-    public AbstractReadCommandBuilder columns(String... columns)
-    {
-        if (this.columns == null)
-            this.columns = new HashSet<>();
-
-        for (String column : columns)
-            this.columns.add(ColumnIdentifier.getInterned(column, true));
-        return this;
-    }
-
-    private ByteBuffer bb(Object value, AbstractType<?> type)
-    {
-        return value instanceof ByteBuffer ? (ByteBuffer)value : ((AbstractType)type).decompose(value);
-    }
-
-    private AbstractType<?> forValues(AbstractType<?> collectionType)
-    {
-        assert collectionType instanceof CollectionType;
-        CollectionType ct = (CollectionType)collectionType;
-        switch (ct.kind)
-        {
-            case LIST:
-            case MAP:
-                return ct.valueComparator();
-            case SET:
-                return ct.nameComparator();
-        }
-        throw new AssertionError();
-    }
-
-    private AbstractType<?> forKeys(AbstractType<?> collectionType)
-    {
-        assert collectionType instanceof CollectionType;
-        CollectionType ct = (CollectionType)collectionType;
-        switch (ct.kind)
-        {
-            case LIST:
-            case MAP:
-                return ct.nameComparator();
-        }
-        throw new AssertionError();
-    }
-
-    public AbstractReadCommandBuilder filterOn(String column, Operator op, Object value)
-    {
-        ColumnDefinition def = cfs.metadata.getColumnDefinition(ColumnIdentifier.getInterned(column, true));
-        assert def != null;
-
-        AbstractType<?> type = def.type;
-        if (op == Operator.CONTAINS)
-            type = forValues(type);
-        else if (op == Operator.CONTAINS_KEY)
-            type = forKeys(type);
-
-        this.filter.add(def, op, bb(value, type));
-        return this;
-    }
-
-    protected ColumnFilter makeColumnFilter()
-    {
-        if (columns == null)
-            return ColumnFilter.all(cfs.metadata);
-
-        ColumnFilter.Builder builder = ColumnFilter.allColumnsBuilder(cfs.metadata);
-        for (ColumnIdentifier column : columns)
-            builder.add(cfs.metadata.getColumnDefinition(column));
-        return builder.build();
-    }
-
-    protected ClusteringIndexFilter makeFilter()
-    {
-        if (clusterings != null)
-        {
-            return new ClusteringIndexNamesFilter(clusterings, reversed);
-        }
-        else
-        {
-            Slice slice = Slice.make(lowerClusteringBound == null ? Slice.Bound.BOTTOM : lowerClusteringBound,
-                                     upperClusteringBound == null ? Slice.Bound.TOP : upperClusteringBound);
-            return new ClusteringIndexSliceFilter(Slices.with(cfs.metadata.comparator, slice), reversed);
-        }
-    }
-
-    protected DataLimits makeLimits()
-    {
-        DataLimits limits = cqlLimit < 0 ? DataLimits.NONE : DataLimits.cqlLimits(cqlLimit);
-        if (pagingLimit >= 0)
-            limits = limits.forPaging(pagingLimit);
-        return limits;
-    }
-
-    public Row getOnlyRow()
-    {
-        return Util.getOnlyRow(build());
-    }
-
-    public Row getOnlyRowUnfiltered()
-    {
-        return Util.getOnlyRowUnfiltered(build());
-    }
-
-    public FilteredPartition getOnlyPartition()
-    {
-        return Util.getOnlyPartition(build());
-    }
-
-    public Partition getOnlyPartitionUnfiltered()
-    {
-        return Util.getOnlyPartitionUnfiltered(build());
-    }
-
-    public abstract ReadCommand build();
-
-    public static class SinglePartitionBuilder extends AbstractReadCommandBuilder
-    {
-        private final DecoratedKey partitionKey;
-
-        SinglePartitionBuilder(ColumnFamilyStore cfs, DecoratedKey key)
-        {
-            super(cfs);
-            this.partitionKey = key;
-        }
-
-        @Override
-        public ReadCommand build()
-        {
-            return SinglePartitionReadCommand.create(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), partitionKey, makeFilter());
-        }
-    }
-
-    public static class PartitionRangeBuilder extends AbstractReadCommandBuilder
-    {
-        private DecoratedKey startKey;
-        private boolean startInclusive;
-        private DecoratedKey endKey;
-        private boolean endInclusive;
-
-        PartitionRangeBuilder(ColumnFamilyStore cfs)
-        {
-            super(cfs);
-        }
-
-        public PartitionRangeBuilder fromKeyIncl(Object... values)
-        {
-            assert startKey == null;
-            this.startInclusive = true;
-            this.startKey = Util.makeKey(cfs.metadata, values);
-            return this;
-        }
-
-        public PartitionRangeBuilder fromKeyExcl(Object... values)
-        {
-            assert startKey == null;
-            this.startInclusive = false;
-            this.startKey = Util.makeKey(cfs.metadata, values);
-            return this;
-        }
-
-        public PartitionRangeBuilder toKeyIncl(Object... values)
-        {
-            assert endKey == null;
-            this.endInclusive = true;
-            this.endKey = Util.makeKey(cfs.metadata, values);
-            return this;
-        }
-
-        public PartitionRangeBuilder toKeyExcl(Object... values)
-        {
-            assert endKey == null;
-            this.endInclusive = false;
-            this.endKey = Util.makeKey(cfs.metadata, values);
-            return this;
-        }
-
-        @Override
-        public ReadCommand build()
-        {
-            PartitionPosition start = startKey;
-            if (start == null)
-            {
-                start = StorageService.getPartitioner().getMinimumToken().maxKeyBound();
-                startInclusive = false;
-            }
-            PartitionPosition end = endKey;
-            if (end == null)
-            {
-                end = StorageService.getPartitioner().getMinimumToken().maxKeyBound();
-                endInclusive = true;
-            }
-            
-            AbstractBounds<PartitionPosition> bounds;
-            if (startInclusive && endInclusive)
-                bounds = new Bounds<PartitionPosition>(start, end);
-            else if (startInclusive && !endInclusive)
-                bounds = new IncludingExcludingBounds<PartitionPosition>(start, end);
-            else if (!startInclusive && endInclusive)
-                bounds = new Range<PartitionPosition>(start, end);
-            else
-                bounds = new ExcludingBounds<PartitionPosition>(start, end);
-
-            return new PartitionRangeReadCommand(cfs.metadata, nowInSeconds, makeColumnFilter(), filter, makeLimits(), new DataRange(bounds, makeFilter()));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/test/unit/org/apache/cassandra/cql3/CQLTester.java
----------------------------------------------------------------------
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index ddc41c7..458d7dd 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -760,8 +760,8 @@ public abstract class CQLTester
 
                 if (!Objects.equal(expectedByteValue, actualValue))
                 {
-                    Object actualValueDecoded = column.type.getSerializer().deserialize(actualValue);
-                    if (!actualValueDecoded.equals(expected[j]))
+                    Object actualValueDecoded = actualValue == null ? null : column.type.getSerializer().deserialize(actualValue);
+                    if (!expected[j].equals(actualValueDecoded))
                         Assert.fail(String.format("Invalid value for row %d column %d (%s of type %s), expected <%s> but got <%s>",
                                                   i,
                                                   j,

http://git-wip-us.apache.org/repos/asf/cassandra/blob/3bdcaa33/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
new file mode 100644
index 0000000..78c40e2
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/MaterializedViewTest.java
@@ -0,0 +1,955 @@
+/*
+ * 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.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;
+
+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 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 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 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));
+    }
+}