You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by ad...@apache.org on 2021/11/10 12:38:09 UTC

[cassandra] branch trunk updated: Add MV utilities to CQLTester

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

adelapena pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 42b284d  Add MV utilities to CQLTester
42b284d is described below

commit 42b284d5c957675f7b551e36a75a0cc043753530
Author: Andrés de la Peña <a....@gmail.com>
AuthorDate: Fri Nov 5 17:11:09 2021 +0000

    Add MV utilities to CQLTester
    
    patch by Andrés de la Peña and Zhao Yang; reviewed by Berenguer Blasi for CASSANDRA-17122
    
    Co-authored-by: Andrés de la Peña <a....@gmail.com>
    Co-authored-by: Zhao Yang <ja...@gmail.com>
---
 .../org/apache/cassandra/cql3/ViewLongTest.java    | 193 ++----
 test/unit/org/apache/cassandra/cql3/CQLTester.java | 153 ++++-
 .../cql3/ViewAbstractParameterizedTest.java        |  86 +++
 .../apache/cassandra/cql3/ViewAbstractTest.java    |  75 +--
 .../cassandra/cql3/ViewComplexDeletionsTest.java   | 174 +++--
 .../cassandra/cql3/ViewComplexLivenessTest.java    |  69 +-
 .../apache/cassandra/cql3/ViewComplexTTLTest.java  |  56 +-
 .../org/apache/cassandra/cql3/ViewComplexTest.java |  60 +-
 .../apache/cassandra/cql3/ViewComplexTester.java   | 130 ----
 .../cassandra/cql3/ViewComplexUpdatesTest.java     |  83 +--
 .../cql3/ViewFilteringClustering1Test.java         | 215 ++----
 .../cql3/ViewFilteringClustering2Test.java         | 168 ++---
 .../apache/cassandra/cql3/ViewFilteringPKTest.java | 344 +++-------
 .../apache/cassandra/cql3/ViewFilteringTest.java   | 741 +++++++++------------
 .../unit/org/apache/cassandra/cql3/ViewPKTest.java | 172 ++---
 .../org/apache/cassandra/cql3/ViewRangesTest.java  |  67 +-
 .../org/apache/cassandra/cql3/ViewSchemaTest.java  | 257 +++----
 test/unit/org/apache/cassandra/cql3/ViewTest.java  | 254 +++----
 .../org/apache/cassandra/cql3/ViewTimesTest.java   |  94 ++-
 19 files changed, 1387 insertions(+), 2004 deletions(-)

diff --git a/test/long/org/apache/cassandra/cql3/ViewLongTest.java b/test/long/org/apache/cassandra/cql3/ViewLongTest.java
index 7c7a688..cec4d21 100644
--- a/test/long/org/apache/cassandra/cql3/ViewLongTest.java
+++ b/test/long/org/apache/cassandra/cql3/ViewLongTest.java
@@ -18,86 +18,25 @@
 
 package org.apache.cassandra.cql3;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.CyclicBarrier;
-import java.util.stream.Collectors;
 
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
 
 import com.datastax.driver.core.Row;
 import com.datastax.driver.core.exceptions.NoHostAvailableException;
-import com.datastax.driver.core.exceptions.OperationTimedOutException;
 import com.datastax.driver.core.exceptions.WriteTimeoutException;
 import org.apache.cassandra.batchlog.BatchlogManager;
 import org.apache.cassandra.concurrent.NamedThreadFactory;
-import org.apache.cassandra.transport.ProtocolVersion;
-import org.apache.cassandra.concurrent.SEPExecutor;
-import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.WrappedRunnable;
 
-@RunWith(Parameterized.class)
-public class ViewLongTest extends CQLTester
+public class ViewLongTest extends ViewAbstractParameterizedTest
 {
-    @Parameterized.Parameter
-    public ProtocolVersion version;
-
-    @Parameterized.Parameters()
-    public static Collection<Object[]> versions()
-    {
-        return ProtocolVersion.SUPPORTED.stream()
-                                        .map(v -> new Object[]{v})
-                                        .collect(Collectors.toList());
-    }
-
-    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(version, "DROP MATERIALIZED VIEW " + viewName);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        try
-        {
-            executeNet(version, 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);
-        }
-        catch (OperationTimedOutException ex)
-        {
-            // ... except for timeout, when we actually do not know whether the view was created or not
-            views.add(name);
-            throw ex;
-        }
-    }
-
     @Test
     public void testConflictResolution() throws Throwable
     {
@@ -111,9 +50,9 @@ public class ViewLongTest extends CQLTester
                     "c int," +
                     "PRIMARY KEY (a, b))");
 
-        executeNet(version, "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)");
+        createView("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);
 
@@ -133,7 +72,7 @@ public class ViewLongTest extends CQLTester
                         {
                             try
                             {
-                                executeNet(version, "INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP 1",
+                                executeNet("INSERT INTO %s (a, b, c) VALUES (?, ?, ?) USING TIMESTAMP 1",
                                            1,
                                            1,
                                            i + writerOffset);
@@ -159,7 +98,7 @@ public class ViewLongTest extends CQLTester
 
         for (int i = 0; i < writers * insertsPerWriter; i++)
         {
-            if (executeNet(version, "SELECT COUNT(*) FROM system.batches").one().getLong(0) == 0)
+            if (executeNet("SELECT COUNT(*) FROM system.batches").one().getLong(0) == 0)
                 break;
             try
             {
@@ -173,9 +112,9 @@ public class ViewLongTest extends CQLTester
             }
         }
 
-        int value = executeNet(version, "SELECT c FROM %s WHERE a = 1 AND b = 1").one().getInt("c");
+        int value = executeNet("SELECT c FROM %s WHERE a = 1 AND b = 1").one().getInt("c");
 
-        List<Row> rows = executeNet(version, "SELECT c FROM " + keyspace() + ".mv").all();
+        List<Row> rows = executeNet("SELECT c FROM " + keyspace() + "." + currentView()).all();
 
         boolean containsC = false;
         StringBuilder others = new StringBuilder();
@@ -203,13 +142,19 @@ public class ViewLongTest extends CQLTester
 
         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));
+            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)
+        else if (!containsC)
         {
             throw new AssertionError(String.format("Single row had c = %d, expected %d", rows.get(0).getInt("c"), value));
         }
@@ -231,104 +176,102 @@ public class ViewLongTest extends CQLTester
     {
         createTable("CREATE TABLE %s (field1 int,field2 int,date int,PRIMARY KEY ((field1), field2)) WITH default_time_to_live = 5;");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        createView("mv",
-                   "CREATE MATERIALIZED VIEW mv AS SELECT * FROM %%s WHERE field1 IS NOT NULL AND field2 IS NOT NULL AND date IS NOT NULL PRIMARY KEY ((field1), date, field2) WITH CLUSTERING ORDER BY (date desc, field2 asc);");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE field1 IS NOT NULL AND field2 IS NOT NULL AND date IS NOT NULL " +
+                   "PRIMARY KEY ((field1), date, field2) WITH CLUSTERING ORDER BY (date desc, field2 asc)");
 
         updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 111);", flush);
         assertRows(execute("select * from %s"), row(1, 2, 111));
-        assertRows(execute("select * from mv"), row(1, 111, 2));
+        assertRows(executeView("select * from %s"), row(1, 111, 2));
 
         updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 222);", flush);
         assertRows(execute("select * from %s"), row(1, 2, 222));
-        assertRows(execute("select * from mv"), row(1, 222, 2));
+        assertRows(executeView("select * from %s"), row(1, 222, 2));
 
         updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 333);", flush);
 
         assertRows(execute("select * from %s"), row(1, 2, 333));
-        assertRows(execute("select * from mv"), row(1, 333, 2));
+        assertRows(executeView("select * from %s"), row(1, 333, 2));
 
         if (flush)
         {
-            Keyspace.open(keyspace()).getColumnFamilyStore("mv").forceMajorCompaction();
+            Keyspace.open(keyspace()).getColumnFamilyStore(currentView()).forceMajorCompaction();
             assertRows(execute("select * from %s"), row(1, 2, 333));
-            assertRows(execute("select * from mv"), row(1, 333, 2));
+            assertRows(executeView("select * from %s"), row(1, 333, 2));
         }
 
         // wait for ttl, data should be removed
         updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 444);", flush);
         assertRows(execute("select * from %s"), row(1, 2, 444));
-        assertRows(execute("select * from mv"), row(1, 444, 2));
+        assertRows(executeView("select * from %s"), row(1, 444, 2));
 
         Thread.sleep(5000);
         assertRows(execute("select * from %s"));
-        assertRows(execute("select * from mv"));
+        assertRows(executeView("select * from %s"));
 
         // shadow mv with date=555 and then update it back to live, wait for ttl
         updateView("update %s set date=555 where field1=1 and field2=2;");
         updateView("update %s set date=666 where field1=1 and field2=2;");
         updateViewWithFlush("update %s set date=555 where field1=1 and field2=2;", flush);
         assertRows(execute("select * from %s"), row(1, 2, 555));
-        assertRows(execute("select * from mv"), row(1, 555, 2));
+        assertRows(executeView("select * from %s"), row(1, 555, 2));
 
         Thread.sleep(5000);
         assertRows(execute("select * from %s"));
-        assertRows(execute("select * from mv"));
+        assertRows(executeView("select * from %s"));
 
         // test user-provided ttl for table with/without default-ttl
         for (boolean withDefaultTTL : Arrays.asList(true, false))
         {
             execute("TRUNCATE %s");
             if (withDefaultTTL)
-                execute("ALTER TABLE %s with default_time_to_live=" + (withDefaultTTL ? 10 : 0));
+                execute("ALTER TABLE %s with default_time_to_live=" + 10);
             updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 666) USING TTL 1000;", flush);
 
             assertRows(execute("select * from %s"), row(1, 2, 666));
-            assertRows(execute("select * from mv"), row(1, 666, 2));
+            assertRows(executeView("select * from %s"), row(1, 666, 2));
 
             updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 777) USING TTL 1100;", flush);
             assertRows(execute("select * from %s"), row(1, 2, 777));
-            assertRows(execute("select * from mv"), row(1, 777, 2));
+            assertRows(executeView("select * from %s"), row(1, 777, 2));
 
             updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 888) USING TTL 800;", flush);
 
             assertRows(execute("select * from %s"), row(1, 2, 888));
-            assertRows(execute("select * from mv"), row(1, 888, 2));
+            assertRows(executeView("select * from %s"), row(1, 888, 2));
 
             if (flush)
             {
-                Keyspace.open(keyspace()).getColumnFamilyStore("mv").forceMajorCompaction();
+                Keyspace.open(keyspace()).getColumnFamilyStore(currentView()).forceMajorCompaction();
                 assertRows(execute("select * from %s"), row(1, 2, 888));
-                assertRows(execute("select * from mv"), row(1, 888, 2));
+                assertRows(executeView("select * from %s"), row(1, 888, 2));
             }
 
             // wait for ttl, data should be removed
             updateViewWithFlush("insert into %s (field1, field2, date) values (1, 2, 999) USING TTL 5;", flush);
             assertRows(execute("select * from %s"), row(1, 2, 999));
-            assertRows(execute("select * from mv"), row(1, 999, 2));
+            assertRows(executeView("select * from %s"), row(1, 999, 2));
 
             Thread.sleep(5000);
             assertRows(execute("select * from %s"));
-            assertRows(execute("select * from mv"));
+            assertRows(executeView("select * from %s"));
 
             // shadow mv with date=555 and then update it back to live with ttl=5, wait for ttl to expire
             updateViewWithFlush("update %s  USING TTL 800 set date=555 where field1=1 and field2=2;", flush);
             assertRows(execute("select * from %s"), row(1, 2, 555));
-            assertRows(execute("select * from mv"), row(1, 555, 2));
+            assertRows(executeView("select * from %s"), row(1, 555, 2));
 
             updateViewWithFlush("update %s set date=666 where field1=1 and field2=2;", flush);
             assertRows(execute("select * from %s"), row(1, 2, 666));
-            assertRows(execute("select * from mv"), row(1, 666, 2));
+            assertRows(executeView("select * from %s"), row(1, 666, 2));
 
             updateViewWithFlush("update %s USING TTL 5 set date=555 where field1=1 and field2=2;", flush);
             assertRows(execute("select * from %s"), row(1, 2, 555));
-            assertRows(execute("select * from mv"), row(1, 555, 2));
+            assertRows(executeView("select * from %s"), row(1, 555, 2));
 
             Thread.sleep(5000);
             assertRows(execute("select * from %s"));
-            assertRows(execute("select * from mv"));
+            assertRows(executeView("select * from %s"));
         }
     }
 
@@ -348,35 +291,32 @@ public class ViewLongTest extends CQLTester
     {
         createTable("CREATE TABLE %s (k int,c int,a int, b int, PRIMARY KEY ((k), c)) WITH default_time_to_live = 1000;");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        createView("mv",
-                   "CREATE MATERIALIZED VIEW mv AS SELECT k,c,a FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL "
-                           + "PRIMARY KEY (c, k)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT k,c,a FROM %s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL " +
+                   "PRIMARY KEY (c, k)");
 
         // table default ttl
         updateViewWithFlush("UPDATE %s SET b = 111 WHERE k = 1 AND c = 2", flush);
         assertRows(execute("select k,c,a,b from %s"), row(1, 2, null, 111));
-        assertRows(execute("select k,c,a from mv"), row(1, 2, null));
+        assertRows(executeView("select k,c,a from %s"), row(1, 2, null));
 
         updateViewWithFlush("UPDATE %s SET b = null WHERE k = 1 AND c = 2", flush);
         assertRows(execute("select k,c,a,b from %s"));
-        assertRows(execute("select k,c,a from mv"));
+        assertRows(executeView("select k,c,a from %s"));
 
         updateViewWithFlush("UPDATE %s SET b = 222 WHERE k = 1 AND c = 2", flush);
         assertRows(execute("select k,c,a,b from %s"), row(1, 2, null, 222));
-        assertRows(execute("select k,c,a from mv"), row(1, 2, null));
+        assertRows(executeView("select k,c,a from %s"), row(1, 2, null));
 
         updateViewWithFlush("DELETE b FROM %s WHERE k = 1 AND c = 2", flush);
         assertRows(execute("select k,c,a,b from %s"));
-        assertRows(execute("select k,c,a from mv"));
+        assertRows(executeView("select k,c,a from %s"));
 
         if (flush)
         {
-            Keyspace.open(keyspace()).getColumnFamilyStore("mv").forceMajorCompaction();
+            Keyspace.open(keyspace()).getColumnFamilyStore(currentView()).forceMajorCompaction();
             assertRows(execute("select k,c,a,b from %s"));
-            assertRows(execute("select k,c,a from mv"));
+            assertRows(executeView("select k,c,a from %s"));
         }
 
         // test user-provided ttl for table with/without default-ttl
@@ -384,59 +324,42 @@ public class ViewLongTest extends CQLTester
         {
             execute("TRUNCATE %s");
             if (withDefaultTTL)
-                execute("ALTER TABLE %s with default_time_to_live=" + (withDefaultTTL ? 10 : 0));
+                execute("ALTER TABLE %s with default_time_to_live=" + 10);
 
             updateViewWithFlush("UPDATE %s USING TTL 100 SET b = 666 WHERE k = 1 AND c = 2", flush);
             assertRows(execute("select k,c,a,b from %s"), row(1, 2, null, 666));
-            assertRows(execute("select k,c,a from mv"), row(1, 2, null));
+            assertRows(executeView("select k,c,a from %s"), row(1, 2, null));
 
             updateViewWithFlush("UPDATE %s USING TTL 90  SET b = null WHERE k = 1 AND c = 2", flush);
             if (flush)
                 FBUtilities.waitOnFutures(Keyspace.open(keyspace()).flush());
             assertRows(execute("select k,c,a,b from %s"));
-            assertRows(execute("select k,c,a from mv"));
+            assertRows(executeView("select k,c,a from %s"));
 
             updateViewWithFlush("UPDATE %s USING TTL 80  SET b = 777 WHERE k = 1 AND c = 2", flush);
             assertRows(execute("select k,c,a,b from %s"), row(1, 2, null, 777));
-            assertRows(execute("select k,c,a from mv"), row(1, 2, null));
+            assertRows(executeView("select k,c,a from %s"), row(1, 2, null));
 
             updateViewWithFlush("DELETE b FROM %s WHERE k = 1 AND c = 2", flush);
             assertRows(execute("select k,c,a,b from %s"));
-            assertRows(execute("select k,c,a from mv"));
+            assertRows(executeView("select k,c,a from %s"));
 
             updateViewWithFlush("UPDATE %s USING TTL 110  SET b = 888 WHERE k = 1 AND c = 2", flush);
             assertRows(execute("select k,c,a,b from %s"), row(1, 2, null, 888));
-            assertRows(execute("select k,c,a from mv"), row(1, 2, null));
+            assertRows(executeView("select k,c,a from %s"), row(1, 2, null));
 
             updateViewWithFlush("UPDATE %s USING TTL 5  SET b = 999 WHERE k = 1 AND c = 2", flush);
             assertRows(execute("select k,c,a,b from %s"), row(1, 2, null, 999));
-            assertRows(execute("select k,c,a from mv"), row(1, 2, null));
+            assertRows(executeView("select k,c,a from %s"), row(1, 2, null));
 
             Thread.sleep(5000); // wait for ttl expired
 
             if (flush)
             {
-                Keyspace.open(keyspace()).getColumnFamilyStore("mv").forceMajorCompaction();
+                Keyspace.open(keyspace()).getColumnFamilyStore(currentView()).forceMajorCompaction();
                 assertRows(execute("select k,c,a,b from %s"));
-                assertRows(execute("select k,c,a from mv"));
+                assertRows(executeView("select k,c,a from %s"));
             }
         }
     }
-
-    private void updateView(String query, Object... params) throws Throwable
-    {
-        updateViewWithFlush(query, false, params);
-    }
-
-    private void updateViewWithFlush(String query, boolean flush, Object... params) throws Throwable
-    {
-        executeNet(version, query, params);
-        while (!(((SEPExecutor) Stage.VIEW_MUTATION.executor()).getPendingTaskCount() == 0
-                && ((SEPExecutor) Stage.VIEW_MUTATION.executor()).getActiveTaskCount() == 0))
-        {
-            Thread.sleep(1);
-        }
-        if (flush)
-            Keyspace.open(keyspace()).flush();
-    }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/CQLTester.java b/test/unit/org/apache/cassandra/cql3/CQLTester.java
index ea3c971..92762ce 100644
--- a/test/unit/org/apache/cassandra/cql3/CQLTester.java
+++ b/test/unit/org/apache/cassandra/cql3/CQLTester.java
@@ -67,6 +67,7 @@ import org.apache.cassandra.auth.AuthSchemaChangeListener;
 import org.apache.cassandra.auth.AuthTestUtils;
 import org.apache.cassandra.auth.IRoleManager;
 import org.apache.cassandra.concurrent.ScheduledExecutors;
+import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.virtual.VirtualKeyspaceRegistry;
 import org.apache.cassandra.db.virtual.VirtualSchemaKeyspace;
 import org.apache.cassandra.index.SecondaryIndexManager;
@@ -100,6 +101,7 @@ import org.apache.cassandra.utils.FBUtilities;
 import org.apache.cassandra.utils.JMXServerUtils;
 import org.assertj.core.api.Assertions;
 import org.apache.cassandra.utils.Pair;
+import org.awaitility.Awaitility;
 
 import static com.datastax.driver.core.SocketOptions.DEFAULT_CONNECT_TIMEOUT_MILLIS;
 import static com.datastax.driver.core.SocketOptions.DEFAULT_READ_TIMEOUT_MILLIS;
@@ -180,6 +182,7 @@ public abstract class CQLTester
 
     private List<String> keyspaces = new ArrayList<>();
     private List<String> tables = new ArrayList<>();
+    private List<String> views = new ArrayList<>();
     private List<String> types = new ArrayList<>();
     private List<String> functions = new ArrayList<>();
     private List<String> aggregates = new ArrayList<>();
@@ -358,11 +361,13 @@ public abstract class CQLTester
 
         final List<String> keyspacesToDrop = copy(keyspaces);
         final List<String> tablesToDrop = copy(tables);
+        final List<String> viewsToDrop = copy(views);
         final List<String> typesToDrop = copy(types);
         final List<String> functionsToDrop = copy(functions);
         final List<String> aggregatesToDrop = copy(aggregates);
         keyspaces = null;
         tables = null;
+        views = null;
         types = null;
         functions = null;
         aggregates = null;
@@ -375,6 +380,9 @@ public abstract class CQLTester
             {
                 try
                 {
+                    for (int i = viewsToDrop.size() - 1; i >= 0; i--)
+                        schemaChange(String.format("DROP MATERIALIZED VIEW IF EXISTS %s.%s", KEYSPACE, viewsToDrop.get(i)));
+
                     for (int i = tablesToDrop.size() - 1; i >= 0; i--)
                         schemaChange(String.format("DROP TABLE IF EXISTS %s.%s", KEYSPACE, tablesToDrop.get(i)));
 
@@ -705,6 +713,13 @@ public abstract class CQLTester
         return tables.get(tables.size() - 1);
     }
 
+    protected String currentView()
+    {
+        if (views.isEmpty())
+            return null;
+        return views.get(views.size() - 1);
+    }
+
     protected String currentKeyspace()
     {
         if (keyspaces.isEmpty())
@@ -862,6 +877,119 @@ public abstract class CQLTester
         QueryProcessor.executeOnceInternal(fullQuery);
     }
 
+    /**
+     * Creates a materialized view, waiting for the completion of its builder tasks.
+     *
+     * @param query the {@code CREATE VIEW} query, with {@code %s} placeholders for the view and table names
+     * @return the name of the created view
+     */
+    protected String createView(String query)
+    {
+        return createView(null, query);
+    }
+
+    /**
+     * Creates a materialized view, waiting for the completion of its builder tasks.
+     *
+     * @param viewName the name of the view to be created, or {@code null} for using an automatically generated a name
+     * @param query the {@code CREATE VIEW} query, with {@code %s} placeholders for the view and table names
+     * @return the name of the created view
+     */
+    protected String createView(String viewName, String query)
+    {
+        String currentView = createViewAsync(viewName, query);
+        waitForViewBuild(currentView);
+        return currentView;
+    }
+
+    /**
+     * Creates a materialized view, without waiting for the completion of its builder tasks.
+     *
+     * @param query the {@code CREATE VIEW} query, with {@code %s} placeholders for the view and table names
+     * @return the name of the created view
+     */
+    protected String createViewAsync(String query)
+    {
+        return createViewAsync(null, query);
+    }
+
+    /**
+     * Creates a materialized view, without waiting for the completion of its builder tasks.
+     *
+     * @param viewName the name of the view to be created, or {@code null} for using an automatically generated a name
+     * @param query the {@code CREATE VIEW} query, with {@code %s} placeholders for the view and table names
+     * @return the name of the created view
+     */
+    protected String createViewAsync(String viewName, String query)
+    {
+        String currentView = viewName == null ? createViewName() : viewName;
+        String fullQuery = String.format(query, KEYSPACE + "." + currentView, KEYSPACE + "." + currentTable());
+        logger.info(fullQuery);
+        schemaChange(fullQuery);
+        return currentView;
+    }
+
+    protected void dropView()
+    {
+        dropView(currentView());
+    }
+
+    protected void dropView(String view)
+    {
+        dropFormattedTable(String.format("DROP MATERIALIZED VIEW IF EXISTS %s.%s", KEYSPACE, view));
+        views.remove(view);
+    }
+
+    protected String createViewName()
+    {
+        String currentView = String.format("mv_%02d", seqNumber.getAndIncrement());
+        views.add(currentView);
+        return currentView;
+    }
+
+    protected List<String> getViews()
+    {
+        return copy(views);
+    }
+
+    protected void updateView(String query, Object... params) throws Throwable
+    {
+        updateView(getDefaultVersion(), query, params);
+    }
+
+    protected void updateView(ProtocolVersion version, String query, Object... params) throws Throwable
+    {
+        executeNet(version, query, params);
+        waitForViewMutations();
+    }
+
+    /**
+     * Waits for any pending asynchronous materialized view mutations.
+     */
+    protected static void waitForViewMutations()
+    {
+        Awaitility.await()
+                  .atMost(10, TimeUnit.MINUTES)
+                  .pollDelay(0, TimeUnit.MILLISECONDS)
+                  .pollInterval(1, TimeUnit.MILLISECONDS)
+                  .until(() -> Stage.VIEW_MUTATION.executor().getPendingTaskCount() == 0 &&
+                               Stage.VIEW_MUTATION.executor().getActiveTaskCount() == 0);
+    }
+
+    /**
+     * Waits for the building tasks of the specified materialized view.
+     *
+     * @param view the name of the view
+     */
+    protected void waitForViewBuild(String view)
+    {
+        Awaitility.await()
+                  .atMost(10, TimeUnit.MINUTES)
+                  .pollDelay(0, TimeUnit.MILLISECONDS)
+                  .pollInterval(10, TimeUnit.MILLISECONDS)
+                  .until(() -> SystemKeyspace.isViewBuilt(keyspace(), view));
+    }
+
     protected void alterTable(String query)
     {
         String fullQuery = formatQuery(query);
@@ -1080,17 +1208,22 @@ public abstract class CQLTester
         return sessionNet().execute(formatQuery(query), values);
     }
 
+    protected com.datastax.driver.core.ResultSet executeViewNet(String query, Object... values)
+    {
+        return sessionNet().execute(formatViewQuery(query), values);
+    }
+
     protected com.datastax.driver.core.ResultSet executeNet(ProtocolVersion protocolVersion, Statement statement)
     {
         return sessionNet(protocolVersion).execute(statement);
     }
 
-    protected com.datastax.driver.core.ResultSet executeNetWithPaging(ProtocolVersion version, String query, int pageSize) throws Throwable
+    protected com.datastax.driver.core.ResultSet executeNetWithPaging(ProtocolVersion version, String query, int pageSize)
     {
         return sessionNet(version).execute(new SimpleStatement(formatQuery(query)).setFetchSize(pageSize));
     }
 
-    protected com.datastax.driver.core.ResultSet executeNetWithPaging(String query, int pageSize) throws Throwable
+    protected com.datastax.driver.core.ResultSet executeNetWithPaging(String query, int pageSize)
     {
         return sessionNet().execute(new SimpleStatement(formatQuery(query)).setFetchSize(pageSize));
     }
@@ -1135,6 +1268,17 @@ public abstract class CQLTester
         return currentTable == null ? query : String.format(query, keyspace + "." + currentTable);
     }
 
+    public String formatViewQuery(String query)
+    {
+        return formatViewQuery(KEYSPACE, query);
+    }
+
+    public String formatViewQuery(String keyspace, String query)
+    {
+        String currentView = currentView();
+        return currentView == null ? query : String.format(query, keyspace + "." + currentView);
+    }
+
     protected ResultMessage.Prepared prepare(String query) throws Throwable
     {
         return QueryProcessor.prepare(formatQuery(query), ClientState.forInternalCalls());
@@ -1145,6 +1289,11 @@ public abstract class CQLTester
         return executeFormattedQuery(formatQuery(query), values);
     }
 
+    public UntypedResultSet executeView(String query, Object... values) throws Throwable
+    {
+        return executeFormattedQuery(formatViewQuery(KEYSPACE, query), values);
+    }
+
     protected UntypedResultSet executeFormattedQuery(String query, Object... values) throws Throwable
     {
         UntypedResultSet rs;
diff --git a/test/unit/org/apache/cassandra/cql3/ViewAbstractParameterizedTest.java b/test/unit/org/apache/cassandra/cql3/ViewAbstractParameterizedTest.java
new file mode 100644
index 0000000..629d407
--- /dev/null
+++ b/test/unit/org/apache/cassandra/cql3/ViewAbstractParameterizedTest.java
@@ -0,0 +1,86 @@
+/*
+ * 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.Collection;
+import java.util.stream.Collectors;
+
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import com.datastax.driver.core.ResultSet;
+import org.apache.cassandra.transport.ProtocolVersion;
+
+@Ignore
+@RunWith(Parameterized.class)
+public abstract class ViewAbstractParameterizedTest extends ViewAbstractTest
+{
+    @Parameterized.Parameter
+    public ProtocolVersion version;
+
+    @Parameterized.Parameters()
+    public static Collection<Object[]> versions()
+    {
+        return ProtocolVersion.SUPPORTED.stream()
+                                        .map(v -> new Object[]{v})
+                                        .collect(Collectors.toList());
+    }
+
+    @Before
+    @Override
+    public void beforeTest() throws Throwable
+    {
+        super.beforeTest();
+
+        executeNet("USE " + keyspace());
+    }
+
+    @Override
+    protected com.datastax.driver.core.ResultSet executeNet(String query, Object... values) throws Throwable
+    {
+        return executeNet(version, query, values);
+    }
+
+    @Override
+    protected com.datastax.driver.core.ResultSet executeNetWithPaging(String query, int pageSize)
+    {
+        return executeNetWithPaging(version, query, pageSize);
+    }
+
+    @Override
+    protected void assertRowsNet(ResultSet result, Object[]... rows)
+    {
+        assertRowsNet(version, result, rows);
+    }
+
+    @Override
+    protected void updateView(String query, Object... params) throws Throwable
+    {
+        updateView(version, query, params);
+    }
+
+    protected void updateViewWithFlush(String query, boolean flush, Object... params) throws Throwable
+    {
+        updateView(query, params);
+        if (flush)
+            flush(keyspace());
+    }
+}
diff --git a/test/unit/org/apache/cassandra/cql3/ViewAbstractTest.java b/test/unit/org/apache/cassandra/cql3/ViewAbstractTest.java
index bbd21dc..4b96ae2 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewAbstractTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewAbstractTest.java
@@ -18,24 +18,13 @@
 
 package org.apache.cassandra.cql3;
 
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.junit.After;
 import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 
-import com.datastax.driver.core.exceptions.OperationTimedOutException;
-import org.apache.cassandra.concurrent.Stage;
-import org.awaitility.Awaitility;
-
 @Ignore
 public abstract class ViewAbstractTest extends CQLTester
 {
-    protected final List<String> views = new ArrayList<>();
-
     @BeforeClass
     public static void startup()
     {
@@ -43,66 +32,12 @@ public abstract class ViewAbstractTest extends CQLTester
     }
 
     @Before
-    public void begin()
-    {
-        begin(views);
-    }
-
-    private static void begin(List<String> views)
-    {
-        views.clear();
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        end(views, this);
-    }
-
-    private static void end(List<String> views, CQLTester tester) throws Throwable
-    {
-        for (String viewName : views)
-            tester.executeNet("DROP MATERIALIZED VIEW " + viewName);
-    }
-
-    protected void createView(String name, String query) throws Throwable
+    @Override
+    public void beforeTest() throws Throwable
     {
-        createView(name, query, views, this);
-    }
+        super.beforeTest();
 
-    private static void createView(String name, String query, List<String> views, CQLTester tester) throws Throwable
-    {
-        try
-        {
-            tester.executeNet(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);
-        }
-        catch (OperationTimedOutException ex)
-        {
-            // ... except for timeout, when we actually do not know whether the view was created or not
-            views.add(name);
-            throw ex;
-        }
-    }
-
-    protected void updateView(String query, Object... params) throws Throwable
-    {
-        updateView(query, this, params);
-    }
-
-    private static void updateView(String query, CQLTester tester, Object... params) throws Throwable
-    {
-        tester.executeNet(query, params);
-        waitForViewMutations();
-    }
-
-    protected static void waitForViewMutations()
-    {
-        Awaitility.await()
-                  .atMost(5, TimeUnit.MINUTES)
-                  .until(() -> Stage.VIEW_MUTATION.executor().getPendingTaskCount() == 0
-                               && Stage.VIEW_MUTATION.executor().getActiveTaskCount() == 0);
+        execute("USE " + keyspace());
+        executeNet("USE " + keyspace());
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexDeletionsTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexDeletionsTest.java
index 33fa6d9..831f3ef 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexDeletionsTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexDeletionsTest.java
@@ -45,52 +45,62 @@ import static org.junit.Assert.assertEquals;
  * - ViewComplexTest
  * - ViewComplexLivenessTest
  */
-public class ViewComplexDeletionsTest extends ViewComplexTester
+public class ViewComplexDeletionsTest extends ViewAbstractParameterizedTest
 {
-    // for now, unselected column cannot be fully supported, SEE CASSANDRA-11500
+    // for now, unselected column cannot be fully supported, see CASSANDRA-11500
     @Ignore
     @Test
-    public void testPartialDeleteUnselectedColumn() throws Throwable
+    public void testPartialDeleteUnselectedColumnWithFlush() throws Throwable
+    {
+        testPartialDeleteUnselectedColumn(true);
+    }
+
+    // for now, unselected column cannot be fully supported, see CASSANDRA-11500
+    @Ignore
+    @Test
+    public void testPartialDeleteUnselectedColumnWithoutFlush() throws Throwable
+    {
+        testPartialDeleteUnselectedColumn(false);
+    }
+
+    private void testPartialDeleteUnselectedColumn(boolean flush) throws Throwable
     {
-        boolean flush = true;
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         createTable("CREATE TABLE %s (k int, c int, a int, b int, PRIMARY KEY (k, c))");
-        String mv = createView("CREATE MATERIALIZED VIEW %s " +
-                                 "AS SELECT k,c FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
+        createView("CREATE MATERIALIZED VIEW %s AS " +
+                   "SELECT k,c FROM %s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
         Keyspace ks = Keyspace.open(keyspace());
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         updateView("UPDATE %s USING TIMESTAMP 10 SET b=1 WHERE k=1 AND c=1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRows(execute("SELECT * from %s"), row(1, 1, null, 1));
-        assertRows(execute("SELECT * from " + mv), row(1, 1));
+        assertRows(executeView("SELECT * FROM %s"), row(1, 1));
         updateView("DELETE b FROM %s USING TIMESTAMP 11 WHERE k=1 AND c=1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertEmpty(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * FROM %s"));
         updateView("UPDATE %s USING TIMESTAMP 1 SET a=1 WHERE k=1 AND c=1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRows(execute("SELECT * from %s"), row(1, 1, 1, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1));
+        assertRows(executeView("SELECT * FROM %s"), row(1, 1));
 
         execute("truncate %s;");
 
         // removal generated by unselected column should not shadow PK update with smaller timestamp
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 18 SET a=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, 1, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1));
+        assertRows(executeView("SELECT * FROM %s"), row(1, 1));
 
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 20 SET a=null WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"));
-        assertRows(execute("SELECT * from " + mv));
+        assertRows(executeView("SELECT * FROM %s"));
 
         updateViewWithFlush("INSERT INTO %s(k,c) VALUES(1,1) USING TIMESTAMP 15", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1));
+        assertRows(executeView("SELECT * FROM %s"), row(1, 1));
     }
 
     @Test
@@ -107,78 +117,76 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
 
     private void testPartialDeleteSelectedColumn(boolean flush) throws Throwable
     {
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         createTable("CREATE TABLE %s (k int, c int, a int, b int, e int, f int, PRIMARY KEY (k, c))");
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, k FROM %%s " +
-                                 "WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, k FROM %s " +
+                     "WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
         Keyspace ks = Keyspace.open(keyspace());
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 10 SET b=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, 1, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, 1));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, 1));
 
         updateViewWithFlush("DELETE b FROM %s USING TIMESTAMP 11 WHERE k=1 AND c=1", flush);
         assertEmpty(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 1 SET a=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, 1, null, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, 1, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, 1, null));
 
         updateViewWithFlush("DELETE a FROM %s USING TIMESTAMP 1 WHERE k=1 AND c=1", flush);
         assertEmpty(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         // view livenessInfo should not be affected by selected column ts or tb
         updateViewWithFlush("INSERT INTO %s(k,c) VALUES(1,1) USING TIMESTAMP 0", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, null, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 12 SET b=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, 1, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, 1));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, 1));
 
         updateViewWithFlush("DELETE b FROM %s USING TIMESTAMP 13 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, null, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateViewWithFlush("DELETE FROM %s USING TIMESTAMP 14 WHERE k=1 AND c=1", flush);
         assertEmpty(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         updateViewWithFlush("INSERT INTO %s(k,c) VALUES(1,1) USING TIMESTAMP 15", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, null, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateViewWithFlush("UPDATE %s USING TTL 3 SET b=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, 1, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, 1));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, 1));
 
         TimeUnit.SECONDS.sleep(4);
 
         assertRows(execute("SELECT * from %s"), row(1, 1, null, null, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateViewWithFlush("DELETE FROM %s USING TIMESTAMP 15 WHERE k=1 AND c=1", flush);
         assertEmpty(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         execute("truncate %s;");
 
         // removal generated by unselected column should not shadow selected column with smaller timestamp
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 18 SET e=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, null, null, 1, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 18 SET e=null WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"));
-        assertRows(execute("SELECT * from " + mv));
+        assertRows(executeView("SELECT * from %s"));
 
         updateViewWithFlush("UPDATE %s USING TIMESTAMP 16 SET a=1 WHERE k=1 AND c=1", flush);
         assertRows(execute("SELECT * from %s"), row(1, 1, 1, null, null, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, 1, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, 1, null));
     }
 
     @Test
@@ -198,40 +206,37 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         // for partition range deletion, need to know that existing row is shadowed instead of not existed.
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
-                                 "WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                     "WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b)");
 
         Keyspace ks = Keyspace.open(keyspace());
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) using timestamp 0", 1, 1, 1, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"), row(1, 1, 1, 1));
 
         // remove view row
         updateView("UPDATE %s using timestamp 1 set b = null WHERE a=1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"));
         // remove base row, no view updated generated.
         updateView("DELETE FROM %s using timestamp 2 where a=1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"));
 
         // restor view row with b,c column. d is still tombstone
         updateView("UPDATE %s using timestamp 3 set b = 1,c = 1 where a=1"); // upsert
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv), row(1, 1, 1, null));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"), row(1, 1, 1, null));
     }
 
     @Test
@@ -253,13 +258,11 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         // CASSANDRA-13409 new update should not resurrect previous deleted data in view
         createTable("create table %s (p int primary key, v1 int, v2 int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("create materialized view %s as select * from %%s " +
-                                 "where p is not null and v1 is not null primary key (v1, p)");
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        createView("create materialized view %s as select * from %s " +
+                     "where p is not null and v1 is not null primary key (v1, p)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         // sstable-1, Set initial values TS=1
         updateView("Insert into %s (p, v1, v2) values (3, 1, 3) using timestamp 1;");
@@ -267,40 +270,40 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v2, WRITETIME(v2) from " + mv + " WHERE v1 = ? AND p = ?", 1, 3), row(3, 1L));
+        assertRowsIgnoringOrder(executeView("SELECT v2, WRITETIME(v2) from %s WHERE v1 = ? AND p = ?", 1, 3), row(3, 1L));
         // sstable-2
         updateView("Delete from %s using timestamp 2 where p = 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"));
         // sstable-3
         updateView("Insert into %s (p, v1) values (3, 1) using timestamp 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, null, null));
         // sstable-4
         updateView("UPdate %s using timestamp 4 set v1 = 2 where p = 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(2, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(2, 3, null, null));
         // sstable-5
         updateView("UPdate %s using timestamp 5 set v1 = 1 where p = 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, null, null));
 
         if (flush)
         {
             // compact sstable 2 and 4, 5;
-            ColumnFamilyStore cfs = ks.getColumnFamilyStore(mv);
+            ColumnFamilyStore cfs = ks.getColumnFamilyStore(currentView());
             List<String> sstables = cfs.getLiveSSTables()
                                        .stream()
                                        .sorted(Comparator.comparingInt(s -> s.descriptor.generation))
@@ -311,7 +314,7 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
             assertEquals(3, cfs.getLiveSSTables().size());
         }
         // regular tombstone should be retained after compaction
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, null, null));
     }
 
     @Test
@@ -332,13 +335,11 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
     {
         createTable("create table %s (p1 int, p2 int, v1 int, v2 int, primary key(p1, p2))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("create materialized view %s as select * from %%s " +
-                                 "where p1 is not null and p2 is not null primary key (p2, p1)");
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        createView("create materialized view %s as select * from %s " +
+                     "where p1 is not null and p2 is not null primary key (p2, p1)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         // Set initial values TS=1
         updateView("Insert into %s (p1, p2, v1, v2) values (1, 2, 3, 4) using timestamp 1;");
@@ -346,7 +347,7 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, v2, WRITETIME(v2) from " + mv + " WHERE p1 = ? AND p2 = ?", 1, 2),
+        assertRowsIgnoringOrder(executeView("SELECT v1, v2, WRITETIME(v2) from %s WHERE p1 = ? AND p2 = ?", 1, 2),
                                 row(3, 4, 1L));
         // remove row/mv TS=2
         updateView("Delete from %s using timestamp 2 where p1 = 1 and p2 = 2;");
@@ -354,38 +355,38 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         // view are empty
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"));
         // insert PK with TS=3
         updateView("Insert into %s (p1, p2) values (1, 2) using timestamp 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         // deleted column in MV remained dead
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv), row(2, 1, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"), row(2, 1, null, null));
 
-        ks.getColumnFamilyStore(mv).forceMajorCompaction();
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv), row(2, 1, null, null));
+        ks.getColumnFamilyStore(currentView()).forceMajorCompaction();
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"), row(2, 1, null, null));
 
         // reset values
         updateView("Insert into %s (p1, p2, v1, v2) values (1, 2, 3, 4) using timestamp 10;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, v2, WRITETIME(v2) from " + mv + " WHERE p1 = ? AND p2 = ?", 1, 2),
+        assertRowsIgnoringOrder(executeView("SELECT v1, v2, WRITETIME(v2) from %s WHERE p1 = ? AND p2 = ?", 1, 2),
                                 row(3, 4, 10L));
 
         updateView("UPDATE %s using timestamp 20 SET v2 = 5 WHERE p1 = 1 and p2 = 2");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, v2, WRITETIME(v2) from " + mv + " WHERE p1 = ? AND p2 = ?", 1, 2),
+        assertRowsIgnoringOrder(executeView("SELECT v1, v2, WRITETIME(v2) from %s WHERE p1 = ? AND p2 = ?", 1, 2),
                                 row(3, 5, 20L));
 
         updateView("DELETE FROM %s using timestamp 10 WHERE p1 = 1 and p2 = 2");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, v2, WRITETIME(v2) from " + mv + " WHERE p1 = ? AND p2 = ?", 1, 2),
+        assertRowsIgnoringOrder(executeView("SELECT v1, v2, WRITETIME(v2) from %s WHERE p1 = ? AND p2 = ?", 1, 2),
                                 row(null, 5, 20L));
     }
 
@@ -393,13 +394,11 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
     {
         createTable("create table %s (p int primary key, v1 int, v2 int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("create materialized view %s as select * from %%s " +
-                                 "where p is not null and v1 is not null primary key (v1, p)");
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        createView("create materialized view %s as select * from %s " +
+                     "where p is not null and v1 is not null primary key (v1, p)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         // Set initial values TS=1
         updateView("Insert into %s (p, v1, v2) values (3, 1, 5) using timestamp 1;");
@@ -407,21 +406,21 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v2, WRITETIME(v2) from " + mv + " WHERE v1 = ? AND p = ?", 1, 3), row(5, 1L));
+        assertRowsIgnoringOrder(executeView("SELECT v2, WRITETIME(v2) from %s WHERE v1 = ? AND p = ?", 1, 3), row(5, 1L));
         // remove row/mv TS=2
         updateView("Delete from %s using timestamp 2 where p = 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         // view are empty
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"));
         // insert PK with TS=3
         updateView("Insert into %s (p, v1) values (3, 1) using timestamp 3;");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         // deleted column in MV remained dead
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv), row(1, 3, null));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"), row(1, 3, null));
 
         // insert values TS=2, it should be considered dead due to previous tombstone
         updateView("Insert into %s (p, v1, v2) values (3, 1, 5) using timestamp 2;");
@@ -429,30 +428,27 @@ public class ViewComplexDeletionsTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         // deleted column in MV remained dead
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv), row(1, 3, null));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " limit 1"), row(1, 3, null));
+        assertRowsIgnoringOrder(executeView("SELECT * FROM %s"), row(1, 3, null));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s limit 1"), row(1, 3, null));
 
         // insert values TS=2, it should be considered dead due to previous tombstone
-        executeNet(version, "UPDATE %s USING TIMESTAMP 3 SET v2 = ? WHERE p = ?", 4, 3);
+        executeNet("UPDATE %s USING TIMESTAMP 3 SET v2 = ? WHERE p = ?", 4, 3);
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRows(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, 4, 3L));
+        assertRows(execute("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, 4, 3L));
 
-        ks.getColumnFamilyStore(mv).forceMajorCompaction();
-        assertRows(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, 4, 3L));
-        assertRows(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv + " limit 1"), row(1, 3, 4, 3L));
+        ks.getColumnFamilyStore(currentView()).forceMajorCompaction();
+        assertRows(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, 4, 3L));
+        assertRows(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s limit 1"), row(1, 3, 4, 3L));
     }
 
     @Test
     public void testNoBatchlogCleanupForLocalMutations() throws Throwable
     {
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
         createTable("CREATE TABLE %s (k1 int primary key, v1 int)");
-        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
                    "WHERE k1 IS NOT NULL AND v1 IS NOT NULL PRIMARY KEY (v1, k1)");
 
         ColumnFamilyStore batchlog = Keyspace.open(SchemaConstants.SYSTEM_KEYSPACE_NAME).getColumnFamilyStore(SystemKeyspace.BATCHES);
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexLivenessTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexLivenessTest.java
index 7b4aa1a..66738ce 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexLivenessTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexLivenessTest.java
@@ -36,28 +36,36 @@ import static org.junit.Assert.assertEquals;
  * - ViewComplexTest
  * - ViewComplexLivenessTest
  */
-public class ViewComplexLivenessTest extends ViewComplexTester
+public class ViewComplexLivenessTest extends ViewAbstractParameterizedTest
 {
     @Test
-    public void testUnselectedColumnWithExpiredLivenessInfo() throws Throwable
+    public void testUnselectedColumnWithExpiredLivenessInfoWithFlush() throws Throwable
+    {
+        testUnselectedColumnWithExpiredLivenessInfo(true);
+    }
+
+    @Test
+    public void testUnselectedColumnWithExpiredLivenessInfoWithoutFlush() throws Throwable
+    {
+        testUnselectedColumnWithExpiredLivenessInfo(false);
+    }
+
+    private void testUnselectedColumnWithExpiredLivenessInfo(boolean flush) throws Throwable
     {
-        boolean flush = true;
         createTable("create table %s (k int, c int, a int, b int, PRIMARY KEY(k, c))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String name = createView("create materialized view %s as select k,c,b from %%s " +
-                                 "where c is not null and k is not null primary key (c, k)");
-        ks.getColumnFamilyStore(name).disableAutoCompaction();
+        createView("create materialized view %s as select k,c,b from %s " +
+                   "where c is not null and k is not null primary key (c, k)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         // sstable-1, Set initial values TS=1
         updateViewWithFlush("UPDATE %s SET a = 1 WHERE k = 1 AND c = 1;", flush);
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT k,c,b from " + name + " WHERE k = 1 AND c = 1;"),
+        assertRowsIgnoringOrder(executeView("SELECT k,c,b from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, null));
 
         // sstable-2
@@ -65,28 +73,28 @@ public class ViewComplexLivenessTest extends ViewComplexTester
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT k,c,b from " + name + " WHERE k = 1 AND c = 1;"),
+        assertRowsIgnoringOrder(executeView("SELECT k,c,b from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, null));
 
         Thread.sleep(5001);
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT k,c,b from " + name + " WHERE k = 1 AND c = 1;"),
+        assertRowsIgnoringOrder(executeView("SELECT k,c,b from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, null));
 
         // sstable-3
         updateViewWithFlush("Update %s set a = null where k = 1 AND c = 1;", flush);
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE k = 1 AND c = 1;"));
-        assertRowsIgnoringOrder(execute("SELECT k,c,b from " + name + " WHERE k = 1 AND c = 1;"));
+        assertRowsIgnoringOrder(executeView("SELECT k,c,b from %s WHERE k = 1 AND c = 1;"));
 
         // sstable-4
         updateViewWithFlush("Update %s USING TIMESTAMP 1 set b = 1 where k = 1 AND c = 1;", flush);
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT k,c,b from " + name + " WHERE k = 1 AND c = 1;"),
+        assertRowsIgnoringOrder(executeView("SELECT k,c,b from %s WHERE k = 1 AND c = 1;"),
                                 row(1, 1, 1));
     }
 
@@ -108,13 +116,11 @@ public class ViewComplexLivenessTest extends ViewComplexTester
     {
         createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int);");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
                                 "WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (k, a)");
-        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
                                 "WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (a, k)");
         ks.getColumnFamilyStore(mv1).disableAutoCompaction();
         ks.getColumnFamilyStore(mv2).disableAutoCompaction();
@@ -128,6 +134,7 @@ public class ViewComplexLivenessTest extends ViewComplexTester
             // create expired liveness
             updateView("DELETE a FROM %s WHERE k = ?;", i);
         }
+
         if (flush)
         {
             ks.getColumnFamilyStore(mv1).forceBlockingFlush();
@@ -137,10 +144,10 @@ public class ViewComplexLivenessTest extends ViewComplexTester
         for (String view : Arrays.asList(mv1, mv2))
         {
             // paging
-            assertEquals(1, executeNetWithPaging(version, String.format("SELECT k,a,b FROM %s limit 1", view), 1).all().size());
-            assertEquals(2, executeNetWithPaging(version, String.format("SELECT k,a,b FROM %s limit 2", view), 1).all().size());
-            assertEquals(2, executeNetWithPaging(version, String.format("SELECT k,a,b FROM %s", view), 1).all().size());
-            assertRowsNet(version, executeNetWithPaging(version, String.format("SELECT k,a,b FROM %s ", view), 1),
+            assertEquals(1, executeNetWithPaging(String.format("SELECT k,a,b FROM %s limit 1", view), 1).all().size());
+            assertEquals(2, executeNetWithPaging(String.format("SELECT k,a,b FROM %s limit 2", view), 1).all().size());
+            assertEquals(2, executeNetWithPaging(String.format("SELECT k,a,b FROM %s", view), 1).all().size());
+            assertRowsNet(executeNetWithPaging(String.format("SELECT k,a,b FROM %s ", view), 1),
                           row(50, 50, 50),
                           row(100, 100, 100));
             // limit
@@ -156,22 +163,20 @@ public class ViewComplexLivenessTest extends ViewComplexTester
     {
         createTable("create table %s (p int primary key, v1 int, v2 int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String name = createView("create materialized view %s as select * from %%s " +
-                                 "where p is not null and v1 is not null primary key (v1, p) " +
-                                 "with gc_grace_seconds=5");
-        ColumnFamilyStore cfs = ks.getColumnFamilyStore(name);
+        createView("create materialized view %s as select * from %s " +
+                   "where p is not null and v1 is not null primary key (v1, p) " +
+                   "with gc_grace_seconds=5");
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(currentView());
         cfs.disableAutoCompaction();
 
         updateView("Insert into %s (p, v1, v2) values (1, 1, 1)");
-        assertRowsIgnoringOrder(execute("SELECT p, v1, v2 from " + name), row(1, 1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT p, v1, v2 from %s"), row(1, 1, 1));
 
         updateView("Update %s set v1 = null WHERE p = 1");
         FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT p, v1, v2 from " + name));
+        assertRowsIgnoringOrder(executeView("SELECT p, v1, v2 from %s"));
 
         cfs.forceMajorCompaction(); // before gc grace second, strict-liveness tombstoned dead row remains
         assertEquals(1, cfs.getLiveSSTables().size());
@@ -184,16 +189,16 @@ public class ViewComplexLivenessTest extends ViewComplexTester
 
         updateView("Update %s using ttl 5 set v1 = 1 WHERE p = 1");
         FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT p, v1, v2 from " + name), row(1, 1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT p, v1, v2 from %s"), row(1, 1, 1));
 
         cfs.forceMajorCompaction(); // before ttl+gc_grace_second, strict-liveness ttled dead row remains
         assertEquals(1, cfs.getLiveSSTables().size());
-        assertRowsIgnoringOrder(execute("SELECT p, v1, v2 from " + name), row(1, 1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT p, v1, v2 from %s"), row(1, 1, 1));
 
         Thread.sleep(5500); // after expired, before gc_grace_second
         cfs.forceMajorCompaction();// before ttl+gc_grace_second, strict-liveness ttled dead row remains
         assertEquals(1, cfs.getLiveSSTables().size());
-        assertRowsIgnoringOrder(execute("SELECT p, v1, v2 from " + name));
+        assertRowsIgnoringOrder(executeView("SELECT p, v1, v2 from %s"));
 
         Thread.sleep(5500); // after expired + gc_grace_second
         assertEquals(1, cfs.getLiveSSTables().size()); // no auto compaction.
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexTTLTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexTTLTest.java
index 7c49ffa..f8dfa0f 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexTTLTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexTTLTest.java
@@ -34,7 +34,7 @@ import static org.junit.Assert.assertTrue;
  * - ViewComplexTest
  * - ViewComplexLivenessTest
  */
-public class ViewComplexTTLTest extends ViewComplexTester
+public class ViewComplexTTLTest extends ViewAbstractParameterizedTest
 {
     @Test
     public void testUpdateColumnInViewPKWithTTLWithFlush() throws Throwable
@@ -55,13 +55,11 @@ public class ViewComplexTTLTest extends ViewComplexTester
         // CASSANDRA-13657 if base column used in view pk is ttled, then view row is considered dead
         createTable("create table %s (k int primary key, a int, b int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
-                               "WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (a, k)");
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (a, k)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         updateView("UPDATE %s SET a = 1 WHERE k = 1;");
 
@@ -69,7 +67,7 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRows(execute("SELECT * from %s"), row(1, 1, null));
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null));
 
         updateView("DELETE a FROM %s WHERE k = 1");
 
@@ -77,7 +75,7 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRows(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         updateView("INSERT INTO %s (k) VALUES (1);");
 
@@ -85,7 +83,7 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRows(execute("SELECT * from %s"), row(1, null, null));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         updateView("UPDATE %s USING TTL 5 SET a = 10 WHERE k = 1;");
 
@@ -93,7 +91,7 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRows(execute("SELECT * from %s"), row(1, 10, null));
-        assertRows(execute("SELECT * from " + mv), row(10, 1, null));
+        assertRows(executeView("SELECT * from %s"), row(10, 1, null));
 
         updateView("UPDATE %s SET b = 100 WHERE k = 1;");
 
@@ -101,14 +99,14 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRows(execute("SELECT * from %s"), row(1, 10, 100));
-        assertRows(execute("SELECT * from " + mv), row(10, 1, 100));
+        assertRows(executeView("SELECT * from %s"), row(10, 1, 100));
 
         Thread.sleep(5000);
 
         // 'a' is TTL of 5 and removed.
         assertRows(execute("SELECT * from %s"), row(1, null, 100));
-        assertEmpty(execute("SELECT * from " + mv));
-        assertEmpty(execute("SELECT * from " + mv + " WHERE k = ? AND a = ?", 1, 10));
+        assertEmpty(executeView("SELECT * from %s"));
+        assertEmpty(executeView("SELECT * from %s WHERE k = ? AND a = ?", 1, 10));
 
         updateView("DELETE b FROM %s WHERE k=1");
 
@@ -116,7 +114,7 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRows(execute("SELECT * from %s"), row(1, null, null));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
 
         updateView("DELETE FROM %s WHERE k=1;");
 
@@ -124,7 +122,7 @@ public class ViewComplexTTLTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertEmpty(execute("SELECT * from %s"));
-        assertEmpty(execute("SELECT * from " + mv));
+        assertEmpty(executeView("SELECT * from %s"));
     }
     @Test
     public void testUnselectedColumnsTTLWithFlush() throws Throwable
@@ -145,48 +143,46 @@ public class ViewComplexTTLTest extends ViewComplexTester
         // CASSANDRA-13127 not ttled unselected column in base should keep view row alive
         createTable("create table %s (p int, c int, v int, primary key(p, c))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT p, c FROM %%s " +
-                               "WHERE p IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, p)");
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT p, c FROM %s " +
+                   "WHERE p IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, p)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         updateViewWithFlush("INSERT INTO %s (p, c) VALUES (0, 0) USING TTL 3;", flush);
 
         updateViewWithFlush("UPDATE %s USING TTL 1000 SET v = 0 WHERE p = 0 and c = 0;", flush);
 
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         Thread.sleep(3000);
 
         UntypedResultSet.Row row = execute("SELECT v, ttl(v) from %s WHERE c = ? AND p = ?", 0, 0).one();
         assertEquals("row should have value of 0", 0, row.getInt("v"));
         assertTrue("row should have ttl less than 1000", row.getInt("ttl(v)") < 1000);
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         updateViewWithFlush("DELETE FROM %s WHERE p = 0 and c = 0;", flush);
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
 
         updateViewWithFlush("INSERT INTO %s (p, c) VALUES (0, 0) ", flush);
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         // already have a live row, no need to apply the unselected cell ttl
         updateViewWithFlush("UPDATE %s USING TTL 3 SET v = 0 WHERE p = 0 and c = 0;", flush);
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         updateViewWithFlush("INSERT INTO %s (p, c) VALUES (1, 1) USING TTL 3", flush);
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 1, 1), row(1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 1, 1), row(1, 1));
 
         Thread.sleep(4000);
 
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 1, 1));
 
         // unselected should keep view row alive
         updateViewWithFlush("UPDATE %s SET v = 0 WHERE p = 1 and c = 1;", flush);
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 1, 1), row(1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 1, 1), row(1, 1));
 
     } 
 
@@ -196,8 +192,6 @@ public class ViewComplexTTLTest extends ViewComplexTester
         // CASSANDRA-13127 when liveness timestamp tie, greater localDeletionTime should win if both are expiring.
         createTable("create table %s (p int, c int, v int, primary key(p, c))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
         updateView("INSERT INTO %s (p, c, v) VALUES (0, 0, 0) using timestamp 1;");
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
index 27cbf07..c462ebf 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexTest.java
@@ -47,7 +47,7 @@ import static org.junit.Assert.fail;
  * - ViewComplexTest
  * - ViewComplexLivenessTest
  */
-public class ViewComplexTest extends ViewComplexTester
+public class ViewComplexTest extends ViewAbstractParameterizedTest
 {
     @Test
     public void testNonBaseColumnInViewPkWithFlush() throws Throwable
@@ -65,51 +65,49 @@ public class ViewComplexTest extends ViewComplexTester
     {
         createTable("create table %s (p1 int, p2 int, v1 int, v2 int, primary key (p1,p2))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("create materialized view %s as select * from %%s " +
-                               "where p1 is not null and p2 is not null primary key (p2, p1) " +
-                               "with gc_grace_seconds=5");
-        ColumnFamilyStore cfs = ks.getColumnFamilyStore(mv);
+        createView("create materialized view %s as select * from %s " +
+                   "where p1 is not null and p2 is not null primary key (p2, p1) " +
+                   "with gc_grace_seconds=5");
+        ColumnFamilyStore cfs = ks.getColumnFamilyStore(currentView());
         cfs.disableAutoCompaction();
 
         updateView("UPDATE %s USING TIMESTAMP 1 set v1 =1 where p1 = 1 AND p2 = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from %s"), row(1, 1, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from " + mv), row(1, 1, 1, null));
+        assertRowsIgnoringOrder(executeView("SELECT p1, p2, v1, v2 from %s"), row(1, 1, 1, null));
 
         updateView("UPDATE %s USING TIMESTAMP 2 set v1 = null, v2 = 1 where p1 = 1 AND p2 = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from %s"), row(1, 1, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from " + mv), row(1, 1, null, 1));
+        assertRowsIgnoringOrder(executeView("SELECT p1, p2, v1, v2 from %s"), row(1, 1, null, 1));
 
         updateView("UPDATE %s USING TIMESTAMP 2 set v2 = null where p1 = 1 AND p2 = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from %s"));
-        assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT p1, p2, v1, v2 from %s"));
 
         updateView("INSERT INTO %s (p1,p2) VALUES(1,1) USING TIMESTAMP 3;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from %s"), row(1, 1, null, null));
-        assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from " + mv), row(1, 1, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT p1, p2, v1, v2 from %s"), row(1, 1, null, null));
 
         updateView("DELETE FROM %s USING TIMESTAMP 4 WHERE p1 =1 AND p2 = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from %s"));
-        assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT p1, p2, v1, v2 from %s"));
 
         updateView("UPDATE %s USING TIMESTAMP 5 set v2 = 1 where p1 = 1 AND p2 = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from %s"), row(1, 1, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT p1, p2, v1, v2 from " + mv), row(1, 1, null, 1));
+        assertRowsIgnoringOrder(executeView("SELECT p1, p2, v1, v2 from %s"), row(1, 1, null, 1));
     }
 
     @Test
@@ -128,13 +126,11 @@ public class ViewComplexTest extends ViewComplexTester
     {
         createTable("create table %s (p int primary key, v1 int, v2 int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("create materialized view %s as select * from %%s " +
-                               "where p is not null and v1 is not null primary key (v1, p)");
-        ks.getColumnFamilyStore(mv).disableAutoCompaction();
+        createView("create materialized view %s as select * from %s " +
+                   "where p is not null and v1 is not null primary key (v1, p)");
+        ks.getColumnFamilyStore(currentView()).disableAutoCompaction();
 
         // sstable 1, Set initial values TS=1
         updateView("Insert into %s (p, v1, v2) values (3, 1, 3) using timestamp 1;");
@@ -142,14 +138,14 @@ public class ViewComplexTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v2, WRITETIME(v2) from " + mv + " WHERE v1 = ? AND p = ?", 1, 3), row(3, 1L));
+        assertRowsIgnoringOrder(executeView("SELECT v2, WRITETIME(v2) from %s WHERE v1 = ? AND p = ?", 1, 3), row(3, 1L));
         // sstable 2
         updateView("UPdate %s using timestamp 2 set v2 = null where p = 3");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v2, WRITETIME(v2) from " + mv + " WHERE v1 = ? AND p = ?", 1, 3),
+        assertRowsIgnoringOrder(executeView("SELECT v2, WRITETIME(v2) from %s WHERE v1 = ? AND p = ?", 1, 3),
                                 row(null, null));
         // sstable 3
         updateView("UPdate %s using timestamp 3 set v1 = 2 where p = 3");
@@ -157,19 +153,19 @@ public class ViewComplexTest extends ViewComplexTester
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(2, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(2, 3, null, null));
         // sstable 4
         updateView("UPdate %s using timestamp 4 set v1 = 1 where p = 3");
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, null, null));
 
         if (flush)
         {
             // compact sstable 2 and 3;
-            ColumnFamilyStore cfs = ks.getColumnFamilyStore(mv);
+            ColumnFamilyStore cfs = ks.getColumnFamilyStore(currentView());
             List<String> sstables = cfs.getLiveSSTables()
                                        .stream()
                                        .sorted(Comparator.comparingInt(s -> s.descriptor.generation))
@@ -179,8 +175,8 @@ public class ViewComplexTest extends ViewComplexTester
             CompactionManager.instance.forceUserDefinedCompaction(dataFiles);
         }
         // cell-tombstone in sstable 4 is not compacted away, because the shadowable tombstone is shadowed by new row.
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, null, null));
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv + " limit 1"), row(1, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s limit 1"), row(1, 3, null, null));
     }
 
     @Test
@@ -199,22 +195,20 @@ public class ViewComplexTest extends ViewComplexTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, f int, PRIMARY KEY(a, b))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         List<String> viewNames = new ArrayList<>();
         List<String> mvStatements = Arrays.asList(
                                                   // all selected
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
                                                   // unselected e,f
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b,c,d FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b,c,d FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
                                                   // no selected
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a,b)",
                                                   // all selected, re-order keys
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b,a)",
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b,a)",
                                                   // unselected e,f, re-order keys
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b,c,d FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b,a)",
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b,c,d FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b,a)",
                                                   // no selected, re-order keys
-                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b,a)");
+                                                  "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b,a)");
 
         Keyspace ks = Keyspace.open(keyspace());
 
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexTester.java b/test/unit/org/apache/cassandra/cql3/ViewComplexTester.java
deleted file mode 100644
index a934649..0000000
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexTester.java
+++ /dev/null
@@ -1,130 +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.Collection;
-import java.util.List;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.stream.Collectors;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import com.datastax.driver.core.exceptions.OperationTimedOutException;
-import org.apache.cassandra.concurrent.Stage;
-import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.transport.ProtocolVersion;
-
-/* ViewComplexTest class has been split into multiple ones because of timeout issues (CASSANDRA-16670)
- * Any changes here check if they apply to the other classes:
- * - ViewComplexUpdatesTest
- * - ViewComplexDeletionsTest
- * - ViewComplexTTLTest
- * - ViewComplexTest
- * - ViewComplexLivenessTest
- */
-@RunWith(Parameterized.class)
-public abstract class ViewComplexTester extends CQLTester
-{
-    private static final AtomicInteger seqNumber = new AtomicInteger();
-
-    @Parameterized.Parameter
-    public ProtocolVersion version;
-
-    @Parameterized.Parameters()
-    public static Collection<Object[]> versions()
-    {
-        return ProtocolVersion.SUPPORTED.stream()
-                                        .map(v -> new Object[]{v})
-                                        .collect(Collectors.toList());
-    }
-
-    protected final List<String> views = new ArrayList<>();
-
-    @BeforeClass
-    public static void startup()
-    {
-        requireNetwork();
-    }
-
-    @Before
-    public void begin() throws Throwable
-    {
-        views.clear();
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        dropMViews();
-    }
-
-    protected void dropMViews() throws Throwable
-    {
-        for (String viewName : views)
-            executeNet(version, "DROP MATERIALIZED VIEW " + viewName);
-    }
-
-    protected String createView(String query) throws Throwable
-    {
-        String name = createViewName();
-
-        try
-        {
-            executeNet(version, 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);
-        }
-        catch (OperationTimedOutException ex)
-        {
-            // ... except for timeout, when we actually do not know whether the view was created or not
-            views.add(name);
-            throw ex;
-        }
-
-        return name;
-    }
-
-    protected static String createViewName()
-    {
-        return "mv" + seqNumber.getAndIncrement();
-    }
-
-    protected void updateView(String query, Object... params) throws Throwable
-    {
-        updateViewWithFlush(query, false, params);
-    }
-
-    protected void updateViewWithFlush(String query, boolean flush, Object... params) throws Throwable
-    {
-        executeNet(version, query, params);
-        while (!(Stage.VIEW_MUTATION.executor().getPendingTaskCount() == 0
-                 && Stage.VIEW_MUTATION.executor().getActiveTaskCount() == 0))
-        {
-            Thread.sleep(1);
-        }
-        if (flush)
-            Keyspace.open(keyspace()).flush();
-    }
-}
diff --git a/test/unit/org/apache/cassandra/cql3/ViewComplexUpdatesTest.java b/test/unit/org/apache/cassandra/cql3/ViewComplexUpdatesTest.java
index 76ae361..275296e 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewComplexUpdatesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewComplexUpdatesTest.java
@@ -33,7 +33,7 @@ import org.apache.cassandra.utils.FBUtilities;
  * - ViewComplexTest
  * - ViewComplexLivenessTest
  */
-public class ViewComplexUpdatesTest extends ViewComplexTester
+public class ViewComplexUpdatesTest extends ViewAbstractParameterizedTest
 {
     @Test
     public void testUpdateColumnNotInViewWithFlush() throws Throwable
@@ -52,12 +52,9 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
     {
         // CASSANDRA-13127: if base column not selected in view are alive, then pk of view row should be alive
         String baseTable = createTable("create table %s (p int, c int, v1 int, v2 int, primary key(p, c))");
-
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT p, c FROM %%s " +
+        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT p, c from %s " +
                                "WHERE p IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, p)");
         ks.getColumnFamilyStore(mv).disableAutoCompaction();
 
@@ -67,7 +64,7 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         updateView("DELETE v1 FROM %s USING TIMESTAMP 1 WHERE p = 0 AND c = 0");
 
@@ -75,7 +72,7 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertEmpty(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
-        assertEmpty(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0));
+        assertEmpty(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
 
         // shadowed by tombstone
         updateView("UPDATE %s USING TIMESTAMP 1 SET v1 = 1 WHERE p = 0 AND c = 0");
@@ -84,7 +81,7 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertEmpty(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
-        assertEmpty(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0));
+        assertEmpty(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
 
         updateView("UPDATE %s USING TIMESTAMP 2 SET v2 = 1 WHERE p = 0 AND c = 0");
 
@@ -92,7 +89,7 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         updateView("DELETE v1 FROM %s USING TIMESTAMP 3 WHERE p = 0 AND c = 0");
 
@@ -100,7 +97,7 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         updateView("DELETE v2 FROM %s USING TIMESTAMP 4 WHERE p = 0 AND c = 0");
 
@@ -108,7 +105,7 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertEmpty(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
-        assertEmpty(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0));
+        assertEmpty(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
 
         updateView("UPDATE %s USING TTL 3 SET v2 = 1 WHERE p = 0 AND c = 0");
 
@@ -116,12 +113,12 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         Thread.sleep(TimeUnit.SECONDS.toMillis(3));
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
 
         updateView("UPDATE %s SET v2 = 1 WHERE p = 0 AND c = 0");
 
@@ -129,15 +126,15 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0), row(0, 0));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0), row(0, 0));
 
         assertInvalidMessage(String.format("Cannot drop column v2 on base table %s with materialized views", baseTable), "ALTER TABLE %s DROP v2");
         // // drop unselected base column, unselected metadata should be removed, thus view row is dead
         // updateView("ALTER TABLE %s DROP v2");
         // assertRowsIgnoringOrder(execute("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
-        // assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE c = ? AND p = ?", 0, 0));
+        // assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE c = ? AND p = ?", 0, 0));
         // assertRowsIgnoringOrder(execute("SELECT * from %s"));
-        // assertRowsIgnoringOrder(execute("SELECT * from " + mv));
+        // assertRowsIgnoringOrder(executeView("SELECT * from %s"));
     }
 
     @Test
@@ -154,10 +151,8 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
 
     private void testPartialUpdateWithUnselectedCollections(boolean flush) throws Throwable
     {
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         String baseTable = createTable("CREATE TABLE %s (k int, c int, a int, b int, l list<int>, s set<int>, m map<int,int>, PRIMARY KEY (k, c))");
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, k FROM %%s " +
+        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, k from %s " +
                                "WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (c, k)");
         Keyspace ks = Keyspace.open(keyspace());
         ks.getColumnFamilyStore(mv).disableAutoCompaction();
@@ -165,17 +160,17 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
         updateView("UPDATE %s SET l=l+[1,2,3] WHERE k = 1 AND c = 1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateView("UPDATE %s SET l=l-[1,2] WHERE k = 1 AND c = 1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         updateView("UPDATE %s SET b=3 WHERE k=1 AND c=1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRows(execute("SELECT * from " + mv), row(1, 1, null, 3));
+        assertRows(executeView("SELECT * from %s"), row(1, 1, null, 3));
 
         updateView("UPDATE %s SET b=null, l=l-[3], s=s-{3} WHERE k = 1 AND c = 1");
         if (flush)
@@ -184,21 +179,21 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
             ks.getColumnFamilyStore(mv).forceMajorCompaction();
         }
         assertRowsIgnoringOrder(execute("SELECT k,c,a,b from %s"));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s"));
 
         updateView("UPDATE %s SET m=m+{3:3}, l=l-[1], s=s-{2} WHERE k = 1 AND c = 1");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         assertRowsIgnoringOrder(execute("SELECT k,c,a,b from %s"), row(1, 1, null, null));
-        assertRowsIgnoringOrder(execute("SELECT * from " + mv), row(1, 1, null, null));
+        assertRowsIgnoringOrder(executeView("SELECT * from %s"), row(1, 1, null, null));
 
         assertInvalidMessage(String.format("Cannot drop column m on base table %s with materialized views", baseTable), "ALTER TABLE %s DROP m");
         // executeNet(version, "ALTER TABLE %s DROP m");
         // ks.getColumnFamilyStore(mv).forceMajorCompaction();
         // assertRowsIgnoringOrder(execute("SELECT k,c,a,b from %s WHERE k = 1 AND c = 1"));
-        // assertRowsIgnoringOrder(execute("SELECT * from " + mv + " WHERE k = 1 AND c = 1"));
+        // assertRowsIgnoringOrder(executeView("SELECT * from %s WHERE k = 1 AND c = 1"));
         // assertRowsIgnoringOrder(execute("SELECT k,c,a,b from %s"));
-        // assertRowsIgnoringOrder(execute("SELECT * from " + mv));
+        // assertRowsIgnoringOrder(executeView("SELECT * from %s"));
     }
 
     @Test
@@ -217,11 +212,9 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
     {
         createTable("create table %s (p int primary key, v1 int, v2 int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("create materialized view %s as select * from %%s " +
+        String mv = createView("create materialized view %s as select * from %s " +
                                "where p is not null and v1 is not null primary key (v1, p)");
         ks.getColumnFamilyStore(mv).disableAutoCompaction();
 
@@ -229,23 +222,23 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
         updateView("Insert into %s (p, v1, v2) values (3, 1, 3) using timestamp 6;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, 3, 6L));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, 3, 6L));
         // increase pk's timestamp to 20
         updateView("Insert into %s (p) values (3) using timestamp 20;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, 3, 6L));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, 3, 6L));
         // change v1's to 2 and remove existing view row with ts7
         updateView("UPdate %s using timestamp 7 set v1 = 2 where p = 3;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(2, 3, 3, 6L));
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv + " limit 1"), row(2, 3, 3, 6L));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(2, 3, 3, 6L));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s" + " limit 1"), row(2, 3, 3, 6L));
         // change v1's to 1 and remove existing view row with ts8
         updateView("UPdate %s using timestamp 8 set v1 = 1 where p = 3;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT v1, p, v2, WRITETIME(v2) from " + mv), row(1, 3, 3, 6L));
+        assertRowsIgnoringOrder(executeView("SELECT v1, p, v2, WRITETIME(v2) from %s"), row(1, 3, 3, 6L));
     }
 
     @Test
@@ -267,11 +260,9 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
         // CASSANDRA-11500 able to shadow old view row with column ts greater tahn pk's ts and re-insert the view row
         String baseTable = createTable("CREATE TABLE %s (k int PRIMARY KEY, a int, b int);");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
         Keyspace ks = Keyspace.open(keyspace());
 
-        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
+        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * from %s " +
                                "WHERE k IS NOT NULL AND a IS NOT NULL PRIMARY KEY (k, a)");
         ks.getColumnFamilyStore(mv).disableAutoCompaction();
         updateView("DELETE FROM %s USING TIMESTAMP 0 WHERE k = 1;");
@@ -281,37 +272,37 @@ public class ViewComplexUpdatesTest extends ViewComplexTester
         updateView("INSERT INTO %s(k, a, b) VALUES (1, 1, 1) USING TIMESTAMP 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 1, 1));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 1, 1));
         updateView("UPDATE %s USING TIMESTAMP 10 SET b = 2 WHERE k = 1;");
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 1, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 1, 2));
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 1, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 1, 2));
         updateView("UPDATE %s USING TIMESTAMP 2 SET a = 2 WHERE k = 1;");
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 2, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 2, 2));
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
         ks.getColumnFamilyStore(mv).forceMajorCompaction();
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 2, 2));
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv + " limit 1"), row(1, 2, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 2, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s limit 1"), row(1, 2, 2));
         updateView("UPDATE %s USING TIMESTAMP 11 SET a = 1 WHERE k = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 1, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 1, 2));
         assertRowsIgnoringOrder(execute("SELECT k,a,b from %s"), row(1, 1, 2));
 
         // set non-key base column as tombstone, view row is removed with shadowable
         updateView("UPDATE %s USING TIMESTAMP 12 SET a = null WHERE k = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"));
         assertRowsIgnoringOrder(execute("SELECT k,a,b from %s"), row(1, null, 2));
 
         // column b should be alive
         updateView("UPDATE %s USING TIMESTAMP 13 SET a = 1 WHERE k = 1;");
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRowsIgnoringOrder(execute("SELECT k,a,b from " + mv), row(1, 1, 2));
+        assertRowsIgnoringOrder(executeView("SELECT k,a,b from %s"), row(1, 1, 2));
         assertRowsIgnoringOrder(execute("SELECT k,a,b from %s"), row(1, 1, 2));
 
         assertInvalidMessage(String.format("Cannot drop column a on base table %s with materialized views", baseTable), "ALTER TABLE %s DROP a");
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering1Test.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering1Test.java
index 6d4e487..8a976a0 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering1Test.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering1Test.java
@@ -18,21 +18,10 @@
 
 package org.apache.cassandra.cql3;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.transport.ProtocolVersion;
 
 /* ViewFilteringTest class has been split into multiple ones because of timeout issues (CASSANDRA-16670)
  * Any changes here check if they apply to the other classes
@@ -41,54 +30,8 @@ import org.apache.cassandra.transport.ProtocolVersion;
  * - ViewFilteringClustering2Test
  * - ViewFilteringTest
  */
-@RunWith(Parameterized.class)
-public class ViewFilteringClustering1Test extends CQLTester
+public class ViewFilteringClustering1Test extends ViewAbstractParameterizedTest
 {
-    @Parameterized.Parameter
-    public ProtocolVersion version;
-
-    @Parameterized.Parameters()
-    public static Collection<Object[]> versions()
-    {
-        return ViewFilteringTest.versions();
-    }
-
-    private final List<String> views = new ArrayList<>();
-
-    @BeforeClass
-    public static void startup()
-    {
-        ViewFilteringTest.startup();
-    }
-
-    @AfterClass
-    public static void tearDown()
-    {
-        ViewFilteringTest.tearDown();
-    }
-
-    @Before
-    public void begin()
-    {
-        ViewFilteringTest.beginSetup(views);
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        ViewFilteringTest.endSetup(views, version, this);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        ViewFilteringTest.createView(name, query, views, version, this);
-    }
-
-    private void dropView(String name) throws Throwable
-    {
-        ViewFilteringTest.dropView(name, views, version, this);
-    }
-
     @Test
     public void testClusteringKeyEQRestrictions() throws Throwable
     {
@@ -97,9 +40,6 @@ public class ViewFilteringClustering1Test extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -112,88 +52,79 @@ public class ViewFilteringClustering1Test extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where b = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b = 1 AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a IS NOT NULL AND b = 1 AND c IS NOT NULL " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
-
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 2, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, 0, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 2, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
-                                    row(0, 1, 1, 0)
-            );
+                                    row(0, 1, 1, 0));
 
-            dropView("mv_test" + i);
+            dropView();
             dropTable("DROP TABLE %s");
         }
     }
@@ -206,9 +137,6 @@ public class ViewFilteringClustering1Test extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -220,88 +148,79 @@ public class ViewFilteringClustering1Test extends CQLTester
 
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b >= 1 AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a IS NOT NULL AND b >= 1 AND c IS NOT NULL " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
-
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, -1, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, -1, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, -1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
-                                    row(0, 1, 1, 0)
-            );
+                                    row(0, 1, 1, 0));
 
-            dropView("mv_test" + i);
+            dropView();
             dropTable("DROP TABLE %s");
         }
     }
@@ -314,9 +233,6 @@ public class ViewFilteringClustering1Test extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -330,95 +246,86 @@ public class ViewFilteringClustering1Test extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where b = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IN (1, 2) AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
-
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a IS NOT NULL AND b IN (1, 2) AND c IS NOT NULL " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, -1, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
                                     row(1, 1, 2, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, -1, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
                                     row(1, 1, 2, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
                                     row(1, 1, 2, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, -1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
                                     row(1, 1, 2, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 1, 0),
                                     row(1, 1, 2, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
-                                    row(0, 1, 1, 0)
-            );
+                                    row(0, 1, 1, 0));
 
-            dropView("mv_test" + i);
+            dropView();
             dropTable("DROP TABLE %s");
         }
     }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering2Test.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering2Test.java
index d1ba842..06b1945 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering2Test.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringClustering2Test.java
@@ -18,21 +18,10 @@
 
 package org.apache.cassandra.cql3;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.transport.ProtocolVersion;
 
 /* ViewFilteringTest class has been split into multiple ones because of timeout issues (CASSANDRA-16670)
  * Any changes here check if they apply to the other classes
@@ -41,54 +30,8 @@ import org.apache.cassandra.transport.ProtocolVersion;
  * - ViewFilteringClustering2Test
  * - ViewFilteringTest
  */
-@RunWith(Parameterized.class)
-public class ViewFilteringClustering2Test extends CQLTester
+public class ViewFilteringClustering2Test extends ViewAbstractParameterizedTest
 {
-    @Parameterized.Parameter
-    public ProtocolVersion version;
-
-    @Parameterized.Parameters()
-    public static Collection<Object[]> versions()
-    {
-        return ViewFilteringTest.versions();
-    }
-
-    private final List<String> views = new ArrayList<>();
-
-    @BeforeClass
-    public static void startup()
-    {
-        ViewFilteringTest.startup();
-    }
-
-    @AfterClass
-    public static void tearDown()
-    {
-        ViewFilteringTest.tearDown();
-    }
-
-    @Before
-    public void begin()
-    {
-        ViewFilteringTest.beginSetup(views);
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        ViewFilteringTest.endSetup(views, version, this);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        ViewFilteringTest.createView(name, query, views, version, this);
-    }
-
-    private void dropView(String name) throws Throwable
-    {
-        ViewFilteringTest.dropView(name, views, version, this);
-    }
-
     @Test
     public void testClusteringKeyMultiColumnRestrictions() throws Throwable
     {
@@ -97,9 +40,6 @@ public class ViewFilteringClustering2Test extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -113,91 +53,82 @@ public class ViewFilteringClustering2Test extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where b = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND (b, c) >= (1, 0) PRIMARY KEY " + mvPrimaryKeys.get(i));
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a IS NOT NULL AND (b, c) >= (1, 0) " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
-
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, -1, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, -1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, -1, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, -1, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, -1);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, -1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 1, 0, 0),
-                                    row(0, 1, 1, 0)
-            );
+                                    row(0, 1, 1, 0));
 
-            dropView("mv_test" + i);
+            dropView();
             dropTable("DROP TABLE %s");
         }
     }
@@ -210,9 +141,6 @@ public class ViewFilteringClustering2Test extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -226,87 +154,78 @@ public class ViewFilteringClustering2Test extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where b = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c = 1 PRIMARY KEY " + mvPrimaryKeys.get(i));
-
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a IS NOT NULL AND b IS NOT NULL AND c = 1 " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, -1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, -1, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 2, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 2, 1, 1, 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 2),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, -1);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, -1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, -1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 2),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
                                     row(1, 0, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
-                                    row(0, 1, 1, 0)
-            );
+                                    row(0, 1, 1, 0));
 
             // insert a partition with one matching and one non-matching row using a batch (CASSANDRA-10614)
             String tableName = KEYSPACE + "." + currentTable();
@@ -316,13 +235,12 @@ public class ViewFilteringClustering2Test extends CQLTester
                     "APPLY BATCH",
                     4, 4, 0, 0,
                     4, 4, 1, 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(0, 0, 1, 0),
                                     row(0, 1, 1, 0),
-                                    row(4, 4, 1, 1)
-            );
+                                    row(4, 4, 1, 1));
 
-            dropView("mv_test" + i);
+            dropView();
             dropTable("DROP TABLE %s");
         }
     }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringPKTest.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringPKTest.java
index 09d220d..24b32c7 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringPKTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringPKTest.java
@@ -18,21 +18,10 @@
 
 package org.apache.cassandra.cql3;
 
-import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.List;
 
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
 import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.transport.ProtocolVersion;
 
 /* ViewFilteringTest class has been split into multiple ones because of timeout issues (CASSANDRA-16670)
  * Any changes here check if they apply to the other classes
@@ -41,54 +30,8 @@ import org.apache.cassandra.transport.ProtocolVersion;
  * - ViewFilteringClustering2Test
  * - ViewFilteringTest
  */
-@RunWith(Parameterized.class)
-public class ViewFilteringPKTest extends CQLTester
+public class ViewFilteringPKTest extends ViewAbstractParameterizedTest
 {
-    @Parameterized.Parameter
-    public ProtocolVersion version;
-
-    @Parameterized.Parameters()
-    public static Collection<Object[]> versions()
-    {
-        return ViewFilteringTest.versions();
-    }
-
-    private final List<String> views = new ArrayList<>();
-
-    @BeforeClass
-    public static void startup()
-    {
-        ViewFilteringTest.startup();
-    }
-
-    @AfterClass
-    public static void tearDown()
-    {
-        ViewFilteringTest.tearDown();
-    }
-
-    @Before
-    public void begin()
-    {
-        ViewFilteringTest.beginSetup(views);
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        ViewFilteringTest.endSetup(views, version, this);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        ViewFilteringTest.createView(name, query, views, version, this);
-    }
-
-    private void dropView(String name) throws Throwable
-    {
-        ViewFilteringTest.dropView(name, views, version, this);
-    }
-
     @Test
     public void testPartitionKeyFilteringUnrestrictedPart() throws Throwable
     {
@@ -97,9 +40,6 @@ public class ViewFilteringPKTest extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
@@ -110,87 +50,79 @@ public class ViewFilteringPKTest extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where a = 1
-            String viewName= "mv_test" + i;
-            createView(viewName, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
-
-            ViewFilteringTest.waitForView(keyspace(), viewName);
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                             "WHERE a = 1 AND b IS NOT NULL AND c IS NOT NULL " +
+                             "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 0, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 1, 0),
                                     row(1, 1, 2, 0));
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 1);
-            assertEmpty(execute("SELECT * FROM mv_test" + i));
+            assertEmpty(executeView("SELECT * FROM %s"));
         }
     }
 
@@ -202,9 +134,6 @@ public class ViewFilteringPKTest extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0,  1, 1);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 10, 1, 2);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0,  2, 1);
@@ -215,70 +144,62 @@ public class ViewFilteringPKTest extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where a = 1
-            String viewName= "mv_test" + i;
-            createView(viewName, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a > 0 AND b > 5 AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
-
-            ViewFilteringTest.waitForView(keyspace(), viewName);
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a > 0 AND b > 5 AND c IS NOT NULL " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
-                                    row(2, 10, 3, 2)
-            );
+                                    row(2, 10, 3, 2));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
-                                    row(2, 10, 3, 2)
-            );
+                                    row(2, 10, 3, 2));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 3, 10, 4, 2);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
                                     row(2, 10, 3, 2),
-                                    row(3, 10, 4, 2)
-            );
+                                    row(3, 10, 4, 2));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 0, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
                                     row(2, 10, 3, 2),
-                                    row(3, 10, 4, 2)
-            );
+                                    row(3, 10, 4, 2));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 100, 3, 10, 4);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
                                     row(2, 10, 3, 2),
-                                    row(3, 10, 4, 100)
-            );
+                                    row(3, 10, 4, 100));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
                                     row(2, 10, 3, 2),
-                                    row(3, 10, 4, 100)
-            );
+                                    row(3, 10, 4, 100));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 10, 2, 2),
                                     row(2, 10, 3, 2),
-                                    row(3, 10, 4, 100)
-            );
+                                    row(3, 10, 4, 100));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 10);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(2, 10, 3, 2),
                                     row(3, 10, 4, 100));
         }
@@ -292,9 +213,6 @@ public class ViewFilteringPKTest extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 0, 0, 0);
@@ -305,83 +223,75 @@ public class ViewFilteringPKTest extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where a = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b IS NOT NULL AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a = 1 AND b IS NOT NULL AND c IS NOT NULL " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
-
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 0, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 0, 0),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertEmpty(execute("SELECT * FROM mv_test" + i));
+            assertEmpty(executeView("SELECT * FROM %s"));
         }
     }
 
@@ -393,9 +303,6 @@ public class ViewFilteringPKTest extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -408,72 +315,64 @@ public class ViewFilteringPKTest extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where a = 1 and b = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND c IS NOT NULL PRIMARY KEY " + mvPrimaryKeys.get(i));
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a = 1 AND b = 1 AND c IS NOT NULL " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
-
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 0, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 0, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 0, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 0, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 0, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 1, 0);
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 0, 1),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 1, 1, 0),
-                                    row(1, 1, 2, 0)
-            );
+                                    row(1, 1, 2, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 1);
-            assertEmpty(execute("SELECT * FROM mv_test" + i));
+            assertEmpty(executeView("SELECT * FROM %s"));
         }
     }
 
@@ -481,8 +380,6 @@ public class ViewFilteringPKTest extends CQLTester
     public void testCompoundPartitionKeyRestrictionsNotIncludeAll() throws Throwable
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY ((a, b), c))");
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
 
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
@@ -494,72 +391,64 @@ public class ViewFilteringPKTest extends CQLTester
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 0);
 
         // only accept rows where a = 1 and b = 1, don't include column d in the selection
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT a, b, c FROM %%s WHERE a = 1 AND b = 1 AND c IS NOT NULL PRIMARY KEY ((a, b), c)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c FROM %s " +
+                   "WHERE a = 1 AND b = 1 AND c IS NOT NULL " +
+                   "PRIMARY KEY ((a, b), c)");
 
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test"))
-            Thread.sleep(10);
-
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 0),
-                   row(1, 1, 1)
-        );
+                   row(1, 1, 1));
 
         // insert new rows that do not match the filter
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 0, 0);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 0, 0);
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 0),
-                   row(1, 1, 1)
-        );
+                   row(1, 1, 1));
 
         // insert new row that does match the filter
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 2, 0);
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 0),
                    row(1, 1, 1),
-                   row(1, 1, 2)
-        );
+                   row(1, 1, 2));
 
         // update rows that don't match the filter
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 0, 0);
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 0, 0);
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 0, 1, 0);
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 0),
                    row(1, 1, 1),
-                   row(1, 1, 2)
-        );
+                   row(1, 1, 2));
 
         // update a row that does match the filter
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, 1, 0);
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 0),
                    row(1, 1, 1),
-                   row(1, 1, 2)
-        );
+                   row(1, 1, 2));
 
         // delete rows that don't match the filter
         execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 0, 0);
         execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 0, 0);
         execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 0, 1, 0);
         execute("DELETE FROM %s WHERE a = ? AND b = ?", 0, 0);
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 0),
                    row(1, 1, 1),
-                   row(1, 1, 2)
-        );
+                   row(1, 1, 2));
 
         // delete a row that does match the filter
         execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 0);
-        assertRows(execute("SELECT * FROM mv_test"),
+        assertRows(executeView("SELECT * FROM %s"),
                    row(1, 1, 1),
-                   row(1, 1, 2)
-        );
+                   row(1, 1, 2));
 
         // delete a partition that matches the filter
         execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 1);
-        assertEmpty(execute("SELECT * FROM mv_test"));
+        assertEmpty(executeView("SELECT * FROM %s"));
     }
 
     @Test
@@ -570,9 +459,6 @@ public class ViewFilteringPKTest extends CQLTester
         {
             createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-            execute("USE " + keyspace());
-            executeNet(version, "USE " + keyspace());
-
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -586,71 +472,63 @@ public class ViewFilteringPKTest extends CQLTester
             logger.info("Testing MV primary key: {}", mvPrimaryKeys.get(i));
 
             // only accept rows where b = 1
-            createView("mv_test" + i, "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b IS NOT NULL AND c = 1 PRIMARY KEY " + mvPrimaryKeys.get(i));
-
-            while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test" + i))
-                Thread.sleep(10);
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE a = 1 AND b IS NOT NULL AND c = 1 " +
+                       "PRIMARY KEY " + mvPrimaryKeys.get(i));
 
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new rows that do not match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 0, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
-                                    row(1, 1, 1, 0)
-            );
+                                    row(1, 1, 1, 0));
 
             // insert new row that does match the filter
             execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // update rows that don't match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 1, 1, -1, 0);
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 0, 1, 1, 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // update a row that does match the filter
             execute("UPDATE %s SET d = ? WHERE a = ? AND b = ? AND c = ?", 2, 1, 1, 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 2),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete rows that don't match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, -1);
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 2, 0, 1);
             execute("DELETE FROM %s WHERE a = ?", 0);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
                                     row(1, 1, 1, 2),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete a row that does match the filter
             execute("DELETE FROM %s WHERE a = ? AND b = ? AND c = ?", 1, 1, 1);
-            assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test" + i),
+            assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                     row(1, 0, 1, 0),
-                                    row(1, 2, 1, 0)
-            );
+                                    row(1, 2, 1, 0));
 
             // delete a partition that matches the filter
             execute("DELETE FROM %s WHERE a = ?", 1);
-            assertEmpty(execute("SELECT a, b, c, d FROM mv_test" + i));
+            assertEmpty(executeView("SELECT a, b, c, d FROM %s"));
 
-            dropView("mv_test" + i);
+            dropView();
             dropTable("DROP TABLE %s");
         }
     }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
index 374f79f..37f755b 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewFilteringTest.java
@@ -18,27 +18,17 @@
 
 package org.apache.cassandra.cql3;
 
-import java.util.*;
-import java.util.stream.Collectors;
+import java.util.Arrays;
+import java.util.List;
 
-import org.junit.After;
 import org.junit.AfterClass;
-import org.junit.Before;
+import org.junit.Assert;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import com.datastax.driver.core.exceptions.InvalidQueryException;
-import org.junit.Assert;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-
-import com.datastax.driver.core.exceptions.OperationTimedOutException;
-import org.apache.cassandra.concurrent.SEPExecutor;
-import org.apache.cassandra.concurrent.Stage;
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.SystemKeyspace;
-import org.apache.cassandra.transport.ProtocolVersion;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.FBUtilities;
 
 /* ViewFilteringTest class has been split into multiple ones because of timeout issues (CASSANDRA-16670)
@@ -48,26 +38,12 @@ import org.apache.cassandra.utils.FBUtilities;
  * - ViewFilteringClustering2Test
  * - ViewFilteringTest
  */
-@RunWith(Parameterized.class)
-public class ViewFilteringTest extends CQLTester
+public class ViewFilteringTest extends ViewAbstractParameterizedTest
 {
-    @Parameterized.Parameter
-    public ProtocolVersion version;
-
-    @Parameterized.Parameters()
-    public static Collection<Object[]> versions()
-    {
-        return ProtocolVersion.SUPPORTED.stream()
-                                        .map(v -> new Object[]{v})
-                                        .collect(Collectors.toList());
-    }
-
-    private final List<String> views = new ArrayList<>();
-
     @BeforeClass
     public static void startup()
     {
-        requireNetwork();
+        ViewAbstractParameterizedTest.startup();
         System.setProperty("cassandra.mv.allow_filtering_nonkey_columns_unsafe", "true");
     }
 
@@ -77,78 +53,6 @@ public class ViewFilteringTest extends CQLTester
         System.setProperty("cassandra.mv.allow_filtering_nonkey_columns_unsafe", "false");
     }
 
-    @Before
-    public void begin()
-    {
-        beginSetup(views);
-    }
-
-    public static void beginSetup(List<String> views)
-    {
-        views.clear();
-    }
-
-    @After
-    public void end() throws Throwable
-    {
-        endSetup(views, version, this);
-    }
-
-    public static void endSetup(List<String> views, ProtocolVersion version, CQLTester tester) throws Throwable
-    {
-        for (String viewName : views)
-            tester.executeNet(version, "DROP MATERIALIZED VIEW " + viewName);
-    }
-
-    private void createView(String name, String query) throws Throwable
-    {
-        createView(name, query, views, version, this);
-    }
-
-    public static void createView(String name, String query, List<String> views, ProtocolVersion version, CQLTester tester) throws Throwable
-    {
-        try
-        {
-            tester.executeNet(version, 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);
-        }
-        catch (OperationTimedOutException ex)
-        {
-            // ... except for timeout, when we actually do not know whether the view was created or not
-            views.add(name);
-            throw ex;
-        }
-    }
-
-    private void updateView(String query, Object... params) throws Throwable
-    {
-        executeNet(version, query, params);
-        while (!(((SEPExecutor) Stage.VIEW_MUTATION.executor()).getPendingTaskCount() == 0
-                 && ((SEPExecutor) Stage.VIEW_MUTATION.executor()).getActiveTaskCount() == 0))
-        {
-            Thread.sleep(1);
-        }
-    }
-
-    private void dropView(String name) throws Throwable
-    {
-        dropView(name, views, version, this);
-    }
-
-    public static void dropView(String name, List<String> views, ProtocolVersion version, CQLTester tester) throws Throwable
-    {
-        tester.executeNet(version, "DROP MATERIALIZED VIEW " + name);
-        views.remove(name);
-    }
-
-    public static void waitForView(String keyspace, String view) throws InterruptedException
-    {
-        while (!SystemKeyspace.isViewBuilt(keyspace, view))
-            Thread.sleep(10);
-    }
-
     // TODO will revise the non-pk filter condition in MV, see CASSANDRA-11500
     @Ignore
     @Test
@@ -170,205 +74,195 @@ public class ViewFilteringTest extends CQLTester
         // CASSANDRA-13547: able to shadow entire view row if base column used in filter condition is modified
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        createView("mv_test1",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL and c = 1  PRIMARY KEY (a, b)");
-        createView("mv_test2",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT c, d FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL and c = 1 and d = 1 PRIMARY KEY (a, b)");
-        createView("mv_test3",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, d FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b)");
-        createView("mv_test4",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT c FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL and c = 1 PRIMARY KEY (a, b)");
-        createView("mv_test5",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT c FROM %%s WHERE a IS NOT NULL and d = 1 PRIMARY KEY (a, d)");
-        createView("mv_test6",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT c FROM %%s WHERE a = 1 and d IS NOT NULL PRIMARY KEY (a, d)");
-
-        waitForView(keyspace(), "mv_test1");
-        waitForView(keyspace(), "mv_test2");
-        waitForView(keyspace(), "mv_test3");
-        waitForView(keyspace(), "mv_test4");
-        waitForView(keyspace(), "mv_test5");
-        waitForView(keyspace(), "mv_test6");
+        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL and c = 1  PRIMARY KEY (a, b)");
+        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT c, d FROM %s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL and c = 1 and d = 1 PRIMARY KEY (a, b)");
+        String mv3 = createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c, d FROM %%s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (a, b)");
+        String mv4 = createView("CREATE MATERIALIZED VIEW %s AS SELECT c FROM %s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL and c = 1 PRIMARY KEY (a, b)");
+        String mv5 = createView("CREATE MATERIALIZED VIEW %s AS SELECT c FROM %s " +
+                                "WHERE a IS NOT NULL and d = 1 PRIMARY KEY (a, d)");
+        String mv6 = createView("CREATE MATERIALIZED VIEW %s AS SELECT c FROM %s " +
+                                "WHERE a = 1 and d IS NOT NULL PRIMARY KEY (a, d)");
 
         Keyspace ks = Keyspace.open(keyspace());
-        ks.getColumnFamilyStore("mv_test1").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test2").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test3").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test4").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test5").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test6").disableAutoCompaction();
+        ks.getColumnFamilyStore(mv1).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv2).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv3).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv4).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv5).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv6).disableAutoCompaction();
 
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) using timestamp 0", 1, 1, 1, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         // views should be updated.
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 1));
 
         updateView("UPDATE %s using timestamp 1 set c = ? WHERE a=?", 0, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowCount(execute("SELECT * FROM mv_test1"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 0, 1));
-        assertRowCount(execute("SELECT * FROM mv_test4"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 0));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 0));
+        assertRowCount(execute("SELECT * FROM " + mv1), 0);
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 0, 1));
+        assertRowCount(execute("SELECT * FROM " + mv4), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 0));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 0));
 
         updateView("UPDATE %s using timestamp 2 set c = ? WHERE a=?", 1, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         // row should be back in views.
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 1));
 
         updateView("UPDATE %s using timestamp 3 set d = ? WHERE a=?", 0, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1, 0));
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 1, 0));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1, 1));
-        assertRowCount(execute("SELECT * FROM mv_test5"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 0, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1, 0));
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 1, 0));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1, 1));
+        assertRowCount(execute("SELECT * FROM " + mv5), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 0, 1));
 
         updateView("UPDATE %s using timestamp 4 set c = ? WHERE a=?", 0, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowCount(execute("SELECT * FROM mv_test1"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 0, 0));
-        assertRowCount(execute("SELECT * FROM mv_test4"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test5"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 0, 0));
+        assertRowCount(execute("SELECT * FROM " + mv1), 0);
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 0, 0));
+        assertRowCount(execute("SELECT * FROM " + mv4), 0);
+        assertRowCount(execute("SELECT * FROM " + mv5), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 0, 0));
 
         updateView("UPDATE %s using timestamp 5 set d = ? WHERE a=?", 1, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         // should not update as c=0
-        assertRowCount(execute("SELECT * FROM mv_test1"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 0, 1));
-        assertRowCount(execute("SELECT * FROM mv_test4"), 0);
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 0));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 0));
+        assertRowCount(execute("SELECT * FROM " + mv1), 0);
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 0, 1));
+        assertRowCount(execute("SELECT * FROM " + mv4), 0);
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 0));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 0));
 
         updateView("UPDATE %s using timestamp 6 set c = ? WHERE a=?", 1, 1);
 
         // row should be back in views.
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 1));
 
         updateView("UPDATE %s using timestamp 7 set b = ? WHERE a=?", 2, 1);
         if (flush)
         {
             FBUtilities.waitOnFutures(ks.flush());
-            for (String view : views)
+            for (String view : getViews())
                 ks.getColumnFamilyStore(view).forceMajorCompaction();
         }
         // row should be back in views.
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 2, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"), row(1, 2, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 2, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 2, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 2, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2), row(1, 2, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 2, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 2, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 1));
 
         updateView("DELETE b, c FROM %s using timestamp 6 WHERE a=?", 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT * FROM %s"), row(1, 2, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 2, null, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, null));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 2, null, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, null));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, null));
 
         updateView("DELETE FROM %s using timestamp 8 where a=?", 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowCount(execute("SELECT * FROM mv_test1"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test3"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test4"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test5"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test6"), 0);
+        assertRowCount(execute("SELECT * FROM " + mv1), 0);
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowCount(execute("SELECT * FROM " + mv3), 0);
+        assertRowCount(execute("SELECT * FROM " + mv4), 0);
+        assertRowCount(execute("SELECT * FROM " + mv5), 0);
+        assertRowCount(execute("SELECT * FROM " + mv6), 0);
 
         updateView("UPDATE %s using timestamp 9 set b = ?,c = ? where a=?", 1, 1, 1); // upsert
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1, null));
-        assertRows(execute("SELECT * FROM mv_test2"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 1, null));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1, 1));
-        assertRows(execute("SELECT * FROM mv_test5"));
-        assertRows(execute("SELECT * FROM mv_test6"));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1, null));
+        assertRows(execute("SELECT * FROM " + mv2));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 1, null));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1, 1));
+        assertRows(execute("SELECT * FROM " + mv5));
+        assertRows(execute("SELECT * FROM " + mv6));
 
         updateView("DELETE FROM %s using timestamp 10 where a=?", 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowCount(execute("SELECT * FROM mv_test1"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test3"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test4"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test5"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test6"), 0);
+        assertRowCount(execute("SELECT * FROM " + mv1), 0);
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowCount(execute("SELECT * FROM " + mv3), 0);
+        assertRowCount(execute("SELECT * FROM " + mv4), 0);
+        assertRowCount(execute("SELECT * FROM " + mv5), 0);
+        assertRowCount(execute("SELECT * FROM " + mv6), 0);
 
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) using timestamp 11", 1, 1, 1, 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         // row should be back in views.
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test5"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test6"), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv5), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv6), row(1, 1, 1));
 
         updateView("DELETE FROM %s using timestamp 12 where a=?", 1);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowCount(execute("SELECT * FROM mv_test1"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test2"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test3"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test4"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test5"), 0);
-        assertRowCount(execute("SELECT * FROM mv_test6"), 0);
-
-        dropView("mv_test1");
-        dropView("mv_test2");
-        dropView("mv_test3");
-        dropView("mv_test4");
-        dropView("mv_test5");
-        dropView("mv_test6");
+        assertRowCount(execute("SELECT * FROM " + mv1), 0);
+        assertRowCount(execute("SELECT * FROM " + mv2), 0);
+        assertRowCount(execute("SELECT * FROM " + mv3), 0);
+        assertRowCount(execute("SELECT * FROM " + mv4), 0);
+        assertRowCount(execute("SELECT * FROM " + mv5), 0);
+        assertRowCount(execute("SELECT * FROM " + mv6), 0);
+
+        dropView(mv1);
+        dropView(mv2);
+        dropView(mv3);
+        dropView(mv4);
+        dropView(mv5);
+        dropView(mv6);
         dropTable("DROP TABLE %s");
     }
 
@@ -379,23 +273,19 @@ public class ViewFilteringTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, l list<int>, s set<int>, m map<int,int>, PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        createView("mv_test1",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a,b,c FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL "
-                   + "and l contains (1) AND s contains (1) AND m contains key (1) PRIMARY KEY (a, b, c)");
-        createView("mv_test2",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s WHERE a IS NOT NULL and b IS NOT NULL AND l contains (1) PRIMARY KEY (a, b)");
-        createView("mv_test3",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND s contains (1) PRIMARY KEY (a, b)");
-        createView("mv_test4",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND m contains key (1) PRIMARY KEY (a, b)");
-
-        waitForView(keyspace(), "mv_test1");
-        waitForView(keyspace(), "mv_test2");
-        waitForView(keyspace(), "mv_test3");
-        waitForView(keyspace(), "mv_test4");
+        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT a,b,c FROM %%s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND l contains (1) " +
+                                "AND s contains (1) AND m contains key (1) " +
+                                "PRIMARY KEY (a, b, c)");
+        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s " +
+                                "WHERE a IS NOT NULL and b IS NOT NULL AND l contains (1) " +
+                                "PRIMARY KEY (a, b)");
+        String mv3 = createView("CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL AND s contains (1) " +
+                                "PRIMARY KEY (a, b)");
+        String mv4 = createView("CREATE MATERIALIZED VIEW %s AS SELECT a,b FROM %%s " +
+                                "WHERE a IS NOT NULL AND b IS NOT NULL AND m contains key (1) " +
+                                "PRIMARY KEY (a, b)");
 
         // not able to drop base column filtered in view
         assertInvalidMessage("Cannot drop column l, depended on by materialized views", "ALTER TABLE %s DROP l");
@@ -403,10 +293,10 @@ public class ViewFilteringTest extends CQLTester
         assertInvalidMessage("Cannot drop column m, depended on by materialized views", "ALTER TABLE %s DROP m");
 
         Keyspace ks = Keyspace.open(keyspace());
-        ks.getColumnFamilyStore("mv_test1").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test2").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test3").disableAutoCompaction();
-        ks.getColumnFamilyStore("mv_test4").disableAutoCompaction();
+        ks.getColumnFamilyStore(mv1).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv2).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv3).disableAutoCompaction();
+        ks.getColumnFamilyStore(mv4).disableAutoCompaction();
 
         execute("INSERT INTO %s (a, b, c, l, s, m) VALUES (?, ?, ?, ?, ?, ?) ",
                 1,
@@ -417,46 +307,46 @@ public class ViewFilteringTest extends CQLTester
                 map(1, 1, 2, 2));
         FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"), row(1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1), row(1, 1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1));
 
-        execute("UPDATE %s SET l=l-[1] WHERE a = 1 AND b = 1" );
+        execute("UPDATE %s SET l=l-[1] WHERE a = 1 AND b = 1");
         FBUtilities.waitOnFutures(ks.flush());
 
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1));
 
         execute("UPDATE %s SET s=s-{2}, m=m-{2} WHERE a = 1 AND b = 1");
         FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT a,b,c FROM %s"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4), row(1, 1));
 
         execute("UPDATE %s SET  m=m-{1} WHERE a = 1 AND b = 1");
         FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT a,b,c FROM %s"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4));
 
         // filter conditions result not changed
         execute("UPDATE %s SET  l=l+[2], s=s-{0}, m=m+{3:3} WHERE a = 1 AND b = 1");
         FBUtilities.waitOnFutures(ks.flush());
 
         assertRowsIgnoringOrder(execute("SELECT a,b,c FROM %s"), row(1, 1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test1"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test2"));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test3"), row(1, 1));
-        assertRowsIgnoringOrder(execute("SELECT * FROM mv_test4"));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv2));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv3), row(1, 1));
+        assertRowsIgnoringOrder(execute("SELECT * FROM " + mv4));
     }
 
     @Test
@@ -464,49 +354,50 @@ public class ViewFilteringTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY((a, b), c, d))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
         // IS NOT NULL is required on all PK statements that are not otherwise restricted
         List<String> badStatements = Arrays.asList(
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE b IS NOT NULL AND c IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND c IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "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 ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = ? AND b IS NOT NULL AND c is NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = blobAsInt(?) AND b IS NOT NULL AND c is NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s PRIMARY KEY (a, b, c, d)"
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE b IS NOT NULL AND c IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND c IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "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 ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = ? AND b IS NOT NULL AND c is NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = blobAsInt(?) AND b IS NOT NULL AND c is NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s PRIMARY KEY (a, b, c, d)"
         );
 
         for (String badStatement : badStatements)
         {
             try
             {
-                createView("mv1_test", badStatement);
+                createView(badStatement);
                 Assert.fail("Create MV statement should have failed due to missing IS NOT NULL restriction: " + badStatement);
             }
-            catch (InvalidQueryException exc) {}
+            catch (RuntimeException e)
+            {
+                Assert.assertSame(InvalidRequestException.class, e.getCause().getClass());
+            }
         }
 
         List<String> goodStatements = Arrays.asList(
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND c IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c = 1 AND d IS NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND c > 1 AND d IS NOT NULL PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND c = 1 AND d IN (1, 2, 3) PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND (c, d) = (1, 1) PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND (c, d) > (1, 1) PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = 1 AND b = 1 AND (c, d) IN ((1, 1), (2, 2)) PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = (int) 1 AND b = 1 AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a = blobAsInt(intAsBlob(1)) AND b = 1 AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)"
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND c IS NOT NULL AND d is NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL AND c = 1 AND d IS NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND c > 1 AND d IS NOT NULL PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND c = 1 AND d IN (1, 2, 3) PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND (c, d) = (1, 1) PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND (c, d) > (1, 1) PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = 1 AND b = 1 AND (c, d) IN ((1, 1), (2, 2)) PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = (int) 1 AND b = 1 AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)",
+        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a = blobAsInt(intAsBlob(1)) AND b = 1 AND c = 1 AND d = 1 PRIMARY KEY ((a, b), c, d)"
         );
 
         for (int i = 0; i < goodStatements.size(); i++)
         {
+            String mv;
             try
             {
-                createView("mv" + i + "_test", goodStatements.get(i));
+                mv = createView(goodStatements.get(i));
             }
             catch (Exception e)
             {
@@ -515,7 +406,7 @@ public class ViewFilteringTest extends CQLTester
 
             try
             {
-                executeNet(version, "ALTER MATERIALIZED VIEW mv" + i + "_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
+                executeNet("ALTER MATERIALIZED VIEW " + mv + " WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
             }
             catch (Exception e)
             {
@@ -529,36 +420,28 @@ public class ViewFilteringTest extends CQLTester
     {
         createTable("CREATE TABLE %s (\"theKey\" int, \"theClustering\" int, \"the\"\"Value\" int, PRIMARY KEY (\"theKey\", \"theClustering\"))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
         execute("INSERT INTO %s (\"theKey\", \"theClustering\", \"the\"\"Value\") VALUES (?, ?, ?)", 0, 0, 0);
         execute("INSERT INTO %s (\"theKey\", \"theClustering\", \"the\"\"Value\") VALUES (?, ?, ?)", 0, 1, 0);
         execute("INSERT INTO %s (\"theKey\", \"theClustering\", \"the\"\"Value\") VALUES (?, ?, ?)", 1, 0, 0);
         execute("INSERT INTO %s (\"theKey\", \"theClustering\", \"the\"\"Value\") VALUES (?, ?, ?)", 1, 1, 0);
 
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
-                              "WHERE \"theKey\" = 1 AND \"theClustering\" = 1 AND \"the\"\"Value\" IS NOT NULL " +
-                              "PRIMARY KEY (\"theKey\", \"theClustering\")");
+        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                                "WHERE \"theKey\" = 1 AND \"theClustering\" = 1 AND \"the\"\"Value\" 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\", \"the\"\"Value\" FROM %%s " +
-                               "WHERE \"theKey\" = 1 AND \"theClustering\" = 1 AND \"the\"\"Value\" IS NOT NULL " +
-                               "PRIMARY KEY (\"theKey\", \"theClustering\")");
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test2"))
-            Thread.sleep(10);
+        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT \"theKey\", \"theClustering\", \"the\"\"Value\" FROM %s " +
+                                "WHERE \"theKey\" = 1 AND \"theClustering\" = 1 AND \"the\"\"Value\" IS NOT NULL " +
+                                "PRIMARY KEY (\"theKey\", \"theClustering\")");
 
-        for (String mvname : Arrays.asList("mv_test", "mv_test2"))
+        for (String mvname : Arrays.asList(mv1, mv2))
         {
             assertRowsIgnoringOrder(execute("SELECT \"theKey\", \"theClustering\", \"the\"\"Value\" FROM " + mvname),
-                                    row(1, 1, 0)
-            );
+                                    row(1, 1, 0));
         }
 
-        executeNet(version, "ALTER TABLE %s RENAME \"theClustering\" TO \"Col\"");
+        executeNet("ALTER TABLE %s RENAME \"theClustering\" TO \"Col\"");
 
-        for (String mvname : Arrays.asList("mv_test", "mv_test2"))
+        for (String mvname : Arrays.asList(mv1, mv2))
         {
             assertRowsIgnoringOrder(execute("SELECT \"theKey\", \"Col\", \"the\"\"Value\" FROM " + mvname),
                                     row(1, 1, 0)
@@ -571,29 +454,23 @@ public class ViewFilteringTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 0, 2);
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 3);
 
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
-                              "WHERE a = blobAsInt(intAsBlob(1)) AND b IS NOT NULL " +
-                              "PRIMARY KEY (a, b)");
-
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test"))
-            Thread.sleep(10);
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE a = blobAsInt(intAsBlob(1)) AND b IS NOT NULL " +
+                   "PRIMARY KEY (a, b)");
 
-        assertRows(execute("SELECT a, b, c FROM mv_test"),
+        assertRows(executeView("SELECT a, b, c FROM %s"),
                    row(1, 0, 2),
                    row(1, 1, 3)
         );
 
-        executeNet(version, "ALTER TABLE %s RENAME a TO foo");
+        executeNet("ALTER TABLE %s RENAME a TO foo");
 
-        assertRows(execute("SELECT foo, b, c FROM mv_test"),
+        assertRows(executeView("SELECT foo, b, c FROM %s"),
                    row(1, 0, 2),
                    row(1, 1, 3)
         );
@@ -604,33 +481,27 @@ public class ViewFilteringTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 1, 1);
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 0, 2);
         execute("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 3);
 
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
-                              "WHERE a = (int) 1 AND b IS NOT NULL " +
-                              "PRIMARY KEY (a, b)");
-
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test"))
-            Thread.sleep(10);
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE a = (int) 1 AND b IS NOT NULL " +
+                   "PRIMARY KEY (a, b)");
 
-        assertRows(execute("SELECT a, b, c FROM mv_test"),
+        assertRows(executeView("SELECT a, b, c FROM %s"),
                    row(1, 0, 2),
                    row(1, 1, 3)
         );
 
-        executeNet(version, "ALTER TABLE %s RENAME a TO foo");
+        executeNet("ALTER TABLE %s RENAME a TO foo");
 
-        assertRows(execute("SELECT foo, b, c FROM mv_test"),
+        assertRows(executeView("SELECT foo, b, c FROM %s"),
                    row(1, 0, 2),
                    row(1, 1, 3)
         );
-    } 
+    }
 
     @Test
     public void testAllTypes() throws Throwable
@@ -685,35 +556,30 @@ public class ViewFilteringTest extends CQLTester
         "udtval frozen<" + myType + ">, " +
         "PRIMARY KEY (" + columnNames + "))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        createView(
-        "mv_test",
-        "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE " +
-        "asciival = 'abc' AND " +
-        "bigintval = 123 AND " +
-        "blobval = 0xfeed AND " +
-        "booleanval = true AND " +
-        "dateval = '1987-03-23' AND " +
-        "decimalval = 123.123 AND " +
-        "doubleval = 123.123 AND " +
-        "floatval = 123.123 AND " +
-        "inetval = '127.0.0.1' AND " +
-        "intval = 123 AND " +
-        "textval = 'abc' AND " +
-        "timeval = '07:35:07.000111222' AND " +
-        "timestampval = 123123123 AND " +
-        "timeuuidval = 6BDDC89A-5644-11E4-97FC-56847AFE9799 AND " +
-        "uuidval = 6BDDC89A-5644-11E4-97FC-56847AFE9799 AND " +
-        "varcharval = 'abc' AND " +
-        "varintval = 123123123 AND " +
-        "frozenlistval = [1, 2, 3] AND " +
-        "frozensetval = {6BDDC89A-5644-11E4-97FC-56847AFE9799} AND " +
-        "frozenmapval = {'a': 1, 'b': 2} AND " +
-        "tupleval = (1, 'foobar', 6BDDC89A-5644-11E4-97FC-56847AFE9799) AND " +
-        "udtval = {a: 1, b: 6BDDC89A-5644-11E4-97FC-56847AFE9799, c: {'foo', 'bar'}} " +
-        "PRIMARY KEY (" + columnNames + ")");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE " +
+                   "asciival = 'abc' AND " +
+                   "bigintval = 123 AND " +
+                   "blobval = 0xfeed AND " +
+                   "booleanval = true AND " +
+                   "dateval = '1987-03-23' AND " +
+                   "decimalval = 123.123 AND " +
+                   "doubleval = 123.123 AND " +
+                   "floatval = 123.123 AND " +
+                   "inetval = '127.0.0.1' AND " +
+                   "intval = 123 AND " +
+                   "textval = 'abc' AND " +
+                   "timeval = '07:35:07.000111222' AND " +
+                   "timestampval = 123123123 AND " +
+                   "timeuuidval = 6BDDC89A-5644-11E4-97FC-56847AFE9799 AND " +
+                   "uuidval = 6BDDC89A-5644-11E4-97FC-56847AFE9799 AND " +
+                   "varcharval = 'abc' AND " +
+                   "varintval = 123123123 AND " +
+                   "frozenlistval = [1, 2, 3] AND " +
+                   "frozensetval = {6BDDC89A-5644-11E4-97FC-56847AFE9799} AND " +
+                   "frozenmapval = {'a': 1, 'b': 2} AND " +
+                   "tupleval = (1, 'foobar', 6BDDC89A-5644-11E4-97FC-56847AFE9799) AND " +
+                   "udtval = {a: 1, b: 6BDDC89A-5644-11E4-97FC-56847AFE9799, c: {'foo', 'bar'}} " +
+                   "PRIMARY KEY (" + columnNames + ")");
 
         execute("INSERT INTO %s (" + columnNames + ") VALUES (" +
                 "'abc'," +
@@ -739,24 +605,26 @@ public class ViewFilteringTest extends CQLTester
                 "(1, 'foobar', 6BDDC89A-5644-11E4-97FC-56847AFE9799)," +
                 "{a: 1, b: 6BDDC89A-5644-11E4-97FC-56847AFE9799, c: {'foo', 'bar'}})");
 
-        assert !execute("SELECT * FROM mv_test").isEmpty();
+        assert !executeView("SELECT * FROM %s").isEmpty();
 
-        executeNet(version, "ALTER TABLE %s RENAME inetval TO foo");
-        assert !execute("SELECT * FROM mv_test").isEmpty();
+        executeNet("ALTER TABLE %s RENAME inetval TO foo");
+        assert !executeView("SELECT * FROM %s").isEmpty();
     }
 
     @Test
-    public void testMVCreationWithNonPrimaryRestrictions() throws Throwable
+    public void testMVCreationWithNonPrimaryRestrictions()
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        try {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d = 1 PRIMARY KEY (a, b, c)");
-            dropView("mv_test");
-        } catch(Exception e) {
+        try
+        {
+            String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                                   "WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d = 1 " +
+                                   "PRIMARY KEY (a, b, c)");
+            dropView(mv);
+        }
+        catch (Exception e)
+        {
             throw new RuntimeException("MV creation with non primary column restrictions failed.", e);
         }
 
@@ -768,9 +636,6 @@ public class ViewFilteringTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 1, 0);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 1, 0, 0);
@@ -781,12 +646,11 @@ public class ViewFilteringTest extends CQLTester
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 0);
 
         // only accept rows where c = 1
-        createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND c = 1 PRIMARY KEY (a, b, c)");
-
-        while (!SystemKeyspace.isViewBuilt(keyspace(), "mv_test"))
-            Thread.sleep(10);
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND c = 1 " +
+                   "PRIMARY KEY (a, b, c)");
 
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 0),
@@ -796,7 +660,7 @@ public class ViewFilteringTest extends CQLTester
         // insert new rows that do not match the filter
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 0, 0, 0);
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 2, 1, 2, 0);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 0),
@@ -805,7 +669,7 @@ public class ViewFilteringTest extends CQLTester
 
         // insert new row that does match the filter
         execute("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 1, 0);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 0),
@@ -816,7 +680,7 @@ public class ViewFilteringTest extends CQLTester
         // update rows that don't match the filter
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 2, 2, 0);
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 1, 2, 1);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 0),
@@ -826,7 +690,7 @@ public class ViewFilteringTest extends CQLTester
 
         // update a row that does match the filter
         execute("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 1, 1, 0);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 1),
@@ -836,7 +700,7 @@ public class ViewFilteringTest extends CQLTester
 
         // delete rows that don't match the filter
         execute("DELETE FROM %s WHERE a = ? AND b = ?", 2, 0);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 1),
@@ -846,7 +710,7 @@ public class ViewFilteringTest extends CQLTester
 
         // delete a row that does match the filter
         execute("DELETE FROM %s WHERE a = ? AND b = ?", 1, 2);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0),
                                 row(1, 0, 1, 1),
@@ -855,12 +719,12 @@ public class ViewFilteringTest extends CQLTester
 
         // delete a partition that matches the filter
         execute("DELETE FROM %s WHERE a = ?", 1);
-        assertRowsIgnoringOrder(execute("SELECT a, b, c, d FROM mv_test"),
+        assertRowsIgnoringOrder(executeView("SELECT a, b, c, d FROM %s"),
                                 row(0, 0, 1, 0),
                                 row(0, 1, 1, 0)
         );
 
-        dropView("mv_test");
+        dropView();
         dropTable("DROP TABLE %s");
     }
 
@@ -879,98 +743,97 @@ public class ViewFilteringTest extends CQLTester
     public void complexRestrictedTimestampUpdateTest(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(version, "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 AND c = 1 PRIMARY KEY (c, a, b)");
-        ks.getColumnFamilyStore("mv").disableAutoCompaction();
+        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                               "WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND c = 1 " +
+                               "PRIMARY KEY (c, a, b)");
+        ks.getColumnFamilyStore(mv).disableAutoCompaction();
 
         //Set initial values TS=0, matching the restriction and verify view
-        executeNet(version, "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));
+        executeNet("INSERT INTO %s (a, b, c, d) VALUES (0, 0, 1, 0) USING TIMESTAMP 0");
+        assertRows(executeView("SELECT d FROM %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         //update c's timestamp TS=2
-        executeNet(version, "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));
+        executeNet("UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        assertRows(executeView("SELECT d FROM %s 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(version, "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 = ?", 0, 0, 0));
+        executeNet("UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? and b = ? ", 0, 0, 0);
+        assertRows(executeView("SELECT d FROM %s WHERE c = ? and a = ? and b = ?", 0, 0, 0));
 
-        if(flush)
+        if (flush)
         {
-            ks.getColumnFamilyStore("mv").forceMajorCompaction();
+            ks.getColumnFamilyStore(mv).forceMajorCompaction();
             FBUtilities.waitOnFutures(ks.flush());
         }
 
         //change c's value back to 1 with TS=4, check we can see d
-        executeNet(version, "UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        executeNet("UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
         if (flush)
         {
-            ks.getColumnFamilyStore("mv").forceMajorCompaction();
+            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));
+        assertRows(executeView("SELECT d, e FROM %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, null));
 
         //Add e value @ TS=1
-        executeNet(version, "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));
+        executeNet("UPDATE %s USING TIMESTAMP 1 SET e = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        assertRows(executeView("SELECT d, e FROM %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, 1));
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         //Change d value @ TS=2
-        executeNet(version, "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));
+        executeNet("UPDATE %s USING TIMESTAMP 2 SET d = ? WHERE a = ? and b = ? ", 2, 0, 0);
+        assertRows(executeView("SELECT d FROM %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(2));
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
         //Change d value @ TS=3
-        executeNet(version, "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));
+        executeNet("UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? and b = ? ", 1, 0, 0);
+        assertRows(executeView("SELECT d FROM %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(1));
 
         //Tombstone c
-        executeNet(version, "DELETE FROM %s WHERE a = ? and b = ?", 0, 0);
-        assertRowsIgnoringOrder(execute("SELECT d from mv"));
-        assertRows(execute("SELECT d from mv"));
+        executeNet("DELETE FROM %s WHERE a = ? and b = ?", 0, 0);
+        assertRowsIgnoringOrder(executeView("SELECT d FROM %s"));
+        assertRows(executeView("SELECT d FROM %s"));
 
         //Add back without D
-        executeNet(version, "INSERT INTO %s (a, b, c) VALUES (0, 0, 1)");
+        executeNet("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));
+        assertRows(executeView("SELECT d FROM %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row((Object) null));
 
         //New partition
         // insert a row with timestamp 0
-        executeNet(version, "INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP 0", 1, 0, 1, 0, 0);
+        executeNet("INSERT INTO %s (a, b, c, d, e) VALUES (?, ?, ?, ?, ?) USING TIMESTAMP 0", 1, 0, 1, 0, 0);
 
         // overwrite pk and e with timestamp 1, but don't overwrite d
-        executeNet(version, "INSERT INTO %s (a, b, c, e) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 0, 1, 0);
+        executeNet("INSERT INTO %s (a, b, c, e) VALUES (?, ?, ?, ?) USING TIMESTAMP 1", 1, 0, 1, 0);
 
         // delete with timestamp 0 (which should only delete d)
-        executeNet(version, "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 = ?", 1, 1, 0),
+        executeNet("DELETE FROM %s USING TIMESTAMP 0 WHERE a = ? AND b = ?", 1, 0);
+        assertRows(executeView("SELECT a, b, c, d, e FROM %s WHERE c = ? and a = ? and b = ?", 1, 1, 0),
                    row(1, 0, 1, null, 0)
         );
 
-        executeNet(version, "UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? AND b = ?", 1, 1, 1);
-        executeNet(version, "UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? AND b = ?", 1, 1, 0);
-        assertRows(execute("SELECT a, b, c, d, e from mv WHERE c = ? and a = ? and b = ?", 1, 1, 0),
+        executeNet("UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? AND b = ?", 1, 1, 1);
+        executeNet("UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? AND b = ?", 1, 1, 0);
+        assertRows(executeView("SELECT a, b, c, d, e FROM %s WHERE c = ? and a = ? and b = ?", 1, 1, 0),
                    row(1, 0, 1, null, 0)
         );
 
-        executeNet(version, "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 = ?", 1, 1, 0),
+        executeNet("UPDATE %s USING TIMESTAMP 3 SET d = ? WHERE a = ? AND b = ?", 0, 1, 0);
+        assertRows(executeView("SELECT a, b, c, d, e FROM %s WHERE c = ? and a = ? and b = ?", 1, 1, 0),
                    row(1, 0, 1, 0, 0)
         );
     }
@@ -985,15 +848,14 @@ public class ViewFilteringTest extends CQLTester
                     "c int, " +
                     "val int)");
 
-        execute("USE " + keyspace());
-        executeNet(version, "USE " + keyspace());
-
-        createView("mv_rctstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL AND c = 1 PRIMARY KEY (k,c)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL AND c = 1 " +
+                   "PRIMARY KEY (k,c)");
 
         updateView("UPDATE %s SET c = ?, val = ? WHERE k = ?", 0, 0, 0);
         updateView("UPDATE %s SET val = ? WHERE k = ?", 1, 0);
         updateView("UPDATE %s SET c = ? WHERE k = ?", 1, 0);
-        assertRows(execute("SELECT c, k, val FROM mv_rctstest"), row(1, 0, 1));
+        assertRows(executeView("SELECT c, k, val FROM %s"), row(1, 0, 1));
 
         updateView("TRUNCATE %s");
 
@@ -1002,7 +864,7 @@ public class ViewFilteringTest extends CQLTester
         updateView("UPDATE %s USING TIMESTAMP 2 SET val = ? WHERE k = ?", 1, 0);
         updateView("UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE k = ?", 1, 0);
         updateView("UPDATE %s USING TIMESTAMP 3 SET val = ? WHERE k = ?", 2, 0);
-        assertRows(execute("SELECT c, k, val FROM mv_rctstest"), row(1, 0, 2));
+        assertRows(executeView("SELECT c, k, val FROM %s"), row(1, 0, 2));
     }
 
     @Test
@@ -1014,10 +876,9 @@ public class ViewFilteringTest extends CQLTester
                     "val text, " + "" +
                     "PRIMARY KEY(k, c))");
 
-        execute("USE " + keyspace());
-        executeNet(version, "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 AND val = 'baz' PRIMARY KEY (val,k,c)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL AND val = 'baz' " +
+                   "PRIMARY KEY (val,k,c)");
 
         for (int i = 0; i < 100; i++)
             updateView("INSERT into %s (k,c,val)VALUES(?,?,?)", 0, i % 2, "baz");
@@ -1025,21 +886,21 @@ public class ViewFilteringTest extends CQLTester
         Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush();
 
         Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv_tstest").size());
+        Assert.assertEquals(2, executeView("select * from %s").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));
+        assertRows(executeView("SELECT c from %s 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, 1);
         assertRows(execute("SELECT val from %s where k = 0 and c = 1"), 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"));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "baz"), row(0), row(1));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "bar"));
 
         //Latest TS
         updateView("UPDATE %s SET val = ? where k = ? AND c = ?", "bar", 0, 1);
         assertRows(execute("SELECT val from %s where k = 0 and c = 1"), row("bar"));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "bar"));
-        assertRows(execute("SELECT c from mv_tstest where k = 0 and val = ?", "baz"), row(0));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "bar"));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "baz"), row(0));
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewPKTest.java b/test/unit/org/apache/cassandra/cql3/ViewPKTest.java
index 06664cb..6ab27ef 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewPKTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewPKTest.java
@@ -24,9 +24,12 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.apache.cassandra.exceptions.InvalidRequestException;
+import org.apache.cassandra.exceptions.RequestValidationException;
+import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.ColumnMetadata;
 import org.apache.cassandra.schema.TableMetadata;
+import org.assertj.core.api.Assertions;
 
 import static org.junit.Assert.assertTrue;
 
@@ -44,33 +47,30 @@ public class ViewPKTest extends ViewAbstractTest
     {
         createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1, c1, val FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT k1, c1, val 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());
+        Assert.assertEquals(2, executeView("select * from %s").size());
 
         updateView("DELETE FROM %s WHERE k1 = 1");
 
         Assert.assertEquals(0, execute("select * from %s").size());
-        Assert.assertEquals(0, execute("select * from view1").size());
+        Assert.assertEquals(0, executeView("select * from %s").size());
     }
 
     @Test
-    public void createMvWithUnrestrictedPKParts() throws Throwable
+    public void createMvWithUnrestrictedPKParts()
     {
         createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT val, k1, c1 FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
-
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT val, k1, c1 FROM %s " +
+                   "WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL " +
+                   "PRIMARY KEY (val, k1, c1)");
     }
 
     @Test
@@ -78,25 +78,24 @@ public class ViewPKTest extends ViewAbstractTest
     {
         createTable("CREATE TABLE %s (k1 int, c1 int , val int, PRIMARY KEY (k1, c1))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("view1", "CREATE MATERIALIZED VIEW view1 AS SELECT k1, c1, val FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND val IS NOT NULL PRIMARY KEY (val, k1, c1)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT k1, c1, val 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());
+        Assert.assertEquals(2, executeView("select * from %s").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());
+        Assert.assertEquals(1, executeView("select * from %s").size());
     }
 
     @Test
-    public void testPrimaryKeyIsNotNull() throws Throwable
+    public void testPrimaryKeyIsNotNull()
     {
         createTable("CREATE TABLE %s (" +
                     "k int, " +
@@ -104,27 +103,32 @@ public class ViewPKTest extends ViewAbstractTest
                     "bigintval bigint, " +
                     "PRIMARY KEY((k, asciival)))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
         // Must include "IS NOT NULL" for primary keys
         try
         {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s");
             Assert.fail("Should fail if no primary key is filtered as NOT NULL");
         }
         catch (Exception e)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(SyntaxException.class);
+            Assertions.assertThat(cause.getMessage()).contains("mismatched input");
         }
 
         // 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)");
+            createView("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)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(InvalidRequestException.class);
+            Assertions.assertThat(cause.getMessage()).contains("Primary key columns k must be restricted");
         }
 
         dropTable("DROP TABLE %s");
@@ -136,21 +140,29 @@ public class ViewPKTest extends ViewAbstractTest
                     "PRIMARY KEY(k, asciival))");
         try
         {
-            createView("mv_test", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s");
             Assert.fail("Should fail if no primary key is filtered as NOT NULL");
         }
         catch (Exception e)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(SyntaxException.class);
+            Assertions.assertThat(cause.getMessage()).contains("mismatched input");
         }
 
         // Must still include both even 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)");
+            createView("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)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(InvalidRequestException.class);
+            Assertions.assertThat(cause.getMessage()).contains("Primary key columns k must be restricted");
         }
     }
 
@@ -165,39 +177,36 @@ public class ViewPKTest extends ViewAbstractTest
 
         TableMetadata metadata = currentTableMetadata();
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
         for (ColumnMetadata def : new HashSet<>(metadata.columns()))
         {
+            String asciival = def.name.toString().equals("asciival") ? "" : "AND asciival IS NOT NULL ";
             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 ("
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + asciival + "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)
+            catch (Exception 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 ("
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + asciival + " 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)
+            catch (Exception e)
             {
                 if (!def.type.isMultiCell() && !def.isPartitionKey())
                     Assert.fail("MV creation failed on " + def);
@@ -205,41 +214,42 @@ public class ViewPKTest extends ViewAbstractTest
 
             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)";
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + asciival + "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)
+            catch (Exception 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)";
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + asciival + "PRIMARY KEY ((" + def.name + ", k), asciival)";
                 createView("mv3_" + def.name, query);
 
                 Assert.fail("Should fail on duplicate name");
             }
             catch (Exception e)
             {
+                Assertions.assertThat(e.getCause()).isInstanceOf(RequestValidationException.class);
             }
 
             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)";
+                String query = "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE " + def.name + " IS NOT NULL AND k IS NOT NULL "
+                               + asciival + "PRIMARY KEY ((" + def.name + ", k), nonexistentcolumn)";
                 createView("mv4_" + def.name, query);
                 Assert.fail("Should fail with unknown base column");
             }
-            catch (InvalidQueryException e)
+            catch (Exception e)
             {
+                Assertions.assertThat(e.getCause()).isInstanceOf(RequestValidationException.class);
             }
         }
 
@@ -288,24 +298,24 @@ public class ViewPKTest extends ViewAbstractTest
 
         executeNet("USE " + keyspace());
 
-        createView("mv1", "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 (a, b, c) WITH CLUSTERING ORDER BY (b DESC, c ASC)");
-        createView("mv2", "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 (a, c, b) WITH CLUSTERING ORDER BY (c ASC, b ASC)");
-        createView("mv3", "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 (a, b, c)");
-        createView("mv4", "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 (a, c, b) WITH CLUSTERING ORDER BY (c DESC, b ASC)");
+        String mv1 = createView("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 (a, b, c) WITH CLUSTERING ORDER BY (b DESC, c ASC)");
+        String mv2 = createView("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 (a, c, b) WITH CLUSTERING ORDER BY (c ASC, b ASC)");
+        String mv3 = createView("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 (a, b, c)");
+        String mv4 = createView("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 (a, c, b) WITH CLUSTERING ORDER BY (c DESC, b ASC)");
 
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 1, 1, 1);
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 1, 2, 2, 2);
 
-        ResultSet mvRows = executeNet("SELECT b FROM mv1");
+        ResultSet mvRows = executeNet("SELECT b FROM " + mv1);
         assertRowsNet(mvRows, row(2), row(1));
 
-        mvRows = executeNet("SELECT c FROM mv2");
+        mvRows = executeNet("SELECT c FROM " + mv2);
         assertRowsNet(mvRows, row(1), row(2));
 
-        mvRows = executeNet("SELECT b FROM mv3");
+        mvRows = executeNet("SELECT b FROM " + mv3);
         assertRowsNet(mvRows, row(1), row(2));
 
-        mvRows = executeNet("SELECT c FROM mv4");
+        mvRows = executeNet("SELECT c FROM " + mv4);
         assertRowsNet(mvRows, row(2), row(1));
     }
 
@@ -320,11 +330,13 @@ public class ViewPKTest extends ViewAbstractTest
         executeNet("USE " + keyspace());
 
         // Cannot use SELECT *, as those are always handled by the includeAll shortcut in View.updateAffectsView
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %s " +
+                   "WHERE a IS NOT NULL AND b IS NOT NULL " +
+                   "PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 1);
 
-        ResultSet mvRows = executeNet("SELECT a, b FROM mv1");
+        ResultSet mvRows = executeViewNet("SELECT a, b FROM %s");
         assertRowsNet(mvRows, row(1, 1));
     }
 
@@ -339,11 +351,11 @@ public class ViewPKTest extends ViewAbstractTest
         executeNet("USE " + keyspace());
 
         // Cannot use SELECT *, as those are always handled by the includeAll shortcut in View.updateAffectsView
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 1);
 
-        ResultSet mvRows = executeNet("SELECT a, b FROM mv1");
+        ResultSet mvRows = executeViewNet("SELECT a, b FROM %s");
         assertRowsNet(mvRows, row(1, 1));
     }
 
@@ -358,18 +370,20 @@ public class ViewPKTest extends ViewAbstractTest
                     "PRIMARY KEY (a, b))");
 
         executeNet("USE " + keyspace());
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND d IS NOT NULL PRIMARY KEY (a, d, b)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE a IS NOT NULL AND b IS NOT NULL AND d IS NOT NULL " +
+                   "PRIMARY KEY (a, d, b)");
 
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-        ResultSet mvRows = executeNet("SELECT a, d, b, c FROM mv1");
+        ResultSet mvRows = executeViewNet("SELECT a, d, b, c FROM %s");
         assertRowsNet(mvRows, row(0, 0, 0, 0));
 
         updateView("DELETE c FROM %s WHERE a = ? AND b = ?", 0, 0);
-        mvRows = executeNet("SELECT a, d, b, c FROM mv1");
+        mvRows = executeViewNet("SELECT a, d, b, c FROM %s");
         assertRowsNet(mvRows, row(0, 0, 0, null));
 
         updateView("DELETE d FROM %s WHERE a = ? AND b = ?", 0, 0);
-        mvRows = executeNet("SELECT a, d, b FROM mv1");
+        mvRows = executeViewNet("SELECT a, d, b FROM %s");
         assertTrue(mvRows.isExhausted());
     }
 
@@ -384,23 +398,25 @@ public class ViewPKTest extends ViewAbstractTest
                     "PRIMARY KEY (a, b))");
 
         executeNet("USE " + keyspace());
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND d IS NOT NULL PRIMARY KEY (d, a, b)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE a IS NOT NULL AND b IS NOT NULL AND d IS NOT NULL " +
+                   "PRIMARY KEY (d, a, b)");
 
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?)", 0, 0, 0, 0);
-        ResultSet mvRows = executeNet("SELECT a, d, b, c FROM mv1");
+        ResultSet mvRows = executeViewNet("SELECT a, d, b, c FROM %s");
         assertRowsNet(mvRows, row(0, 0, 0, 0));
 
         updateView("DELETE c FROM %s WHERE a = ? AND b = ?", 0, 0);
-        mvRows = executeNet("SELECT a, d, b, c FROM mv1");
+        mvRows = executeViewNet("SELECT a, d, b, c FROM %s");
         assertRowsNet(mvRows, row(0, 0, 0, null));
 
         updateView("DELETE d FROM %s WHERE a = ? AND b = ?", 0, 0);
-        mvRows = executeNet("SELECT a, d, b FROM mv1");
+        mvRows = executeViewNet("SELECT a, d, b FROM %s");
         assertTrue(mvRows.isExhausted());
     }
 
     @Test
-    public void testMultipleNonPrimaryKeysInView() throws Throwable
+    public void testMultipleNonPrimaryKeysInView()
     {
         createTable("CREATE TABLE %s (" +
                     "a int," +
@@ -412,22 +428,27 @@ public class ViewPKTest extends ViewAbstractTest
 
         try
         {
-            createView("mv_de", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d IS NOT NULL AND e IS NOT NULL PRIMARY KEY ((d, a), b, e, c)");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d IS NOT NULL AND e IS NOT NULL PRIMARY KEY ((d, a), b, e, c)");
             Assert.fail("Should have rejected a query including multiple non-primary key base columns");
         }
         catch (Exception e)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(InvalidRequestException.class);
+            Assertions.assertThat(cause.getMessage()).contains("Cannot include more than one non-primary key column");
         }
 
         try
         {
-            createView("mv_de", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d IS NOT NULL AND e IS NOT NULL PRIMARY KEY ((a, b), c, d, e)");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL AND c IS NOT NULL AND d IS NOT NULL AND e IS NOT NULL PRIMARY KEY ((a, b), c, d, e)");
             Assert.fail("Should have rejected a query including multiple non-primary key base columns");
         }
         catch (Exception e)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(InvalidRequestException.class);
+            Assertions.assertThat(cause.getMessage()).contains("Cannot include more than one non-primary key column");
         }
-
     }
 
     @Test
@@ -437,10 +458,9 @@ public class ViewPKTest extends ViewAbstractTest
 
         executeNet("USE " + keyspace());
 
-        createView("mv",
-                   "CREATE MATERIALIZED VIEW %s AS" +
+        createView("CREATE MATERIALIZED VIEW %s AS" +
                    "  SELECT id1, v1, id2, v2" +
-                   "  FROM %%s" +
+                   "  FROM %s" +
                    "  WHERE id1 IS NOT NULL AND v1 IS NOT NULL AND id2 IS NOT NULL" +
                    "  PRIMARY KEY (id1, v1, id2)" +
                    "  WITH CLUSTERING ORDER BY (v1 DESC, id2 ASC)");
@@ -448,14 +468,14 @@ public class ViewPKTest extends ViewAbstractTest
         execute("INSERT INTO %s (id1, id2, v1, v2) VALUES (?, ?, ?, ?)", 0, 1, "foo", "bar");
 
         assertRowsNet(executeNet("SELECT * FROM %s"), row(0, 1, "foo", "bar"));
-        assertRowsNet(executeNet("SELECT * FROM mv"), row(0, "foo", 1, "bar"));
+        assertRowsNet(executeViewNet("SELECT * FROM %s"), row(0, "foo", 1, "bar"));
 
         executeNet("UPDATE %s SET v1=? WHERE id1=? AND id2=?", null, 0, 1);
         assertRowsNet(executeNet("SELECT * FROM %s"), row(0, 1, null, "bar"));
-        assertRowsNet(executeNet("SELECT * FROM mv"));
+        assertRowsNet(executeViewNet("SELECT * FROM %s"));
 
         executeNet("UPDATE %s SET v2=? WHERE id1=? AND id2=?", "rab", 0, 1);
         assertRowsNet(executeNet("SELECT * FROM %s"), row(0, 1, null, "rab"));
-        assertRowsNet(executeNet("SELECT * FROM mv"));
+        assertRowsNet(executeViewNet("SELECT * FROM %s"));
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewRangesTest.java b/test/unit/org/apache/cassandra/cql3/ViewRangesTest.java
index ce67a9b..bb2ea67 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewRangesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewRangesTest.java
@@ -22,7 +22,6 @@ import org.junit.Assert;
 import org.junit.Test;
 
 import org.apache.cassandra.db.Keyspace;
-import org.apache.cassandra.db.SystemKeyspace;
 
 /*
  * This test class was too large and used to timeout CASSANDRA-16777. We're splitting it into:
@@ -49,11 +48,9 @@ public class ViewRangesTest extends ViewAbstractTest
     {
         createTable("CREATE TABLE %s (k1 int, c1 int, c2 int, v1 int, v2 int, PRIMARY KEY (k1, c1, c2))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("view1",
-                   "CREATE MATERIALIZED VIEW view1 AS SELECT * FROM %%s WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL PRIMARY KEY (k1, c2, c1)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE k1 IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL " +
+                   "PRIMARY KEY (k1, c2, c1)");
 
         updateView("DELETE FROM %s USING TIMESTAMP 10 WHERE k1 = 1 and c1=1");
 
@@ -75,7 +72,7 @@ public class ViewRangesTest extends ViewAbstractTest
                                 row(1, 0, 0, 0, 0),
                                 row(1, 0, 1, 0, 1),
                                 row(1, 2, 0, 2, 0));
-        assertRowsIgnoringOrder(execute("select k1,c1,c2,v1,v2 from view1"),
+        assertRowsIgnoringOrder(executeView("select k1,c1,c2,v1,v2 from %s"),
                                 row(1, 0, 0, 0, 0),
                                 row(1, 0, 1, 0, 1),
                                 row(1, 2, 0, 2, 0));
@@ -93,39 +90,37 @@ public class ViewRangesTest extends ViewAbstractTest
                     "PRIMARY KEY((k, asciival), bigintval, textval1)" +
                     ")");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+        createView("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");
+            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());
+        Assert.assertEquals(100, executeView("select * from %s").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(10);
+        createView("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)");
 
-        Assert.assertEquals(100, execute("select * from mv_test2").size());
+        Assert.assertEquals(100, executeView("select * from %s").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)");
+        createView("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(10);
-
-        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());
+        Assert.assertEquals(100, executeView("select * from %s").size());
+        Assert.assertEquals(100, executeView("select asciival from %s 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());
+        Assert.assertEquals(50, executeView("select asciival from %s where textval2 = ? and k = ?", "baz", 0).size());
     }
 
 
@@ -140,10 +135,9 @@ public class ViewRangesTest extends ViewAbstractTest
                     "PRIMARY KEY((k, asciival), bigintval)" +
                     ")");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+        createView("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);
@@ -153,13 +147,13 @@ public class ViewRangesTest extends ViewAbstractTest
 
 
         Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv").size());
+        Assert.assertEquals(2, executeView("select * from %s").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());
+        Assert.assertEquals(1, executeView("select * from %s").size());
     }
 
     @Test
@@ -173,10 +167,9 @@ public class ViewRangesTest extends ViewAbstractTest
                     "PRIMARY KEY((k, asciival), bigintval)" +
                     ")");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+        createView("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);
@@ -186,12 +179,12 @@ public class ViewRangesTest extends ViewAbstractTest
 
 
         Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv").size());
+        Assert.assertEquals(2, executeView("select * from %s").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());
+        Assert.assertEquals(0, executeView("select * from %s").size());
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
index a1cc34a..03dc0c5 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewSchemaTest.java
@@ -22,122 +22,55 @@ import java.math.BigDecimal;
 import java.math.BigInteger;
 import java.net.InetAddress;
 import java.text.SimpleDateFormat;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Date;
 import java.util.HashSet;
-import java.util.List;
 import java.util.UUID;
 
 import org.junit.Assert;
+import org.junit.Test;
 
-import com.datastax.driver.core.exceptions.OperationTimedOutException;
-import org.apache.cassandra.concurrent.SEPExecutor;
-import org.apache.cassandra.concurrent.Stage;
+import com.datastax.driver.core.exceptions.InvalidQueryException;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SchemaCQLHelper;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.schema.ColumnMetadata;
-import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.schema.Schema;
-import org.apache.cassandra.exceptions.InvalidRequestException;
-import org.apache.cassandra.db.SystemKeyspace;
+import org.apache.cassandra.schema.TableMetadata;
 import org.apache.cassandra.serializers.SimpleDateSerializer;
 import org.apache.cassandra.serializers.TimeSerializer;
-import org.apache.cassandra.transport.ProtocolVersion;
 import org.apache.cassandra.utils.ByteBufferUtil;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import com.datastax.driver.core.exceptions.InvalidQueryException;
+import org.assertj.core.api.Assertions;
 
 import static org.junit.Assert.assertTrue;
 
-
-public class ViewSchemaTest extends CQLTester
+public class ViewSchemaTest extends ViewAbstractTest
 {
-    ProtocolVersion protocolVersion = ProtocolVersion.V4;
-    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
-    {
-        try
-        {
-            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);
-        }
-        catch (OperationTimedOutException ex)
-        {
-            // ... except for timeout, when we actually do not know whether the view was created or not
-            views.add(name);
-            throw ex;
-        }
-    }
-
-    private void updateView(String query, Object... params) throws Throwable
-    {
-        executeNet(protocolVersion, query, params);
-        while (!(((SEPExecutor) Stage.VIEW_MUTATION.executor()).getPendingTaskCount() == 0
-                 && ((SEPExecutor) Stage.VIEW_MUTATION.executor()).getActiveTaskCount() == 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\")");
+        String mv1 = createView("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);
+        String mv2 = createView("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\")");
 
-        for (String mvname : Arrays.asList("mv_test", "mv_test2"))
+        for (String mvname : Arrays.asList(mv1, mv2))
         {
             assertRows(execute("SELECT \"theKey\", \"theClustering\", \"theValue\" FROM " + mvname),
-               row(0, 0, 0)
-            );
+                       row(0, 0, 0));
         }
 
-        executeNet(protocolVersion, "ALTER TABLE %s RENAME \"theClustering\" TO \"Col\"");
+        executeNet("ALTER TABLE %s RENAME \"theClustering\" TO \"Col\"");
 
-        for (String mvname : Arrays.asList("mv_test", "mv_test2"))
+        for (String mvname : Arrays.asList(mv1, mv2))
         {
             assertRows(execute("SELECT \"theKey\", \"Col\", \"theValue\" FROM " + mvname),
                        row(0, 0, 0)
@@ -154,54 +87,56 @@ public class ViewSchemaTest extends CQLTester
                     "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)");
+        createView("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);
+            updateView("INSERT INTO " + currentView() + "(k,asciival,bigintval) VALUES(?,?,?)", 1, "foo", 2L);
             Assert.fail("Shouldn't be able to modify a MV directly");
         }
-        catch (Exception e)
+        catch (InvalidQueryException e)
         {
+            Assertions.assertThat(e.getMessage()).contains("Cannot directly modify a materialized view");
         }
 
         try
         {
-            executeNet(protocolVersion, "ALTER TABLE mv1_test ADD foo text");
+            executeViewNet("ALTER TABLE %s ADD foo text");
             Assert.fail("Should not be able to use alter table with MV");
         }
-        catch (Exception e)
+        catch (InvalidQueryException e)
         {
+            Assertions.assertThat(e.getMessage()).contains("Cannot use ALTER TABLE on a materialized view");
         }
 
         try
         {
-            executeNet(protocolVersion, "ALTER TABLE mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
+            executeViewNet("ALTER TABLE %s WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
             Assert.fail("Should not be able to use alter table with MV");
         }
-        catch (Exception e)
+        catch (InvalidQueryException e)
         {
+            Assertions.assertThat(e.getMessage()).contains("Cannot use ALTER TABLE on a materialized view");
         }
 
-        executeNet(protocolVersion, "ALTER MATERIALIZED VIEW mv1_test WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
+        executeViewNet("ALTER MATERIALIZED VIEW %s WITH compaction = { 'class' : 'LeveledCompactionStrategy' }");
 
         //Test alter add
-        executeNet(protocolVersion, "ALTER TABLE %s ADD foo text");
-        TableMetadata metadata = Schema.instance.getTableMetadata(keyspace(), "mv1_test");
+        executeNet("ALTER TABLE %s ADD foo text");
+        TableMetadata metadata = Schema.instance.getTableMetadata(keyspace(), currentView());
         Assert.assertNotNull(metadata.getColumn(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");
+        executeNet("ALTER TABLE %s RENAME asciival TO bar");
 
         assertRows(execute("SELECT bar from %s"), row("foo"));
-        metadata = Schema.instance.getTableMetadata(keyspace(), "mv1_test");
+        metadata = Schema.instance.getTableMetadata(keyspace(), currentView());
         Assert.assertNotNull(metadata.getColumn(ByteBufferUtil.bytes("bar")));
     }
 
@@ -209,36 +144,33 @@ public class ViewSchemaTest extends CQLTester
     @Test
     public void testTwoTablesOneView() throws Throwable
     {
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
         createTable("CREATE TABLE " + keyspace() + ".dummy_table (" +
-                "j int, " +
-                "intval int, " +
-                "PRIMARY KEY (j))");
+                    "j int, " +
+                    "intval int, " +
+                    "PRIMARY KEY (j))");
 
         createTable("CREATE TABLE " + keyspace() + ".real_base (" +
-                "k int, " +
-                "intval int, " +
-                "PRIMARY KEY (k))");
+                    "k int, " +
+                    "intval int, " +
+                    "PRIMARY KEY (k))");
 
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM " + keyspace() + ".real_base WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
-        createView("mv2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM " + keyspace() + ".dummy_table WHERE j IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, j)");
+        String mv = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM " + keyspace() + ".real_base WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM " + keyspace() + ".dummy_table WHERE j IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, j)");
 
         updateView("INSERT INTO " + keyspace() + ".real_base (k, intval) VALUES (?, ?)", 0, 0);
         assertRows(execute("SELECT k, intval FROM " + keyspace() + ".real_base WHERE k = ?", 0), row(0, 0));
-        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 0), row(0, 0));
+        assertRows(execute("SELECT k, intval from " + mv + " WHERE intval = ?", 0), row(0, 0));
 
         updateView("INSERT INTO " + keyspace() + ".real_base (k, intval) VALUES (?, ?)", 0, 1);
         assertRows(execute("SELECT k, intval FROM " + keyspace() + ".real_base WHERE k = ?", 0), row(0, 1));
-        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1));
+        assertRows(execute("SELECT k, intval from " + mv + " WHERE intval = ?", 1), row(0, 1));
 
         assertRows(execute("SELECT k, intval FROM " + keyspace() + ".real_base WHERE k = ?", 0), row(0, 1));
-        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1));
+        assertRows(execute("SELECT k, intval from " + mv + " WHERE intval = ?", 1), row(0, 1));
 
-        updateView("INSERT INTO " + keyspace() +".dummy_table (j, intval) VALUES(?, ?)", 0, 1);
+        updateView("INSERT INTO " + keyspace() + ".dummy_table (j, intval) VALUES(?, ?)", 0, 1);
         assertRows(execute("SELECT j, intval FROM " + keyspace() + ".dummy_table WHERE j = ?", 0), row(0, 1));
-        assertRows(execute("SELECT k, intval from mv WHERE intval = ?", 1), row(0, 1));
+        assertRows(execute("SELECT k, intval from " + mv + " WHERE intval = ?", 1), row(0, 1));
     }
 
     @Test
@@ -249,23 +181,23 @@ public class ViewSchemaTest extends CQLTester
                     "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)");
+        String view = createView("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));
+        assertRows(executeView("SELECT k, intval from %s WHERE intval = ?", 0), row(0, 0));
 
-        executeNet(protocolVersion, "DROP MATERIALIZED VIEW mv");
-        views.remove("mv");
+        executeNet("DROP MATERIALIZED VIEW " + view);
 
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND intval IS NOT NULL PRIMARY KEY (intval, k)");
+        createView(view, "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));
+        assertRows(executeView("SELECT k, intval from %s WHERE intval = ?", 1), row(0, 1));
     }
 
     @Test
@@ -303,14 +235,11 @@ public class ViewSchemaTest extends CQLTester
 
         TableMetadata metadata = currentTableMetadata();
 
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
         for (ColumnMetadata def : new HashSet<>(metadata.columns()))
         {
             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)");
+                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);
@@ -318,7 +247,7 @@ public class ViewSchemaTest extends CQLTester
                 if (def.isPartitionKey())
                     Assert.fail("MV on partition key should fail " + def);
             }
-            catch (InvalidQueryException e)
+            catch (Exception e)
             {
                 if (!def.type.isMultiCell() && !def.isPartitionKey())
                     Assert.fail("MV creation failed on " + def);
@@ -678,20 +607,19 @@ public class ViewSchemaTest extends CQLTester
     public void testDropTableWithMV() throws Throwable
     {
         createTable("CREATE TABLE %s (" +
-                "a int," +
-                "b int," +
-                "c int," +
-                "d int," +
-                "PRIMARY KEY (a, b, c))");
+                    "a int," +
+                    "b int," +
+                    "c int," +
+                    "d int," +
+                    "PRIMARY KEY (a, b, c))");
 
-        executeNet(protocolVersion, "USE " + keyspace());
+        executeNet("USE " + keyspace());
 
-        createView(keyspace() + ".mv1",
-                   "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 (a, b, c)");
+        String mv = createView("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 (a, b, c)");
 
         try
         {
-            executeNet(protocolVersion, "DROP TABLE " + keyspace() + ".mv1");
+            executeNet("DROP TABLE " + keyspace() + '.' + mv);
             Assert.fail();
         }
         catch (InvalidQueryException e)
@@ -707,12 +635,12 @@ public class ViewSchemaTest extends CQLTester
         // format changes.
         createTable("CREATE TABLE %s ( a int, b int, c int, d int, PRIMARY KEY (a, b, c))");
 
-        executeNet(protocolVersion, "USE " + keyspace());
+        executeNet("USE " + keyspace());
 
         assertInvalidMessage("Non-primary key columns can only be restricted with 'IS NOT NULL'",
                              "CREATE MATERIALIZED VIEW " + keyspace() + ".mv AS SELECT * FROM %s "
-                                     + "WHERE b IS NOT NULL AND c IS NOT NULL AND a IS NOT NULL "
-                                     + "AND d = 1 PRIMARY KEY (c, b, a)");
+                             + "WHERE b IS NOT NULL AND c IS NOT NULL AND a IS NOT NULL "
+                             + "AND d = 1 PRIMARY KEY (c, b, a)");
     }
 
     @Test
@@ -720,9 +648,6 @@ public class ViewSchemaTest extends CQLTester
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, PRIMARY KEY(a))");
 
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
         execute("INSERT into %s (a,b,c,d) VALUES (?,?,?,?)", 1, 2, 3, 4);
 
         assertInvalidThrowMessage("Cannot use token relation when defining a materialized view", InvalidRequestException.class,
@@ -740,11 +665,8 @@ public class ViewSchemaTest extends CQLTester
                     "v int, " +
                     "PRIMARY KEY (pk, c1, c2, c3))");
 
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 DESC, c1 ASC, c3 ASC)");
-        createView("mv2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 ASC, c1 DESC, c3 DESC)");
+        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 DESC, c1 ASC, c3 ASC)");
+        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 ASC, c1 DESC, c3 DESC)");
 
         updateView("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
         updateView("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 1, 1);
@@ -767,7 +689,7 @@ public class ViewSchemaTest extends CQLTester
                    row(0, 1, 2, 1, 7),
                    row(0, 2, 1, 1, 8));
 
-        assertRows(execute("SELECT * FROM mv1 WHERE pk = ?", 0),
+        assertRows(execute("SELECT * FROM " + mv1 + " WHERE pk = ?", 0),
                    row(0, 2, 1, 1, 7),
                    row(0, 1, 0, 0, 3),
                    row(0, 1, 0, 1, 4),
@@ -778,7 +700,7 @@ public class ViewSchemaTest extends CQLTester
                    row(0, 0, 0, 1, 1),
                    row(0, 0, 0, 2, 2));
 
-        assertRows(execute("SELECT * FROM mv2 WHERE pk = ?", 0),
+        assertRows(execute("SELECT * FROM " + mv2 + " WHERE pk = ?", 0),
                    row(0, 0, 0, 2, 2),
                    row(0, 0, 0, 1, 1),
                    row(0, 0, 0, 0, 0),
@@ -801,12 +723,9 @@ public class ViewSchemaTest extends CQLTester
                     "v int, " +
                     "PRIMARY KEY (pk, c1, c2, c3)) WITH CLUSTERING ORDER BY (c1 DESC, c2 ASC, c3 DESC)");
 
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3)");
-        createView("mv2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 DESC, c1 ASC, c3 ASC)");
-        createView("mv3", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 ASC, c1 DESC, c3 DESC)");
+        String mv1 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3)");
+        String mv2 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 DESC, c1 ASC, c3 ASC)");
+        String mv3 = createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE pk IS NOT NULL AND c1 IS NOT NULL AND c2 IS NOT NULL and c3 IS NOT NULL PRIMARY KEY (pk, c2, c1, c3) WITH CLUSTERING ORDER BY (c2 ASC, c1 DESC, c3 DESC)");
 
         updateView("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 0, 0);
         updateView("INSERT INTO %s (pk, c1, c2, c3, v) VALUES (?, ?, ?, ?, ?)", 0, 0, 0, 1, 1);
@@ -827,9 +746,9 @@ public class ViewSchemaTest extends CQLTester
                    row(0, 0, 0, 0, 0),
                    row(0, 0, 1, 2, 5),
                    row(0, 0, 1, 1, 4),
-                  row(0, 0, 1, 0, 3));
+                   row(0, 0, 1, 0, 3));
 
-        assertRows(execute("SELECT * FROM mv1 WHERE pk = ?", 0),
+        assertRows(execute("SELECT * FROM " + mv1 + " WHERE pk = ?", 0),
                    row(0, 0, 0, 2, 2),
                    row(0, 0, 0, 1, 1),
                    row(0, 0, 0, 0, 0),
@@ -840,7 +759,7 @@ public class ViewSchemaTest extends CQLTester
                    row(0, 1, 0, 0, 3),
                    row(0, 2, 1, 1, 7));
 
-        assertRows(execute("SELECT * FROM mv2 WHERE pk = ?", 0),
+        assertRows(execute("SELECT * FROM " + mv2 + " WHERE pk = ?", 0),
                    row(0, 2, 1, 1, 7),
                    row(0, 1, 0, 0, 3),
                    row(0, 1, 0, 1, 4),
@@ -851,7 +770,7 @@ public class ViewSchemaTest extends CQLTester
                    row(0, 0, 0, 1, 1),
                    row(0, 0, 0, 2, 2));
 
-        assertRows(execute("SELECT * FROM mv3 WHERE pk = ?", 0),
+        assertRows(execute("SELECT * FROM " + mv3 + " WHERE pk = ?", 0),
                    row(0, 0, 0, 2, 2),
                    row(0, 0, 0, 1, 1),
                    row(0, 0, 0, 0, 0),
@@ -864,7 +783,7 @@ public class ViewSchemaTest extends CQLTester
     }
 
     @Test
-    public void testViewMetadataCQLNotIncludeAllColumn() throws Throwable
+    public void testViewMetadataCQLNotIncludeAllColumn()
     {
         String createBase = "CREATE TABLE IF NOT EXISTS %s (" +
                             "pk1 int," +
@@ -877,7 +796,7 @@ public class ViewSchemaTest extends CQLTester
                             "PRIMARY KEY ((pk1, pk2), ck1, ck2)) WITH " +
                             "CLUSTERING ORDER BY (ck1 ASC, ck2 ASC);";
 
-        String createView = "CREATE MATERIALIZED VIEW IF NOT EXISTS %s AS SELECT pk1, pk2, ck1, ck2, reg1, reg2 FROM %%s "
+        String createView = "CREATE MATERIALIZED VIEW IF NOT EXISTS %s AS SELECT pk1, pk2, ck1, ck2, reg1, reg2 FROM %s "
                             + "WHERE pk2 IS NOT NULL AND pk1 IS NOT NULL AND ck2 IS NOT NULL AND ck1 IS NOT NULL PRIMARY KEY((pk2, pk1), ck2, ck1)";
 
         String expectedViewSnapshot = "CREATE MATERIALIZED VIEW IF NOT EXISTS %s.%s AS\n" +
@@ -894,7 +813,7 @@ public class ViewSchemaTest extends CQLTester
     }
 
     @Test
-    public void testViewMetadataCQLIncludeAllColumn() throws Throwable
+    public void testViewMetadataCQLIncludeAllColumn()
     {
         String createBase = "CREATE TABLE IF NOT EXISTS %s (" +
                             "pk1 int," +
@@ -907,7 +826,7 @@ public class ViewSchemaTest extends CQLTester
                             "PRIMARY KEY ((pk1, pk2), ck1, ck2)) WITH " +
                             "CLUSTERING ORDER BY (ck1 ASC, ck2 DESC);";
 
-        String createView = "CREATE MATERIALIZED VIEW IF NOT EXISTS %s AS SELECT * FROM %%s "
+        String createView = "CREATE MATERIALIZED VIEW IF NOT EXISTS %s AS SELECT * FROM %s "
                             + "WHERE pk2 IS NOT NULL AND pk1 IS NOT NULL AND ck2 IS NOT NULL AND ck1 IS NOT NULL PRIMARY KEY((pk2, pk1), ck2, ck1)";
 
         String expectedViewSnapshot = "CREATE MATERIALIZED VIEW IF NOT EXISTS %s.%s AS\n" +
@@ -923,18 +842,14 @@ public class ViewSchemaTest extends CQLTester
                             expectedViewSnapshot);
     }
 
-    private void testViewMetadataCQL(String createBase, String createView, String viewSnapshotSchema) throws Throwable
+    private void testViewMetadataCQL(String createBase, String createView, String viewSnapshotSchema)
     {
-        execute("USE " + keyspace());
-        executeNet(protocolVersion, "USE " + keyspace());
-
         String base = createTable(createBase);
 
-        String view = "mv";
-        createView(view, createView);
+        String view = createView(createView);
 
         ColumnFamilyStore mv = Keyspace.open(keyspace()).getColumnFamilyStore(view);
-        
+
         assertTrue(SchemaCQLHelper.getTableMetadataAsCQL(mv.metadata(), true, true, true)
                                   .startsWith(String.format(viewSnapshotSchema,
                                                             keyspace(),
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTest.java b/test/unit/org/apache/cassandra/cql3/ViewTest.java
index a1af740..1230eec 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTest.java
@@ -21,28 +21,25 @@ package org.apache.cassandra.cql3;
 import java.util.List;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.google.common.util.concurrent.Uninterruptibles;
 
 import org.junit.Assert;
 import org.junit.Test;
 import org.junit.runner.RunWith;
 
 import com.datastax.driver.core.ResultSet;
-import com.datastax.driver.core.exceptions.InvalidQueryException;
 import org.apache.cassandra.config.DatabaseDescriptor;
 import org.apache.cassandra.db.ColumnFamilyStore;
 import org.apache.cassandra.db.Keyspace;
 import org.apache.cassandra.db.SystemKeyspace;
 import org.apache.cassandra.db.compaction.CompactionManager;
 import org.apache.cassandra.db.view.View;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.exceptions.SyntaxException;
 import org.apache.cassandra.schema.SchemaConstants;
 import org.apache.cassandra.schema.SchemaKeyspaceTables;
 import org.apache.cassandra.service.ClientWarn;
 import org.apache.cassandra.utils.FBUtilities;
+import org.assertj.core.api.Assertions;
 import org.awaitility.Awaitility;
 import org.jboss.byteman.contrib.bmunit.BMRule;
 import org.jboss.byteman.contrib.bmunit.BMRules;
@@ -67,7 +64,6 @@ public class ViewTest extends ViewAbstractTest
     /** Latch used by {@link #testTruncateWhileBuilding()} Byteman injections. */
     @SuppressWarnings("unused")
     private static final CountDownLatch blockViewBuild = new CountDownLatch(1);
-    private static final AtomicInteger viewNameSeqNumber = new AtomicInteger();
 
     @Test
     public void testNonExistingOnes() throws Throwable
@@ -89,37 +85,37 @@ public class ViewTest extends ViewAbstractTest
                     "val text, " +
                     "PRIMARY KEY(k,c))");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+            createView("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("Use of static column in a MV primary key should fail");
         }
-        catch (InvalidQueryException e)
+        catch (Exception e)
         {
+            Assert.assertTrue(e.getCause() instanceof InvalidRequestException);
         }
 
         try
         {
-            createView("mv_static", "CREATE MATERIALIZED VIEW %%s AS SELECT val, sval FROM %s WHERE val IS NOT NULL AND  k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val, k, c)");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT val, sval FROM %s WHERE val IS NOT NULL AND  k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val, k, c)");
             Assert.fail("Explicit select of static column in MV should fail");
         }
-        catch (InvalidQueryException e)
+        catch (Exception e)
         {
+            Assert.assertTrue(e.getCause() instanceof InvalidRequestException);
         }
 
         try
         {
-            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)");
+            createView("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)");
             Assert.fail("Implicit select of static column in MV should fail");
         }
-        catch (InvalidQueryException e)
+        catch (Exception e)
         {
+            Assert.assertTrue(e.getCause() instanceof InvalidRequestException);
         }
 
-        createView("mv_static", "CREATE MATERIALIZED VIEW %s AS SELECT val,k,c FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT val,k,c 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");
@@ -128,9 +124,9 @@ public class ViewTest extends ViewAbstractTest
 
         assertRows(execute("SELECT sval from %s"), row("bar99"), row("bar99"));
 
-        Assert.assertEquals(2, execute("select * from mv_static").size());
+        Assert.assertEquals(2, executeView("select * from %s").size());
 
-        assertInvalid("SELECT sval from mv_static");
+        assertInvalid("SELECT sval from " + currentView());
     }
 
 
@@ -143,10 +139,7 @@ public class ViewTest extends ViewAbstractTest
                     "val text, " +
                     "PRIMARY KEY(k,c))");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+        createView("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");
@@ -154,62 +147,58 @@ public class ViewTest extends ViewAbstractTest
         Keyspace.open(keyspace()).getColumnFamilyStore(currentTable()).forceBlockingFlush();
 
         Assert.assertEquals(2, execute("select * from %s").size());
-        Assert.assertEquals(2, execute("select * from mv_tstest").size());
+        Assert.assertEquals(2, executeView("select * from %s").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));
+        assertRows(executeView("SELECT c from %s 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"));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "baz"), row(0), row(1));
+        assertRows(executeView("SELECT c from %s 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));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "bar"), row(0));
+        assertRows(executeView("SELECT c from %s where k = 0 and val = ?", "baz"), row(1));
     }
 
     @Test
-    public void testCountersTable() throws Throwable
+    public void testCountersTable()
     {
         createTable("CREATE TABLE %s (" +
                     "k int PRIMARY KEY, " +
                     "count counter)");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+            createView("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)
+        catch (Exception e)
         {
+            Assert.assertTrue(e.getCause() instanceof InvalidRequestException);
         }
     }
 
     @Test
-    public void testDurationsTable() throws Throwable
+    public void testDurationsTable()
     {
         createTable("CREATE TABLE %s (" +
                     "k int PRIMARY KEY, " +
                     "result duration)");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
         try
         {
-            createView("mv_duration", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE result IS NOT NULL AND k IS NOT NULL PRIMARY KEY (result,k)");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE result IS NOT NULL AND k IS NOT NULL PRIMARY KEY (result,k)");
             Assert.fail("MV on duration should fail");
         }
-        catch (InvalidQueryException e)
+        catch (Exception e)
         {
-            Assert.assertEquals("duration type is not supported for PRIMARY KEY column 'result'", e.getMessage());
+            Throwable cause = e.getCause();
+            Assert.assertEquals("duration type is not supported for PRIMARY KEY column 'result'", cause.getMessage());
         }
     }
 
@@ -222,21 +211,13 @@ public class ViewTest extends ViewAbstractTest
                     "intval int, " +
                     "PRIMARY KEY (k, c))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-
-        for(int i = 0; i < 1024; i++)
+        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);
+        createView("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)");
 
         assertRows(execute("SELECT count(*) from %s WHERE k = ?", 0), row(1024L));
-        assertRows(execute("SELECT count(*) from mv WHERE intval = ?", 0), row(1024L));
+        assertRows(executeView("SELECT count(*) from %s WHERE intval = ?", 0), row(1024L));
     }
 
     @Test
@@ -248,19 +229,16 @@ public class ViewTest extends ViewAbstractTest
                     "listval list<int>, " +
                     "PRIMARY KEY (k))");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+        createView("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)));
+        assertRows(executeView("SELECT k, listval from %s 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)));
+        assertRows(executeView("SELECT k, listval from %s WHERE intval = ?", 1), row(1, list(1, 2, 3)));
     }
 
     @Test
@@ -268,11 +246,7 @@ public class ViewTest extends ViewAbstractTest
     {
         createTable("CREATE TABLE %s (k int, intval int,  listval frozen<list<tuple<text,text>>>, PRIMARY KEY (k))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("mv",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND listval IS NOT NULL PRIMARY KEY (k, listval)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE k IS NOT NULL AND listval IS NOT NULL PRIMARY KEY (k, listval)");
 
         updateView("INSERT INTO %s (k, intval, listval) VALUES (?, ?, fromJson(?))",
                    0,
@@ -282,7 +256,7 @@ public class ViewTest extends ViewAbstractTest
         // verify input
         assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0),
                    row(0, list(tuple("a", "1"), tuple("b", "2"), tuple("c", "3"))));
-        assertRows(execute("SELECT k, listval from mv"),
+        assertRows(executeView("SELECT k, listval from %s"),
                    row(0, list(tuple("a", "1"), tuple("b", "2"), tuple("c", "3"))));
 
         // update listval with the same value and it will be compared in view generator
@@ -292,7 +266,7 @@ public class ViewTest extends ViewAbstractTest
         // verify result
         assertRows(execute("SELECT k, listval FROM %s WHERE k = ?", 0),
                    row(0, list(tuple("a", "1"), tuple("b", "2"), tuple("c", "3"))));
-        assertRows(execute("SELECT k, listval from mv"),
+        assertRows(executeView("SELECT k, listval from %s"),
                    row(0, list(tuple("a", "1"), tuple("b", "2"), tuple("c", "3"))));
     }
 
@@ -304,18 +278,15 @@ public class ViewTest extends ViewAbstractTest
                     "intval int, " +
                     "PRIMARY KEY (k))");
 
-        execute("USE " + keyspace());
-        executeNet("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)");
+        createView("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));
+        assertRows(executeView("SELECT k, intval from %s 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));
+        assertRows(executeView("SELECT k, intval from %s WHERE intval = ?", 1), row(0, 1));
     }
 
     @Test
@@ -330,31 +301,28 @@ public class ViewTest extends ViewAbstractTest
                     "d int, " +
                     "PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("mv", "CREATE MATERIALIZED VIEW %s AS SELECT a, b, c FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b, c FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, 0);
-        assertRows(execute("SELECT a, b, c from mv WHERE b = ?", 0), row(0, 0, 0));
+        assertRows(executeView("SELECT a, b, c from %s WHERE b = ?", 0), row(0, 0, 0));
 
         updateView("UPDATE %s SET d = ? WHERE a = ? AND b = ?", 0, 0, 0);
-        assertRows(execute("SELECT a, b, c from mv WHERE b = ?", 0), row(0, 0, 0));
+        assertRows(executeView("SELECT a, b, c from %s WHERE b = ?", 0), row(0, 0, 0));
 
         // Note: errors here may result in the test hanging when the memtables are flushed as part of the table drop,
         // because empty rows in the memtable will cause the flush to fail.  This will result in a test timeout that
         // should not be ignored.
         String table = KEYSPACE + "." + currentTable();
         updateView("BEGIN BATCH " +
-                "INSERT INTO " + table + " (a, b, c, d) VALUES (?, ?, ?, ?); " + // should be accepted
-                "UPDATE " + table + " SET d = ? WHERE a = ? AND b = ?; " +  // should be accepted
-                "APPLY BATCH",
-                0, 0, 0, 0,
-                1, 0, 1);
-        assertRows(execute("SELECT a, b, c from mv WHERE b = ?", 0), row(0, 0, 0));
-        assertRows(execute("SELECT a, b, c from mv WHERE b = ?", 1), row(0, 1, null));
-
-        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore("mv");
+                   "INSERT INTO " + table + " (a, b, c, d) VALUES (?, ?, ?, ?); " + // should be accepted
+                   "UPDATE " + table + " SET d = ? WHERE a = ? AND b = ?; " +  // should be accepted
+                   "APPLY BATCH",
+                   0, 0, 0, 0,
+                   1, 0, 1);
+        assertRows(executeView("SELECT a, b, c from %s WHERE b = ?", 0), row(0, 0, 0));
+        assertRows(executeView("SELECT a, b, c from %s WHERE b = ?", 1), row(0, 1, null));
+
+        ColumnFamilyStore cfs = Keyspace.open(keyspace()).getColumnFamilyStore(currentView());
         cfs.forceBlockingFlush();
         Assert.assertEquals(1, cfs.getLiveSSTables().size());
     }
@@ -371,12 +339,12 @@ public class ViewTest extends ViewAbstractTest
 
         executeNet("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)");
+        createView("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)");
 
         String table = keyspace() + "." + currentTable();
         updateView("DELETE FROM " + table + " USING TIMESTAMP 6 WHERE a = 1 AND b = 1;");
         updateView("INSERT INTO %s (a, b, c, d) VALUES (?, ?, ?, ?) USING TIMESTAMP 3", 1, 1, 1, 1);
-        Assert.assertEquals(0, executeNet("SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+        Assert.assertEquals(0, executeViewNet("SELECT * FROM %s WHERE c = 1 AND a = 1 AND b = 1").all().size());
     }
 
     @Test
@@ -389,13 +357,13 @@ public class ViewTest extends ViewAbstractTest
 
         executeNet("USE " + keyspace());
 
-        createView("mv1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 1);
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 2);
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 1, 3);
 
-        ResultSet mvRows = executeNet("SELECT a, b FROM mv1");
+        ResultSet mvRows = executeViewNet("SELECT a, b FROM %s");
         assertRowsNet(mvRows, row(1, 1), row(1, 2), row(1, 3));
 
         updateView(String.format("BEGIN UNLOGGED BATCH " +
@@ -403,7 +371,7 @@ public class ViewTest extends ViewAbstractTest
                                  "DELETE FROM %s WHERE a = 1;" +
                                  "APPLY BATCH", currentTable(), currentTable()));
 
-        mvRows = executeNet("SELECT a, b FROM mv1");
+        mvRows = executeViewNet("SELECT a, b FROM %s");
         assertRowsNet(mvRows);
     }
 
@@ -417,18 +385,18 @@ public class ViewTest extends ViewAbstractTest
                     "PRIMARY KEY (a))");
 
         executeNet("USE " + keyspace());
-        createView("mvmap", "CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %%s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT a, b FROM %s WHERE a IS NOT NULL AND b IS NOT NULL PRIMARY KEY (b, a)");
 
         updateView("INSERT INTO %s (a, b) VALUES (?, ?)", 0, 0);
-        ResultSet mvRows = executeNet("SELECT a, b FROM mvmap WHERE b = ?", 0);
+        ResultSet mvRows = executeViewNet("SELECT a, b FROM %s WHERE b = ?", 0);
         assertRowsNet(mvRows, row(0, 0));
 
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, map(1, "1"));
-        mvRows = executeNet("SELECT a, b FROM mvmap WHERE b = ?", 1);
+        mvRows = executeViewNet("SELECT a, b FROM %s WHERE b = ?", 1);
         assertRowsNet(mvRows, row(1, 1));
 
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 0, 0, map(0, "0"));
-        mvRows = executeNet("SELECT a, b FROM mvmap WHERE b = ?", 0);
+        mvRows = executeViewNet("SELECT a, b FROM %s WHERE b = ?", 0);
         assertRowsNet(mvRows, row(0, 0));
     }
 
@@ -439,17 +407,16 @@ public class ViewTest extends ViewAbstractTest
 
         executeNet("USE " + keyspace());
 
-        createView("mv",
-                   "CREATE MATERIALIZED VIEW %s AS" +
+        createView("CREATE MATERIALIZED VIEW %s AS" +
                    "  SELECT \"keyspace\", \"token\"" +
-                   "  FROM %%s" +
+                   "  FROM %s" +
                    "  WHERE \"keyspace\" IS NOT NULL AND \"token\" IS NOT NULL" +
                    "  PRIMARY KEY (\"keyspace\", \"token\")");
 
         execute("INSERT INTO %s (\"token\", \"keyspace\") VALUES (?, ?)", 0, 1);
 
         assertRowsNet(executeNet("SELECT * FROM %s"), row(0, 1));
-        assertRowsNet(executeNet("SELECT * FROM mv"), row(1, 0));
+        assertRowsNet(executeViewNet("SELECT * FROM %s"), row(1, 0));
     }
 
     private void testViewBuilderResume(int concurrentViewBuilders) throws Throwable
@@ -460,9 +427,6 @@ public class ViewTest extends ViewAbstractTest
                     "val text, " +
                     "PRIMARY KEY(k,c))");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
         CompactionManager.instance.setConcurrentViewBuilders(concurrentViewBuilders);
         CompactionManager.instance.setCoreCompactorThreads(1);
         CompactionManager.instance.setMaximumCompactorThreads(1);
@@ -470,43 +434,41 @@ public class ViewTest extends ViewAbstractTest
         cfs.disableAutoCompaction();
 
         for (int i = 0; i < 1024; i++)
-            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i);
+            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, String.valueOf(i));
 
         cfs.forceBlockingFlush();
 
         for (int i = 0; i < 1024; i++)
-            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i);
+            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, String.valueOf(i));
 
         cfs.forceBlockingFlush();
 
         for (int i = 0; i < 1024; i++)
-            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i);
+            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, String.valueOf(i));
 
         cfs.forceBlockingFlush();
 
         for (int i = 0; i < 1024; i++)
-            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, ""+i);
+            execute("INSERT into %s (k,c,val)VALUES(?,?,?)", i, i, String.valueOf(i));
 
         cfs.forceBlockingFlush();
 
-        String viewName1 = "mv_test_" + concurrentViewBuilders;
-        createView(viewName1, "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)");
+        String mv1 = createViewAsync("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)");
 
         cfs.enableAutoCompaction();
         List<? extends Future<?>> futures = CompactionManager.instance.submitBackground(cfs);
 
-        String viewName2 = viewName1 + "_2";
         //Force a second MV on the same base table, which will restart the first MV builder...
-        createView(viewName2, "CREATE MATERIALIZED VIEW %s AS SELECT val, k, c FROM %%s WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
-
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT val, k, c FROM %s " +
+                   "WHERE val IS NOT NULL AND k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (val,k,c)");
 
         //Compact the base table
         FBUtilities.waitOnFutures(futures);
 
-        while (!SystemKeyspace.isViewBuilt(keyspace(), viewName1))
-            Uninterruptibles.sleepUninterruptibly(1, TimeUnit.SECONDS);
+        waitForViewBuild(mv1);
 
-        assertRows(execute("SELECT count(*) FROM " + viewName1), row(1024L));
+        assertRows(executeView("SELECT count(*) FROM %s"), row(1024L));
     }
 
     @Test
@@ -522,15 +484,13 @@ public class ViewTest extends ViewAbstractTest
      * Tests that a client warning is issued on materialized view creation.
      */
     @Test
-    public void testClientWarningOnCreate() throws Throwable
+    public void testClientWarningOnCreate()
     {
         createTable("CREATE TABLE %s (k int PRIMARY KEY, v int)");
 
         ClientWarn.instance.captureWarnings();
-        String viewName = keyspace() + ".warning_view";
-        execute("CREATE MATERIALIZED VIEW " + viewName +
-                " AS SELECT * FROM %s WHERE k IS NOT NULL AND v IS NOT NULL PRIMARY KEY (v, k)");
-        views.add(viewName);
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE k IS NOT NULL AND v IS NOT NULL PRIMARY KEY (v, k)");
         List<String> warnings = ClientWarn.instance.getWarnings();
 
         Assert.assertNotNull(warnings);
@@ -552,13 +512,14 @@ public class ViewTest extends ViewAbstractTest
         try
         {
             DatabaseDescriptor.setEnableMaterializedViews(false);
-            createView("view1", "CREATE MATERIALIZED VIEW %s AS SELECT v FROM %%s WHERE k IS NOT NULL AND v IS NOT NULL PRIMARY KEY (v, k)");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT v FROM %s WHERE k IS NOT NULL AND v IS NOT NULL PRIMARY KEY (v, k)");
             Assert.fail("Should not be able to create a materialized view if they are disabled");
         }
-        catch (Throwable e)
+        catch (RuntimeException e)
         {
-            Assert.assertTrue(e instanceof InvalidQueryException);
-            Assert.assertTrue(e.getMessage().contains("Materialized views are disabled"));
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(InvalidRequestException.class);
+            Assertions.assertThat(cause.getMessage()).contains("Materialized views are disabled");
         }
         finally
         {
@@ -573,8 +534,8 @@ public class ViewTest extends ViewAbstractTest
 
         executeNet("USE " + keyspace());
 
-        createView("view1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE \"theKey\" IS NOT NULL AND \"theClustering_1\" IS NOT NULL AND \"theClustering_2\" IS NOT NULL AND \"theValue\" IS NOT NULL  PRIMARY KEY (\"theKey\", \"theClustering_1\", \"theClustering_2\");");
-        createView("view2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE \"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL  PRIMARY KEY (\"theKey\", \"theClustering_1\", \"theClustering_2\");");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE \"theKey\" IS NOT NULL AND \"theClustering_1\" IS NOT NULL AND \"theClustering_2\" IS NOT NULL AND \"theValue\" IS NOT NULL  PRIMARY KEY (\"theKey\", \"theClustering_1\", \"theClustering_2\");");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE \"theKey\" IS NOT NULL AND (\"theClustering_1\", \"theClustering_2\") = (1, 2) AND \"theValue\" IS NOT NULL  PRIMARY KEY (\"theKey\", \"theClustering_1\", \"theClustering_2\");");
 
         assertRows(execute("SELECT where_clause FROM system_schema.views"),
                    row("\"theKey\" IS NOT NULL AND \"theClustering_1\" IS NOT NULL AND \"theClustering_2\" IS NOT NULL AND \"theValue\" IS NOT NULL"),
@@ -600,7 +561,7 @@ public class ViewTest extends ViewAbstractTest
         assertEmpty(testFunctionInWhereClause("CREATE TABLE %s (k bigint PRIMARY KEY, v int)",
                                               null,
                                               "CREATE MATERIALIZED VIEW %s AS" +
-                                              "   SELECT * FROM %%s WHERE k = token(1) AND v IS NOT NULL " +
+                                              "   SELECT * FROM %s WHERE k = token(1) AND v IS NOT NULL " +
                                               "   PRIMARY KEY (v, k)",
                                               "k = token(1) AND v IS NOT NULL",
                                               "INSERT INTO %s(k, v) VALUES (0, 1)",
@@ -610,7 +571,7 @@ public class ViewTest extends ViewAbstractTest
         assertEmpty(testFunctionInWhereClause("CREATE TABLE %s (k bigint PRIMARY KEY, v int)",
                                               null,
                                               "CREATE MATERIALIZED VIEW %s AS" +
-                                              "   SELECT * FROM %%s WHERE k = TOKEN(1) AND v IS NOT NULL" +
+                                              "   SELECT * FROM %s WHERE k = TOKEN(1) AND v IS NOT NULL" +
                                               "   PRIMARY KEY (v, k)",
                                               "k = token(1) AND v IS NOT NULL",
                                               "INSERT INTO %s(k, v) VALUES (0, 1)",
@@ -623,7 +584,7 @@ public class ViewTest extends ViewAbstractTest
                                              "   RETURNS int LANGUAGE java" +
                                              "   AS 'return 2;'",
                                              "CREATE MATERIALIZED VIEW %s AS " +
-                                             "   SELECT * FROM %%s WHERE k = fun() AND v IS NOT NULL" +
+                                             "   SELECT * FROM %s WHERE k = fun() AND v IS NOT NULL" +
                                              "   PRIMARY KEY (v, k)",
                                              "k = fun() AND v IS NOT NULL",
                                              "INSERT INTO %s(k, v) VALUES (0, 1)",
@@ -637,7 +598,7 @@ public class ViewTest extends ViewAbstractTest
                                              "   LANGUAGE java" +
                                              "   AS 'return 2;'",
                                              "CREATE MATERIALIZED VIEW %s AS " +
-                                             "   SELECT * FROM %%s WHERE k = \"FUN\"() AND v IS NOT NULL" +
+                                             "   SELECT * FROM %s WHERE k = \"FUN\"() AND v IS NOT NULL" +
                                              "   PRIMARY KEY (v, k)",
                                              "k = \"FUN\"() AND v IS NOT NULL",
                                              "INSERT INTO %s(k, v) VALUES (0, 1)",
@@ -652,7 +613,7 @@ public class ViewTest extends ViewAbstractTest
                                              "   LANGUAGE java" +
                                              "   AS 'return x;'",
                                              "CREATE MATERIALIZED VIEW %s AS" +
-                                             "   SELECT * FROM %%s WHERE k = \"TOKEN\"(2) AND v IS NOT NULL" +
+                                             "   SELECT * FROM %s WHERE k = \"TOKEN\"(2) AND v IS NOT NULL" +
                                              "   PRIMARY KEY (v, k)",
                                              "k = \"TOKEN\"(2) AND v IS NOT NULL",
                                              "INSERT INTO %s(k, v) VALUES (0, 1)",
@@ -667,7 +628,7 @@ public class ViewTest extends ViewAbstractTest
                                              "   LANGUAGE java" +
                                              "   AS 'return x;'",
                                              "CREATE MATERIALIZED VIEW %s AS" +
-                                             "   SELECT * FROM %%s " +
+                                             "   SELECT * FROM %s " +
                                              "   WHERE k = " + keyspace() + ".\"token\"(2) AND v IS NOT NULL" +
                                              "   PRIMARY KEY (v, k)",
                                              "k = " + keyspace() + ".\"token\"(2) AND v IS NOT NULL",
@@ -678,7 +639,7 @@ public class ViewTest extends ViewAbstractTest
     /**
      * Tests that truncating a table stops the ongoing builds of its materialized views,
      * so they don't write into the MV data that has been truncated in the base table.
-     *
+     * <p>
      * See CASSANDRA-16567 for further details.
      */
     @Test
@@ -696,33 +657,30 @@ public class ViewTest extends ViewAbstractTest
     public void testTruncateWhileBuilding() throws Throwable
     {
         createTable("CREATE TABLE %s (k int, c int, v int, PRIMARY KEY(k, c))");
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
         execute("INSERT INTO %s (k, c, v) VALUES (?, ?, ?)", 0, 0, 0);
-        createView("mv",
-                   "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s " +
-                   "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
-                   "PRIMARY KEY (v, c, k)");
+        createViewAsync("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                        "WHERE k IS NOT NULL AND c IS NOT NULL AND v IS NOT NULL " +
+                        "PRIMARY KEY (v, c, k)");
 
         // check that the delayed view builder tasks are either running or pending,
         // and that they haven't written anything yet
         assertThat(runningViewBuilds()).isPositive();
-        assertFalse(SystemKeyspace.isViewBuilt(KEYSPACE, "mv"));
+        assertFalse(SystemKeyspace.isViewBuilt(KEYSPACE, currentView()));
         waitForViewMutations();
-        assertRows(execute("SELECT * FROM mv"));
+        assertRows(executeView("SELECT * FROM %s"));
 
         // truncate the view, this should unblock the view builders, wait for their cancellation,
         // drop the sstables and, finally, start a new view build
         updateView("TRUNCATE %s");
 
         // check that there aren't any rows after truncating
-        assertRows(execute("SELECT * FROM mv"));
+        assertRows(executeView("SELECT * FROM %s"));
 
         // check that the view builder tasks finish and that the view is still empty after that
         Awaitility.await().untilAsserted(() -> assertEquals(0, runningViewBuilds()));
-        assertTrue(SystemKeyspace.isViewBuilt(KEYSPACE, "mv"));
+        assertTrue(SystemKeyspace.isViewBuilt(KEYSPACE, currentView()));
         waitForViewMutations();
-        assertRows(execute("SELECT * FROM mv"));
+        assertRows(executeView("SELECT * FROM %s"));
     }
 
     private static int runningViewBuilds()
@@ -740,28 +698,24 @@ public class ViewTest extends ViewAbstractTest
     {
         createTable(createTableQuery);
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
         if (createFunctionQuery != null)
         {
             execute(createFunctionQuery);
         }
 
-        String viewName = "view_" + viewNameSeqNumber.getAndIncrement();
-        createView(viewName, createViewQuery);
+        createView(createViewQuery);
 
         // Test the where clause stored in system_schema.views
         String schemaQuery = String.format("SELECT where_clause FROM %s.%s WHERE keyspace_name = ? AND view_name = ?",
                                            SchemaConstants.SCHEMA_KEYSPACE_NAME,
                                            SchemaKeyspaceTables.VIEWS);
-        assertRows(execute(schemaQuery, keyspace(), viewName), row(expectedSchemaWhereClause));
+        assertRows(execute(schemaQuery, keyspace(), currentView()), row(expectedSchemaWhereClause));
 
         for (String insert : insertQueries)
         {
             execute(insert);
         }
 
-        return execute("SELECT * FROM " + viewName);
+        return executeView("SELECT * FROM %s");
     }
 }
diff --git a/test/unit/org/apache/cassandra/cql3/ViewTimesTest.java b/test/unit/org/apache/cassandra/cql3/ViewTimesTest.java
index 8d5e453..f3178b4 100644
--- a/test/unit/org/apache/cassandra/cql3/ViewTimesTest.java
+++ b/test/unit/org/apache/cassandra/cql3/ViewTimesTest.java
@@ -27,7 +27,9 @@ import org.junit.Test;
 import com.datastax.driver.core.ResultSet;
 import com.datastax.driver.core.Row;
 import org.apache.cassandra.db.Keyspace;
+import org.apache.cassandra.exceptions.InvalidRequestException;
 import org.apache.cassandra.utils.FBUtilities;
+import org.assertj.core.api.Assertions;
 
 import static org.junit.Assert.assertEquals;
 
@@ -50,15 +52,14 @@ public class ViewTimesTest extends ViewAbstractTest
                     "c int, " +
                     "val int)");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
-        createView("mv_rctstest", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                   "WHERE k IS NOT NULL AND c IS NOT NULL " +
+                   "PRIMARY KEY (k,c)");
 
         updateView("UPDATE %s SET c = ?, val = ? WHERE k = ?", 0, 0, 0);
         updateView("UPDATE %s SET val = ? WHERE k = ?", 1, 0);
         updateView("UPDATE %s SET c = ? WHERE k = ?", 1, 0);
-        assertRows(execute("SELECT c, k, val FROM mv_rctstest"), row(1, 0, 1));
+        assertRows(executeView("SELECT c, k, val FROM %s"), row(1, 0, 1));
 
         updateView("TRUNCATE %s");
 
@@ -68,8 +69,8 @@ public class ViewTimesTest extends ViewAbstractTest
         updateView("UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE k = ?", 2, 0);
         updateView("UPDATE %s USING TIMESTAMP 3 SET val = ? WHERE k = ?", 2, 0);
 
-        assertRows(execute("SELECT c, k, val FROM mv_rctstest"), row(2, 0, 2));
-        assertRows(execute("SELECT c, k, val FROM mv_rctstest limit 1"), row(2, 0, 2));
+        assertRows(executeView("SELECT c, k, val FROM %s"), row(2, 0, 2));
+        assertRows(executeView("SELECT c, k, val FROM %s limit 1"), row(2, 0, 2));
     }
 
     @Test
@@ -88,20 +89,20 @@ public class ViewTimesTest extends ViewAbstractTest
     {
         createTable("CREATE TABLE %s (a int, b int, c int, d int, e int, PRIMARY KEY (a, b))");
 
-        execute("USE " + keyspace());
-        executeNet("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();
+        createView("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(currentView()).disableAutoCompaction();
 
         //Set initial values TS=0, leaving e null and verify view
         executeNet("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));
+        assertRows(executeView("SELECT d from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
 
         //update c's timestamp TS=2
         executeNet("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));
+        assertRows(executeView("SELECT d from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0));
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
@@ -110,11 +111,11 @@ public class ViewTimesTest extends ViewAbstractTest
         executeNet("UPDATE %s USING TIMESTAMP 3 SET c = ? WHERE a = ? and b = ? ", 0, 0, 0);
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
-        assertRows(execute("SELECT d from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0));
+        assertRows(executeView("SELECT d from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0));
 
         if(flush)
         {
-            ks.getColumnFamilyStore("mv").forceMajorCompaction();
+            ks.getColumnFamilyStore(currentView()).forceMajorCompaction();
             FBUtilities.waitOnFutures(ks.flush());
         }
 
@@ -123,16 +124,15 @@ public class ViewTimesTest extends ViewAbstractTest
         executeNet("UPDATE %s USING TIMESTAMP 4 SET c = ? WHERE a = ? and b = ? ", 1, 0, 0);
         if (flush)
         {
-            ks.getColumnFamilyStore("mv").forceMajorCompaction();
+            ks.getColumnFamilyStore(currentView()).forceMajorCompaction();
             FBUtilities.waitOnFutures(ks.flush());
         }
 
-        assertRows(execute("SELECT d,e from mv WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, null));
-
+        assertRows(executeView("SELECT d,e from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, null));
 
         //Add e value @ TS=1
         executeNet("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));
+        assertRows(executeView("SELECT d,e from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(0, 1));
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
@@ -140,26 +140,25 @@ public class ViewTimesTest extends ViewAbstractTest
 
         //Change d value @ TS=2
         executeNet("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));
+        assertRows(executeView("SELECT d from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(2));
 
         if (flush)
             FBUtilities.waitOnFutures(ks.flush());
 
-
         //Change d value @ TS=3
         executeNet("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));
+        assertRows(executeView("SELECT d from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row(1));
 
 
         //Tombstone c
         executeNet("DELETE FROM %s WHERE a = ? and b = ?", 0, 0);
-        assertRows(execute("SELECT d from mv"));
+        assertRows(executeView("SELECT d from %s"));
 
         //Add back without D
         executeNet("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));
+        assertRows(executeView("SELECT d from %s WHERE c = ? and a = ? and b = ?", 1, 0, 0), row((Object) null));
 
 
         //New partition
@@ -171,20 +170,17 @@ public class ViewTimesTest extends ViewAbstractTest
 
         // delete with timestamp 0 (which should only delete d)
         executeNet("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)
-        );
+        assertRows(executeView("SELECT a, b, c, d, e from %s WHERE c = ? and a = ? and b = ?", 0, 1, 0),
+                   row(1, 0, 0, null, 0));
 
         executeNet("UPDATE %s USING TIMESTAMP 2 SET c = ? WHERE a = ? AND b = ?", 1, 1, 0);
         executeNet("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)
-        );
+        assertRows(executeView("SELECT a, b, c, d, e from %s WHERE c = ? and a = ? and b = ?", 0, 1, 0),
+                   row(1, 0, 0, null, 0));
 
         executeNet("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)
-        );
+        assertRows(executeView("SELECT a, b, c, d, e from %s WHERE c = ? and a = ? and b = ?", 0, 1, 0),
+                   row(1, 0, 0, 0, 0));
     }
 
     @Test
@@ -199,7 +195,7 @@ public class ViewTimesTest extends ViewAbstractTest
 
         executeNet("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)");
+        createView("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 3", 1, 1, 1, 1);
 
@@ -207,7 +203,7 @@ public class ViewTimesTest extends ViewAbstractTest
         updateView("INSERT INTO %s (a, b, c) VALUES (?, ?, ?)", 1, 1, 2);
 
         Thread.sleep(TimeUnit.SECONDS.toMillis(5));
-        List<Row> results = executeNet("SELECT d FROM mv WHERE c = 2 AND a = 1 AND b = 1").all();
+        List<Row> results = executeViewNet("SELECT d FROM %s 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));
     }
@@ -224,12 +220,12 @@ public class ViewTimesTest extends ViewAbstractTest
 
         executeNet("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)");
+        createView("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 3", 1, 1, 1, 1);
 
         Thread.sleep(TimeUnit.SECONDS.toMillis(4));
-        Assert.assertEquals(0, executeNet("SELECT * FROM mv WHERE c = 1 AND a = 1 AND b = 1").all().size());
+        Assert.assertEquals(0, executeViewNet("SELECT * FROM %s WHERE c = 1 AND a = 1 AND b = 1").all().size());
     }
 
     @Test
@@ -243,14 +239,14 @@ public class ViewTimesTest extends ViewAbstractTest
 
         executeNet("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)");
+        createView("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("SELECT c FROM mv");
+        ResultSet mvRows = executeViewNet("SELECT c FROM %s");
         List<Row> rows = executeNet("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");
@@ -258,24 +254,25 @@ public class ViewTimesTest extends ViewAbstractTest
     }
 
     @Test
-    public void testCreateMvWithTTL() throws Throwable
+    public void testCreateMvWithTTL()
     {
         createTable("CREATE TABLE %s (" +
                     "k int PRIMARY KEY, " +
                     "c int, " +
                     "val int) WITH default_time_to_live = 60");
 
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
-
         // Must NOT include "default_time_to_live" for Materialized View creation
         try
         {
-            createView("mv_ttl1", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c) WITH default_time_to_live = 30");
+            createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s " +
+                       "WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c) WITH default_time_to_live = 30");
             Assert.fail("Should fail if TTL is provided for materialized view");
         }
-        catch (Exception e)
+        catch (RuntimeException e)
         {
+            Throwable cause = e.getCause();
+            Assertions.assertThat(cause).isInstanceOf(InvalidRequestException.class);
+            Assertions.assertThat(cause.getMessage()).contains("Cannot set default_time_to_live for a materialized view");
         }
     }
 
@@ -287,15 +284,12 @@ public class ViewTimesTest extends ViewAbstractTest
                     "c int, " +
                     "val int) WITH default_time_to_live = 60");
 
-        createView("mv_ttl2", "CREATE MATERIALIZED VIEW %s AS SELECT * FROM %%s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
-
-        execute("USE " + keyspace());
-        executeNet("USE " + keyspace());
+        createView("CREATE MATERIALIZED VIEW %s AS SELECT * FROM %s WHERE k IS NOT NULL AND c IS NOT NULL PRIMARY KEY (k,c)");
 
         // Must NOT include "default_time_to_live" on alter Materialized View
         try
         {
-            executeNet("ALTER MATERIALIZED VIEW " + keyspace() + ".mv_ttl2 WITH default_time_to_live = 30");
+            executeNet("ALTER MATERIALIZED VIEW " + currentView() + " WITH default_time_to_live = 30");
             Assert.fail("Should fail if TTL is provided while altering materialized view");
         }
         catch (Exception e)

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