You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2020/11/16 23:59:36 UTC

[phoenix] branch 4.x updated: PHOENIX-6191: Creating a view which has its own new columns should also do checkAndPut checks on SYSTEM.MUTEX

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

chinmayskulkarni pushed a commit to branch 4.x
in repository https://gitbox.apache.org/repos/asf/phoenix.git


The following commit(s) were added to refs/heads/4.x by this push:
     new e58b758  PHOENIX-6191: Creating a view which has its own new columns should also do checkAndPut checks on SYSTEM.MUTEX
e58b758 is described below

commit e58b75815d95ac06b3931a27e683a668b50e4907
Author: Chinmay Kulkarni <ch...@gmail.com>
AuthorDate: Thu Nov 12 23:01:09 2020 -0800

    PHOENIX-6191: Creating a view which has its own new columns should also do checkAndPut checks on SYSTEM.MUTEX
---
 .../end2end/ViewConcurrencyAndFailureIT.java       |  794 +++++++++++++++
 .../it/java/org/apache/phoenix/end2end/ViewIT.java | 1032 +++++++-------------
 .../org/apache/phoenix/end2end/ViewMetadataIT.java |  870 +++++++++++------
 .../org/apache/phoenix/schema/MetaDataClient.java  |   26 +-
 4 files changed, 1745 insertions(+), 977 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewConcurrencyAndFailureIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewConcurrencyAndFailureIT.java
new file mode 100644
index 0000000..4383307
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewConcurrencyAndFailureIT.java
@@ -0,0 +1,794 @@
+/*
+ * 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.phoenix.end2end;
+
+import static java.util.concurrent.Executors.newSingleThreadExecutor;
+import static org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost
+        .PHOENIX_META_DATA_COPROCESSOR_CONF_KEY;
+import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
+import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.hadoop.hbase.DoNotRetryIOException;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.phoenix.coprocessor.BaseMetaDataEndpointObserver;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost
+        .PhoenixMetaDataControllerEnvironment;
+import org.apache.phoenix.exception.PhoenixIOException;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.schema.ConcurrentTableMutationException;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.util.PhoenixRuntime;
+import org.apache.phoenix.util.PropertiesUtil;
+import org.apache.phoenix.util.ReadOnlyProps;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.TestUtil;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Tests for views dealing with other ongoing concurrent operations and
+ * failure scenarios
+ */
+@RunWith(Parameterized.class)
+public class ViewConcurrencyAndFailureIT extends SplitSystemCatalogIT {
+
+    protected String tableDDLOptions;
+    protected String transactionProvider;
+    protected boolean columnEncoded;
+
+    private static final String FAILED_VIEWNAME = SchemaUtil.getTableName(
+            SCHEMA2, "FAILED_VIEW_" + generateUniqueName());
+    private static final String SLOW_VIEWNAME_PREFIX =
+            SchemaUtil.getTableName(SCHEMA2, "SLOW_VIEW");
+
+    private static volatile CountDownLatch latch1 = null;
+    private static volatile CountDownLatch latch2 = null;
+    private static volatile boolean throwExceptionInChildLinkPreHook = false;
+    private static volatile boolean slowDownAddingChildLink = false;
+
+    public ViewConcurrencyAndFailureIT(String transactionProvider,
+            boolean columnEncoded) {
+        StringBuilder optionBuilder = new StringBuilder();
+        this.transactionProvider = transactionProvider;
+        this.columnEncoded = columnEncoded;
+        if (transactionProvider != null) {
+            optionBuilder.append(" TRANSACTION_PROVIDER='")
+                    .append(transactionProvider)
+                    .append("'");
+        }
+        if (!columnEncoded) {
+            if (optionBuilder.length()!=0)
+                optionBuilder.append(",");
+            optionBuilder.append("COLUMN_ENCODED_BYTES=0");
+        }
+        this.tableDDLOptions = optionBuilder.toString();
+    }
+
+    // name is used by failsafe as file name in reports
+    @Parameters(name="ViewIT_transactionProvider={0}, columnEncoded={1}")
+    public static synchronized Collection<Object[]> data() {
+        return TestUtil.filterTxParamData(Arrays.asList(new Object[][] {
+                { "TEPHRA", false }, { "TEPHRA", true },
+                { "OMID", false },
+                { null, false }, { null, true }}),0);
+    }
+
+    @BeforeClass
+    public static synchronized void doSetup() throws Exception {
+        NUM_SLAVES_BASE = 6;
+        boolean splitSystemCatalog = (driver == null);
+        Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
+        serverProps.put(QueryServices.PHOENIX_ACLS_ENABLED, "true");
+        serverProps.put(PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
+                TestMetaDataRegionObserver.class.getName());
+        serverProps.put("hbase.coprocessor.abortonerror", "false");
+        setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
+                ReadOnlyProps.EMPTY_PROPS);
+        // Split SYSTEM.CATALOG once after the mini-cluster is started
+        if (splitSystemCatalog) {
+            // splitSystemCatalog is incompatible with the balancer chore
+            getUtility().getHBaseCluster().getMaster().balanceSwitch(false);
+            splitSystemCatalog();
+        }
+    }
+
+    @After
+    public void cleanup() {
+        latch1 = null;
+        latch2 = null;
+        throwExceptionInChildLinkPreHook = false;
+        slowDownAddingChildLink = false;
+    }
+
+    @Test
+    public void testChildViewCreationFails() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String failingViewName = FAILED_VIEWNAME;
+            String succeedingViewName = SchemaUtil.getTableName(SCHEMA3,
+                    generateUniqueName());
+
+            String createTableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                    + tableDDLOptions;
+            stmt.execute(createTableDdl);
+
+            String createViewDdl = "CREATE VIEW " + failingViewName
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k > 5";
+            try {
+                stmt.execute(createViewDdl);
+                fail();
+            } catch (PhoenixIOException ignored) {
+            }
+            createViewDdl = "CREATE VIEW " + succeedingViewName
+                    + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName
+                    + " WHERE k > 10";
+            stmt.execute(createViewDdl);
+
+            // the first child view should not exist
+            try {
+                PhoenixRuntime.getTableNoCache(conn, failingViewName);
+                fail();
+            } catch (TableNotFoundException ignored) {
+            }
+
+            // we should be able to load the table
+            PhoenixRuntime.getTableNoCache(conn, fullTableName);
+            // we should be able to load the second view
+            PhoenixRuntime.getTableNoCache(conn, succeedingViewName);
+        }
+    }
+
+    @Test
+    public void testConcurrentViewCreationAndTableDrop() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_"
+                    + generateUniqueName();
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            String tableDdl = "CREATE TABLE " + fullTableName +
+                    "  (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER, v2 INTEGER)"
+                    + tableDDLOptions;
+            stmt.execute(tableDdl);
+
+            ExecutorService executorService = newSingleThreadExecutor(
+                    new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // When dropping a table, we check the parent->child links in the
+            // SYSTEM.CHILD_LINK table and check that cascade is set, if it
+            // isn't, we throw an exception (see ViewUtil.hasChildViews).
+            // After PHOENIX-4810, we first send a client-server RPC to add
+            // parent->child links to SYSTEM.CHILD_LINK and then add metadata
+            // for the view in SYSTEM.CATALOG, so we must delay link creation
+            // so that the drop table does not fail
+            slowDownAddingChildLink = true;
+            // create the view in a separate thread (which will take some time
+            // to complete)
+            Future<Exception> future = executorService.submit(
+                    new CreateViewRunnable(fullTableName, fullViewName1));
+            // wait till the thread makes the rpc to create the view
+            latch1.await();
+            tableDdl = "DROP TABLE " + fullTableName;
+
+            // Revert this flag since we don't want to wait in preDropTable
+            slowDownAddingChildLink = false;
+            // drop table goes through first and so the view creation
+            // should fail
+            stmt.execute(tableDdl);
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertTrue("Expected TableNotFoundException since drop table"
+                            + " goes through first",
+                    e instanceof TableNotFoundException &&
+                            fullTableName.equals(((TableNotFoundException) e)
+                                    .getTableName()));
+
+        }
+    }
+
+    @Test
+    public void testChildLinkCreationFailThrowsException() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SchemaUtil.getTableName(SCHEMA3,
+                    generateUniqueName());
+            // create base table
+            String tableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                    + tableDDLOptions;
+            stmt.execute(tableDdl);
+
+            // Throw an exception in ChildLinkMetaDataEndpoint while adding
+            // parent->child links to simulate a failure
+            throwExceptionInChildLinkPreHook = true;
+            // create a view
+            String ddl = "CREATE VIEW " + fullViewName1
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            try {
+                stmt.execute(ddl);
+                fail("Should have thrown an exception");
+            } catch(SQLException sqlE) {
+                assertEquals("Expected a different Error code",
+                        SQLExceptionCode.UNABLE_TO_CREATE_CHILD_LINK
+                                .getErrorCode(), sqlE.getErrorCode());
+            }
+        }
+    }
+
+    @Test
+    public void testConcurrentAddSameColumnDifferentType() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_"
+                    + generateUniqueName();
+            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                    generateUniqueName());
+            // create base table
+            String tableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                    + tableDDLOptions;
+            stmt.execute(tableDdl);
+            // create a view
+            String ddl = "CREATE VIEW " + fullViewName1
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            stmt.execute(ddl);
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = newSingleThreadExecutor(
+                    new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // add a column with the same name and different type to the view
+            // in a separate thread (which will take some time to complete)
+            Future<Exception> future = executorService.submit(
+                    new AddColumnRunnable(fullViewName1, null));
+            // wait till the thread makes the rpc to add the column
+            boolean result = latch1.await(2, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The create view rpc look too long");
+            }
+            tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
+            try {
+                // add the same column to the base table with a different type
+                stmt.execute(tableDdl);
+                fail("Adding a column to a base table should fail when "
+                        + "the same column of a different type is being added "
+                        + "to a child view");
+            } catch (ConcurrentTableMutationException ignored) {
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertNull(e);
+
+            // add the same column to the another view  to ensure that the cell
+            // used to prevent concurrent modifications was removed
+            ddl = "CREATE VIEW " + fullViewName2
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            stmt.execute(ddl);
+            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
+            stmt.execute(tableDdl);
+        }
+    }
+
+    // Test that we do a checkAndPut even in case of tenant-specific connections
+    // (see PHOENIX-6075)
+    @Test
+    public void testConcurrentAddSameColumnDifferentTypeTenantView()
+            throws Exception {
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_"
+                + generateUniqueName();
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
+        String tenantId = "t001";
+        String tableDdl = "CREATE TABLE " + fullTableName
+                + " (TENANT_ID VARCHAR NOT NULL, k INTEGER NOT NULL, v1 DATE "
+                + "CONSTRAINT PK"
+                + " PRIMARY KEY (TENANT_ID, k)) MULTI_TENANT=true"
+                + (!tableDDLOptions.isEmpty() ? ", " : "") + tableDDLOptions;
+        String viewDdl = "CREATE VIEW " + fullViewName1
+                + " (v2 VARCHAR) AS SELECT * FROM "
+                + fullTableName + " WHERE k = 6";
+
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            // create a multi-tenant base table
+            stmt.execute(tableDdl);
+
+            Properties props = new Properties();
+            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
+            try (Connection tenantConn = DriverManager.getConnection(getUrl(),
+                    props);
+                    Statement tenantStmt = tenantConn.createStatement()) {
+                // create a tenant-specific view
+                tenantStmt.execute(viewDdl);
+            }
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = newSingleThreadExecutor(
+                    new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // add a column with the same name and different type to the
+            // tenant-specific view in a
+            // separate thread (which will take some time to complete)
+            Future<Exception> future = executorService.submit(
+                    new AddColumnRunnable(fullViewName1, tenantId));
+            // wait till the thread makes the rpc to add the column
+            boolean result = latch1.await(2, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The tenant-specific view creation rpc look too long");
+            }
+            tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
+            try {
+                // add the same column to the base table with a different type
+                stmt.execute(tableDdl);
+                fail("Adding a column to a base table should fail when "
+                        + "the same column of a different type is being added"
+                        + " to a child view");
+            } catch (ConcurrentTableMutationException ignored) {
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertNull(e);
+
+            // add the same column to the another view  to ensure that the cell
+            // used to prevent concurrent modifications was removed
+            viewDdl = "CREATE VIEW " + fullViewName2
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            stmt.execute(viewDdl);
+            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
+            stmt.execute(tableDdl);
+        }
+    }
+
+    @Test
+    public void testConcurrentAddDifferentColumnParentHasColEncoding()
+            throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_"
+                    + generateUniqueName();
+            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                    generateUniqueName());
+            String fullViewName3 = SchemaUtil.getTableName(SCHEMA4,
+                    generateUniqueName());
+            // create base table
+            String tableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                    + tableDDLOptions;
+            stmt.execute(tableDdl);
+            // create two views
+            String ddl = "CREATE VIEW " + fullViewName1
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            stmt.execute(ddl);
+            ddl = "CREATE VIEW " + fullViewName3
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 7";
+            stmt.execute(ddl);
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = newSingleThreadExecutor(
+                    new ThreadFactory() {
+                @Override
+                public Thread newThread(Runnable r) {
+                    Thread t = Executors.defaultThreadFactory().newThread(r);
+                    t.setDaemon(true);
+                    t.setPriority(Thread.MIN_PRIORITY);
+                    return t;
+                }
+            });
+
+            // add a column to a view in a separate thread (we slow this
+            // operation down)
+            Future<Exception> future = executorService.submit(
+                    new AddColumnRunnable(fullViewName1, null));
+            // wait till the thread makes the rpc to add the column
+            boolean result = latch1.await(2, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The alter view rpc look too long");
+            }
+            tableDdl = "ALTER VIEW " + fullViewName3 + " ADD v4 INTEGER";
+            try {
+                // add a column to another view
+                stmt.execute(tableDdl);
+                if (columnEncoded) {
+                    // this should fail as the previous add column is still
+                    // not complete
+                    fail("Adding columns to two different views concurrently"
+                            + " where the base table"
+                            + " uses encoded columns should fail");
+                }
+            } catch (ConcurrentTableMutationException e) {
+                if (!columnEncoded) {
+                    // this should not fail as we don't need to update the
+                    // parent table for non column encoded tables
+                    fail("Adding columns to two different views concurrently"
+                            + " where the base table does not use encoded"
+                            + " columns should succeed");
+                }
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            // if the base table uses column encoding then the add column
+            // operation for fullViewName1 fails
+            assertNull(e);
+
+            // add the same column to the another view  to ensure that the cell
+            // used to prevent concurrent modifications was removed
+            ddl = "CREATE VIEW " + fullViewName2
+                    + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            stmt.execute(ddl);
+            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
+            stmt.execute(tableDdl);
+        }
+    }
+
+    /**
+     * Concurrently create a view with a WHERE condition and also try to drop
+     * the parent column on which the WHERE condition depends
+     */
+    @Test
+    public void testConcurrentViewCreationParentColDropViewCondition()
+            throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_"
+                    + generateUniqueName();
+
+            // create base table
+            String tableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER)"
+                    + tableDDLOptions;
+            stmt.execute(tableDdl);
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = newSingleThreadExecutor(
+                    new ThreadFactory() {
+                        @Override
+                        public Thread newThread(Runnable r) {
+                            Thread t = Executors.defaultThreadFactory()
+                                    .newThread(r);
+                            t.setDaemon(true);
+                            t.setPriority(Thread.MIN_PRIORITY);
+                            return t;
+                        }
+                    });
+
+            Future<Exception> future = executorService.submit(
+                    new CreateViewRunnable(fullTableName, fullViewName1));
+            // wait till the thread makes the rpc to add the column
+            boolean result = latch1.await(2, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The create view rpc look too long");
+            }
+            tableDdl = "ALTER TABLE " + fullTableName + " DROP COLUMN v1";
+            try {
+                // drop the view WHERE condition column from the parent
+                stmt.execute(tableDdl);
+                fail("Dropping a column from a base table should fail when a"
+                        + " child view is concurrently being created whose"
+                        + " view WHERE condition depends on this column");
+            } catch (ConcurrentTableMutationException ignored) {
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertNull(e);
+
+            // Now doing the same DROP COLUMN from the parent should fail with
+            // a different exception, but not a ConcurrentTableMutationException
+            // since the cell used to prevent concurrent modifications
+            // should be removed
+            try {
+                stmt.execute(tableDdl);
+                fail("Dropping a column from a parent that a child view depends"
+                        + " on should fail");
+            } catch (SQLException sqlE) {
+                assertEquals("Expected a different SQLException",
+                        CANNOT_MUTATE_TABLE.getErrorCode(),sqlE.getErrorCode());
+            }
+        }
+    }
+
+    /**
+     * Concurrently create a view which has its own new column and also try to
+     * add the same column to its parent.
+     * See PHOENIX-6191
+     */
+    @Test
+    public void testConcurrentViewCreationWithNewColParentColAddition()
+    throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl());
+                Statement stmt = conn.createStatement()) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_"
+                    + generateUniqueName();
+
+            // create base table
+            String tableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 INTEGER)"
+                    + tableDDLOptions;
+            stmt.execute(tableDdl);
+
+            latch1 = new CountDownLatch(1);
+            latch2 = new CountDownLatch(1);
+            ExecutorService executorService = newSingleThreadExecutor(
+                    new ThreadFactory() {
+                        @Override
+                        public Thread newThread(Runnable r) {
+                            Thread t = Executors.defaultThreadFactory()
+                                    .newThread(r);
+                            t.setDaemon(true);
+                            t.setPriority(Thread.MIN_PRIORITY);
+                            return t;
+                        }
+                    });
+
+            Future<Exception> future = executorService.submit(
+                    new CreateViewRunnable(fullTableName, fullViewName1));
+            // wait till the thread makes the rpc to add the column
+            boolean result = latch1.await(20, TimeUnit.MINUTES);
+            if (!result) {
+                fail("The create view rpc look too long");
+            }
+            tableDdl = "ALTER TABLE " + fullTableName + " ADD new_col INTEGER";
+            try {
+                // add the same column to the parent which is being introduced
+                // as a new column during view creation
+                stmt.execute(tableDdl);
+                fail("Adding a column to a base table should fail when a"
+                        + " child view is concurrently being created that"
+                        + " has that new column");
+            } catch (ConcurrentTableMutationException ignored) {
+            }
+            latch2.countDown();
+
+            Exception e = future.get();
+            assertNull(e);
+
+            // Now doing the same ADD COLUMN to the parent should fail with
+            // a different exception, but not a ConcurrentTableMutationException
+            // since the cell used to prevent concurrent modifications
+            // should be removed and since the column type is different
+            try {
+                stmt.execute(tableDdl);
+                fail("Adding a column to a parent that its child view already "
+                        + "contains should fail");
+            } catch (SQLException sqlE) {
+                assertEquals("Expected a different SQLException",
+                        CANNOT_MUTATE_TABLE.getErrorCode(),sqlE.getErrorCode());
+            }
+        }
+    }
+
+    public static class TestMetaDataRegionObserver
+            extends BaseMetaDataEndpointObserver {
+
+        @Override
+        public void preAlterTable(
+                final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+                final String tenantId, final String tableName,
+                final TableName physicalTableName,
+                final TableName parentPhysicalTableName, final PTableType type)
+                throws IOException {
+            processTable(tableName);
+        }
+
+        @Override
+        public void preCreateTable(
+                final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+                final String tenantId, final String tableName,
+                final TableName physicalTableName,
+                final TableName parentPhysicalTableName,
+                final PTableType tableType,
+                final Set<byte[]> familySet, final Set<TableName> indexes)
+                throws IOException {
+            processTable(tableName);
+        }
+
+        @Override
+        public void preDropTable(
+                final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+                final String tenantId, final String tableName,
+                final TableName physicalTableName,
+                final TableName parentPhysicalTableName,
+                final PTableType tableType,
+                final List<PTable> indexes) throws IOException {
+            processTable(tableName);
+        }
+
+        @Override
+        public void preCreateViewAddChildLink(
+                final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+                final String tableName) throws IOException {
+            if (throwExceptionInChildLinkPreHook) {
+                throw new IOException();
+            }
+            processTable(tableName);
+        }
+
+        private void processTable(String tableName)
+                throws DoNotRetryIOException {
+            if (tableName.equals(FAILED_VIEWNAME)) {
+                // throwing anything other than instances of IOException result
+                // in this coprocessor being unloaded
+                // DoNotRetryIOException tells HBase not to retry this mutation
+                // multiple times
+                throw new DoNotRetryIOException();
+            } else if (tableName.startsWith(SLOW_VIEWNAME_PREFIX) ||
+                    slowDownAddingChildLink) {
+                // simulate a slow write to SYSTEM.CATALOG or SYSTEM.CHILD_LINK
+                if (latch1 != null) {
+                    latch1.countDown();
+                }
+                if (latch2 != null) {
+                    try {
+                        // wait till the second task is complete before
+                        // completing the first task
+                        boolean result = latch2.await(20, TimeUnit.MINUTES);
+                        if (!result) {
+                            throw new RuntimeException("Second task took too "
+                                    + "long to complete");
+                        }
+                    } catch (InterruptedException ignored) {
+                    }
+                }
+            }
+        }
+    }
+
+    private static class CreateViewRunnable implements Callable<Exception> {
+        private final String fullTableName;
+        private final String fullViewName;
+
+        public CreateViewRunnable(String fullTableName, String fullViewName) {
+            this.fullTableName = fullTableName;
+            this.fullViewName = fullViewName;
+        }
+
+        @Override
+        public Exception call() throws Exception {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            try (Connection conn = DriverManager.getConnection(getUrl(), props);
+                    Statement stmt = conn.createStatement()) {
+                String ddl = "CREATE VIEW " + fullViewName
+                        + " (new_pk VARCHAR PRIMARY KEY, new_col VARCHAR)"
+                        + " AS SELECT * FROM " + fullTableName
+                        + " WHERE v1 = 5";
+                stmt.execute(ddl);
+            } catch (SQLException e) {
+                return e;
+            }
+            return null;
+        }
+    }
+
+    private static class AddColumnRunnable implements Callable<Exception> {
+        private final String fullViewName;
+        private final String tenantId;
+
+        public AddColumnRunnable(String fullViewName, String tenantId) {
+            this.fullViewName = fullViewName;
+            this.tenantId = tenantId;
+        }
+
+        @Override
+        public Exception call() throws Exception {
+            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
+            if (this.tenantId != null) {
+                props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB,
+                        this.tenantId);
+            }
+            try (Connection conn = DriverManager.getConnection(getUrl(), props);
+                    Statement stmt = conn.createStatement()) {
+                String ddl = "ALTER VIEW " + fullViewName + " ADD v3 CHAR(15)";
+                stmt.execute(ddl);
+            } catch (SQLException e) {
+                return e;
+            }
+            return null;
+        }
+    }
+}
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index d4305ea..95e78df 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -18,17 +18,16 @@
 package org.apache.phoenix.end2end;
 
 import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
-import static org.apache.phoenix.util.TestUtil.TEST_PROPERTIES;
+import static org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost
+        .PHOENIX_META_DATA_COPROCESSOR_CONF_KEY;
 import static org.apache.phoenix.util.TestUtil.analyzeTable;
 import static org.apache.phoenix.util.TestUtil.getAllSplits;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
-import java.io.IOException;
 import java.math.BigDecimal;
 import java.sql.Connection;
 import java.sql.DriverManager;
@@ -42,45 +41,25 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Properties;
-import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.curator.shaded.com.google.common.collect.Lists;
-import org.apache.hadoop.hbase.DoNotRetryIOException;
-import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.coprocessor.ObserverContext;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.phoenix.compile.QueryPlan;
-import org.apache.phoenix.coprocessor.BaseMetaDataEndpointObserver;
-import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost;
-import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
-import org.apache.phoenix.exception.PhoenixIOException;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.schema.ConcurrentTableMutationException;
 import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
-import org.apache.phoenix.schema.TableNotFoundException;
 import org.apache.phoenix.transaction.PhoenixTransactionProvider.Feature;
 import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
-import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
-import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -88,29 +67,25 @@ import org.junit.runners.Parameterized;
 import org.junit.runners.Parameterized.Parameters;
 
 import com.google.common.collect.Maps;
+
+/**
+ * Basic test suite for views
+ */
 @RunWith(Parameterized.class)
 public class ViewIT extends SplitSystemCatalogIT {
 
     protected String tableDDLOptions;
     protected String transactionProvider;
     protected boolean columnEncoded;
-    
-    private static final String FAILED_VIEWNAME =
-            SchemaUtil.getTableName(SCHEMA2, "FAILED_VIEW");
-    private static final String SLOW_VIEWNAME_PREFIX =
-            SchemaUtil.getTableName(SCHEMA2, "SLOW_VIEW");
-
-    private static volatile CountDownLatch latch1 = null;
-    private static volatile CountDownLatch latch2 = null;
-    private static volatile boolean throwExceptionInChildLinkPreHook = false;
-    private static volatile boolean slowDownAddingChildLink = false;
 
     public ViewIT(String transactionProvider, boolean columnEncoded) {
         StringBuilder optionBuilder = new StringBuilder();
         this.transactionProvider = transactionProvider;
         this.columnEncoded = columnEncoded;
         if (transactionProvider != null) {
-            optionBuilder.append(" TRANSACTION_PROVIDER='").append(transactionProvider).append("'");
+            optionBuilder.append(" TRANSACTION_PROVIDER='")
+                    .append(transactionProvider)
+                    .append("'");
         }
         if (!columnEncoded) {
             if (optionBuilder.length()!=0)
@@ -135,8 +110,9 @@ public class ViewIT extends SplitSystemCatalogIT {
         boolean splitSystemCatalog = (driver == null);
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put(QueryServices.PHOENIX_ACLS_ENABLED, "true");
-        serverProps.put(PhoenixMetaDataCoprocessorHost.PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
-            TestMetaDataRegionObserver.class.getName());
+        serverProps.put(PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
+            ViewConcurrencyAndFailureIT.TestMetaDataRegionObserver.class
+                    .getName());
         serverProps.put("hbase.coprocessor.abortonerror", "false");
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
                 ReadOnlyProps.EMPTY_PROPS);
@@ -148,87 +124,23 @@ public class ViewIT extends SplitSystemCatalogIT {
         }
     }
 
-    @After
-    public void cleanup() {
-        latch1 = null;
-        latch2 = null;
-        throwExceptionInChildLinkPreHook = false;
-        slowDownAddingChildLink = false;
-    }
-
-    public static class TestMetaDataRegionObserver extends BaseMetaDataEndpointObserver {
-        
-        @Override
-        public void preAlterTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
-                String tenantId, String tableName, TableName physicalTableName,
-                TableName parentPhysicalTableName, PTableType type) throws IOException {
-            processTable(tableName);
-        }
-        
-        @Override
-        public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
-                String tenantId, String tableName, TableName physicalTableName,
-                TableName parentPhysicalTableName, PTableType tableType,
-                Set<byte[]> familySet, Set<TableName> indexes) throws IOException {
-            processTable(tableName);
-        }
-
-        @Override
-        public void preDropTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
-                String tenantId, String tableName, TableName physicalTableName,
-                TableName parentPhysicalTableName, PTableType tableType,
-                List<PTable> indexes) throws IOException {
-            processTable(tableName);
-        }
-
-        @Override
-        public void preCreateViewAddChildLink(
-                final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
-                final String tableName) throws IOException {
-            if (throwExceptionInChildLinkPreHook) {
-                throw new IOException();
-            }
-            processTable(tableName);
-        }
-
-        private void processTable(String tableName) throws DoNotRetryIOException {
-            if (tableName.equals(FAILED_VIEWNAME)) {
-                // throwing anything other than instances of IOException result
-                // in this coprocessor being unloaded
-                // DoNotRetryIOException tells HBase not to retry this mutation
-                // multiple times
-                throw new DoNotRetryIOException();
-            } else if (tableName.startsWith(SLOW_VIEWNAME_PREFIX) || slowDownAddingChildLink) {
-                // simulate a slow write to SYSTEM.CATALOG or SYSTEM.CHILD_LINK
-                if (latch1 != null) {
-                    latch1.countDown();
-                }
-                if (latch2 != null) {
-                    try {
-                        // wait till the second task is complete before completing the first task
-                        boolean result = latch2.await(2, TimeUnit.MINUTES);
-                        if (!result) {
-                            throw new RuntimeException("Second task took took long to complete");
-                        }
-                    } catch (InterruptedException ignored) {
-                    }
-                }
-            }
-        }
-        
-    }
-    
     @Test
     public void testReadOnlyOnUpdatableView() throws Exception {
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-        String ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
+        String ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM "
+                + fullViewName1
                 + " WHERE k3 > 1 and k3 < 50";
-        testUpdatableView(fullTableName, fullViewName1, fullViewName2, ddl, null, tableDDLOptions);
+        testUpdatableView(fullTableName, fullViewName1, fullViewName2, ddl,
+                null, tableDDLOptions);
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            ResultSet rs = stmt.executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2);
+            ResultSet rs = stmt.executeQuery("SELECT k1, k2, k3 FROM "
+                    + fullViewName2);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(109, rs.getInt(2));
@@ -240,9 +152,11 @@ public class ViewIT extends SplitSystemCatalogIT {
             } catch (ReadOnlyTableException ignored) {
             }
 
-            stmt.execute("UPSERT INTO " + fullTableName + "(k1, k2,k3) VALUES(1, 122, 5)");
+            stmt.execute("UPSERT INTO " + fullTableName
+                    + "(k1, k2,k3) VALUES(1, 122, 5)");
             conn.commit();
-            rs = stmt.executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2 + " WHERE k2 >= 120");
+            rs = stmt.executeQuery("SELECT k1, k2, k3 FROM " + fullViewName2
+                    + " WHERE k2 >= 120");
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(122, rs.getInt(2));
@@ -256,26 +170,33 @@ public class ViewIT extends SplitSystemCatalogIT {
         try (Connection earlierCon = DriverManager.getConnection(getUrl());
                 Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            String schemaName = TestUtil.DEFAULT_SCHEMA_NAME + "_" + generateUniqueName();
-            String caseSensitiveTableName = "\"t_" + generateUniqueName() + "\"";
-            String fullTableName = SchemaUtil.getTableName(schemaName, caseSensitiveTableName);
+            String schemaName = TestUtil.DEFAULT_SCHEMA_NAME + "_"
+                    + generateUniqueName();
+            String caseSensitiveTableName = "\"t_" + generateUniqueName()
+                    + "\"";
+            String fullTableName = SchemaUtil.getTableName(schemaName,
+                    caseSensitiveTableName);
             String caseSensitiveViewName = "\"v_" + generateUniqueName() + "\"";
 
             String ddl = "CREATE TABLE " + fullTableName
-                    + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+                    + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
+                    + tableDDLOptions;
             stmt.execute(ddl);
-            ddl = "CREATE VIEW " + caseSensitiveViewName + " (v2 VARCHAR) AS SELECT * FROM "
+            ddl = "CREATE VIEW " + caseSensitiveViewName
+                    + " (v2 VARCHAR) AS SELECT * FROM "
                     + fullTableName + " WHERE k > 5";
             stmt.execute(ddl);
 
             try {
-                stmt.execute("UPSERT INTO " + caseSensitiveViewName + " VALUES(1)");
+                stmt.execute("UPSERT INTO " + caseSensitiveViewName
+                        + " VALUES(1)");
                 fail();
             } catch (ReadOnlyTableException ignored) {
             }
 
             for (int i = 0; i < 10; i++) {
-                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ")");
+                stmt.execute("UPSERT INTO " + fullTableName
+                        + " VALUES(" + i + ")");
             }
             conn.commit();
 
@@ -289,7 +210,8 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertEquals(4, count);
             count = 0;
             try (Statement earlierStmt = earlierCon.createStatement()) {
-                rs = earlierStmt.executeQuery("SELECT k FROM " + caseSensitiveViewName);
+                rs = earlierStmt.executeQuery("SELECT k FROM "
+                        + caseSensitiveViewName);
                 while (rs.next()) {
                     count++;
                     assertEquals(count + 5, rs.getInt(1));
@@ -300,18 +222,22 @@ public class ViewIT extends SplitSystemCatalogIT {
     }
     
     @Test
-    public void testReadOnlyViewWithCaseSensitiveColumnNames() throws Exception {
+    public void testReadOnlyViewWithCaseSensitiveColumnNames()
+            throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
             String ddl = "CREATE TABLE " + fullTableName
-                    + " (\"k\" INTEGER NOT NULL PRIMARY KEY, \"v1\" INTEGER, \"a\".v2 VARCHAR)"
+                    + " (\"k\" INTEGER NOT NULL PRIMARY KEY, \"v1\" INTEGER, "
+                    + "\"a\".v2 VARCHAR)"
                     + tableDDLOptions;
 
             stmt.execute(ddl);
-            ddl = "CREATE VIEW " + viewName + " (v VARCHAR) AS SELECT * FROM " + fullTableName
-                    + " WHERE \"k\" > 5 and \"v1\" > 1";
+            ddl = "CREATE VIEW " + viewName + " (v VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE \"k\" > 5 and \"v1\" > 1";
             stmt.execute(ddl);
 
             try {
@@ -321,13 +247,14 @@ public class ViewIT extends SplitSystemCatalogIT {
 
             }
             for (int i = 0; i < 10; i++) {
-                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ", "
-                        + (i + 10) + ",'A')");
+                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + i
+                        + ", " + (i + 10) + ",'A')");
             }
             conn.commit();
 
             int count = 0;
-            ResultSet rs = stmt.executeQuery("SELECT \"k\", \"v1\",\"a\".v2 FROM " + viewName);
+            ResultSet rs = stmt.executeQuery(
+                    "SELECT \"k\", \"v1\",\"a\".v2 FROM " + viewName);
             while (rs.next()) {
                 count++;
                 assertEquals(count + 5, rs.getInt(1));
@@ -340,13 +267,17 @@ public class ViewIT extends SplitSystemCatalogIT {
     public void testViewWithCurrentDate() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
 
-            String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, "
+            String ddl = "CREATE TABLE " + fullTableName
+                    + " (k INTEGER NOT NULL PRIMARY KEY, "
                     + "v1 INTEGER, v2 DATE)" + tableDDLOptions;
             stmt.execute(ddl);
-            ddl = "CREATE VIEW " + viewName + " (v VARCHAR) AS SELECT * FROM " + fullTableName
+            ddl = "CREATE VIEW " + viewName + " (v VARCHAR) AS SELECT * FROM "
+                    + fullTableName
                     + " WHERE v2 > CURRENT_DATE()-5 AND v2 > DATE '2010-01-01'";
             stmt.execute(ddl);
 
@@ -357,8 +288,8 @@ public class ViewIT extends SplitSystemCatalogIT {
 
             }
             for (int i = 0; i < 10; i++) {
-                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + i + ", " + (i + 10)
-                        + ",CURRENT_DATE()-" + i + ")");
+                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + i
+                        + ", " + (i + 10) + ",CURRENT_DATE()-" + i + ")");
             }
             conn.commit();
 
@@ -381,7 +312,8 @@ public class ViewIT extends SplitSystemCatalogIT {
     public void testViewUsesTableLocalIndex() throws Exception {
         if (transactionProvider == null ||
                 !TransactionFactory.getTransactionProvider(
-                        TransactionFactory.Provider.valueOf(transactionProvider))
+                        TransactionFactory.Provider.valueOf(
+                                transactionProvider))
                         .isUnsupported(Feature.ALLOW_LOCAL_INDEX)) {
             testViewUsesTableIndex(true);
         }
@@ -389,31 +321,41 @@ public class ViewIT extends SplitSystemCatalogIT {
 
     private void testViewUsesTableIndex(boolean localIndex) throws Exception {
         ResultSet rs;
-        // Use unique name for table with local index as otherwise we run into issues
-        // when we attempt to drop the table (with the drop metadata option set to false
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName()) +
+        // Use unique name for table with local index as otherwise we run
+        // into issues when we attempt to drop the table (with the drop
+        // metadata option set to false
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName()) +
                 (localIndex ? "_WITH_LI" : "_WITHOUT_LI");
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()){
-            String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, "
+            String ddl = "CREATE TABLE " + fullTableName
+                    + " (k1 INTEGER NOT NULL, "
                     + "k2 INTEGER NOT NULL, k3 DECIMAL, s1 VARCHAR, s2 VARCHAR"
-                    + " CONSTRAINT pk PRIMARY KEY (k1, k2, k3))" + tableDDLOptions;
+                    + " CONSTRAINT pk PRIMARY KEY (k1, k2, k3))"
+                    + tableDDLOptions;
             stmt.execute(ddl);
             String indexName1 = "I_" + generateUniqueName();
-            String fullIndexName1 = SchemaUtil.getTableName(SCHEMA1, indexName1);
-            stmt.execute("CREATE " + (localIndex ? "LOCAL " : "") + " INDEX " + indexName1 + " ON "
+            String fullIndexName1 = SchemaUtil.getTableName(SCHEMA1,
+                    indexName1);
+            stmt.execute("CREATE " + (localIndex ? "LOCAL " : "") + " INDEX "
+                    + indexName1 + " ON "
                     + fullTableName + "(k3, k2) INCLUDE(s1, s2)");
             String indexName2 = "I_" + generateUniqueName();
-            stmt.execute("CREATE INDEX " + indexName2 + " ON " + fullTableName + "(k3, k2, s2)");
+            stmt.execute("CREATE INDEX " + indexName2 + " ON " + fullTableName
+                    + "(k3, k2, s2)");
 
-            String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-            ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName
-                    + " WHERE s1 = 'foo'";
+            String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
+            ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM "
+                    + fullTableName + " WHERE s1 = 'foo'";
             stmt.execute(ddl);
             String[] s1Values = { "foo", "bar" };
             for (int i = 0; i < 10; i++) {
-                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + (i % 4) + "," + (i + 100)
-                        + "," + (i > 5 ? 2 : 1) + ",'" + s1Values[i % 2] + "','bas')");
+                stmt.execute("UPSERT INTO " + fullTableName + " VALUES("
+                        + (i % 4) + "," + (i + 100)
+                        + "," + (i > 5 ? 2 : 1) + ",'" + s1Values[i % 2]
+                        + "','bas')");
             }
             conn.commit();
 
@@ -423,9 +365,11 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertFalse(rs.next());
 
             String viewIndexName = "I_" + generateUniqueName();
-            stmt.execute("CREATE INDEX " + viewIndexName + " on " + fullViewName + "(k2)");
+            stmt.execute("CREATE INDEX " + viewIndexName + " on "
+                    + fullViewName + "(k2)");
 
-            String query = "SELECT k2 FROM " + fullViewName + " WHERE k2 IN (100,109) AND k3 IN"
+            String query = "SELECT k2 FROM " + fullViewName
+                    + " WHERE k2 IN (100,109) AND k3 IN"
                     + " (1,2) AND s2='bas'";
             rs = stmt.executeQuery(query);
             assertTrue(rs.next());
@@ -433,17 +377,21 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertFalse(rs.next());
             rs = stmt.executeQuery("EXPLAIN " + query);
             String queryPlan = QueryUtil.getExplainPlan(rs);
-            // Assert that in either case (local & global) that index from physical table used
-            // for query on view.
+            // Assert that in either case (local & global) that index from
+            // physical table used for query on view.
             if (localIndex) {
-                assertEquals("CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER " + fullTableName
+                assertEquals("CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER "
+                        + fullTableName
                         + " [1,1,100] - [1,2,109]\n"
-                        + "    SERVER FILTER BY (\"S2\" = 'bas' AND \"S1\" = 'foo')\n"
+                        + "    SERVER FILTER BY (\"S2\" = 'bas' "
+                        + "AND \"S1\" = 'foo')\n"
                         + "CLIENT MERGE SORT", queryPlan);
             } else {
-                assertEquals("CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER " + fullIndexName1
+                assertEquals("CLIENT PARALLEL 1-WAY SKIP SCAN ON 4 KEYS OVER "
+                        + fullIndexName1
                         + " [1,100] - [2,109]\n"
-                        + "    SERVER FILTER BY (\"S2\" = 'bas' AND \"S1\" = 'foo')", queryPlan);
+                        + "    SERVER FILTER BY (\"S2\" = 'bas' "
+                        + "AND \"S1\" = 'foo')", queryPlan);
             }
         }
     }
@@ -458,35 +406,45 @@ public class ViewIT extends SplitSystemCatalogIT {
         testCreateChildViewWithBaseTableIndex(false);
     }
 
-    public void testCreateChildViewWithBaseTableIndex(boolean localIndex) throws Exception {
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+    public void testCreateChildViewWithBaseTableIndex(boolean localIndex)
+            throws Exception {
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
         String indexName = "I_" + generateUniqueName();
-        String fullChildViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullChildViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
             String sql = "CREATE TABLE " + fullTableName
-                    + " (ID INTEGER NOT NULL PRIMARY KEY, HOST VARCHAR(10), FLAG BOOLEAN)";
+                    + " (ID INTEGER NOT NULL PRIMARY KEY, HOST VARCHAR(10),"
+                    + " FLAG BOOLEAN)";
             stmt.execute(sql);
             sql = "CREATE VIEW " + fullViewName
-                    + " (COL1 INTEGER, COL2 INTEGER, COL3 INTEGER, COL4 INTEGER) AS SELECT * FROM "
+                    + " (COL1 INTEGER, COL2 INTEGER, COL3 INTEGER,"
+                    + " COL4 INTEGER) AS SELECT * FROM "
                     + fullTableName + " WHERE ID > 5";
             stmt.execute(sql);
-            sql = "CREATE " + (localIndex ? "LOCAL " : "") + " INDEX " + indexName + " ON "
-                    + fullTableName + "(HOST)";
+            sql = "CREATE " + (localIndex ? "LOCAL " : "") + " INDEX "
+                    + indexName + " ON " + fullTableName + "(HOST)";
             stmt.execute(sql);
-            sql = "CREATE VIEW " + fullChildViewName + " AS SELECT * FROM " + fullViewName
-                    + " WHERE COL1 > 2";
+            sql = "CREATE VIEW " + fullChildViewName + " AS SELECT * FROM "
+                    + fullViewName + " WHERE COL1 > 2";
             stmt.execute(sql);
             // Sanity upserts in baseTable, view, child view
-            stmt.executeUpdate("upsert into " + fullTableName + " values (1, 'host1', TRUE)");
-            stmt.executeUpdate("upsert into " + fullTableName + " values (5, 'host5', FALSE)");
-            stmt.executeUpdate("upsert into " + fullTableName + " values (7, 'host7', TRUE)");
+            stmt.executeUpdate("upsert into " + fullTableName
+                    + " values (1, 'host1', TRUE)");
+            stmt.executeUpdate("upsert into " + fullTableName
+                    + " values (5, 'host5', FALSE)");
+            stmt.executeUpdate("upsert into " + fullTableName
+                    + " values (7, 'host7', TRUE)");
             conn.commit();
             // View is not updateable
             try {
                 stmt.executeUpdate("upsert into " + fullViewName
-                        + " (ID, HOST, FLAG, COL1) values (7, 'host7', TRUE, 1)");
+                        + " (ID, HOST, FLAG, COL1)"
+                        + " values (7, 'host7', TRUE, 1)");
                 fail();
             } catch (Exception ignore) {
             }
@@ -496,7 +454,8 @@ public class ViewIT extends SplitSystemCatalogIT {
             table = PhoenixRuntime.getTable(conn, fullChildViewName);
             assertEquals(0, table.getIndexes().size());
 
-            ResultSet rs = stmt.executeQuery("select count(*) from " + fullTableName);
+            ResultSet rs = stmt.executeQuery("select count(*) from "
+                    + fullTableName);
             assertTrue(rs.next());
             assertEquals(3, rs.getInt(1));
 
@@ -510,14 +469,17 @@ public class ViewIT extends SplitSystemCatalogIT {
     public void testCreateViewDefinesPKColumn() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
 
             String ddl = "CREATE TABLE " + fullTableName
                     + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, "
                     + "CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
             stmt.execute(ddl);
-            ddl = "CREATE VIEW " + fullViewName + "(v2 VARCHAR, k3 VARCHAR PRIMARY KEY) "
+            ddl = "CREATE VIEW " + fullViewName + "(v2 VARCHAR, k3 VARCHAR "
+                    + "PRIMARY KEY) "
                     + "AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
             stmt.execute(ddl);
 
@@ -528,7 +490,8 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertPKs(rs, new String[] { "K1", "K2", "K3" });
 
             // sanity check upserts into base table and view
-            stmt.executeUpdate("upsert into " + fullTableName + " (k1, k2, v1) values (1, 1, 1)");
+            stmt.executeUpdate("upsert into " + fullTableName + " (k1, k2, v1)"
+                    + " values (1, 1, 1)");
             stmt.executeUpdate("upsert into " + fullViewName
                     + " (k1, k2, k3, v2) values (1, 1, 'abc', 'def')");
             conn.commit();
@@ -548,17 +511,22 @@ public class ViewIT extends SplitSystemCatalogIT {
     @Test
     public void testQueryViewStatementOptimization() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
+            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                    generateUniqueName());
             String sql = "CREATE TABLE " + fullTableName
                     + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, "
                     + "CONSTRAINT pk PRIMARY KEY (k1, k2))" + tableDDLOptions;
             try (Statement stmt = conn.createStatement()) {
                 stmt.execute(sql);
-                sql = "CREATE VIEW " + fullViewName1 + "  AS SELECT * FROM " + fullTableName;
+                sql = "CREATE VIEW " + fullViewName1 + "  AS SELECT * FROM "
+                        + fullTableName;
                 stmt.execute(sql);
-                sql = "CREATE VIEW " + fullViewName2 + "  AS SELECT * FROM " + fullTableName
+                sql = "CREATE VIEW " + fullViewName2 + "  AS SELECT * FROM "
+                        + fullTableName
                         + " WHERE k1 = 1.0";
                 stmt.execute(sql);
             }
@@ -566,17 +534,20 @@ public class ViewIT extends SplitSystemCatalogIT {
             QueryPlan plan;
             try (PreparedStatement stmt = conn.prepareStatement(sql)) {
                 plan = PhoenixRuntime.getOptimizedQueryPlan(stmt);
-                assertEquals(0, plan.getOrderBy().getOrderByExpressions().size());
+                assertEquals(0, plan.getOrderBy().getOrderByExpressions()
+                        .size());
             }
             sql = "SELECT * FROM " + fullViewName2 + " order by k1, k2";
             try (PreparedStatement stmt = conn.prepareStatement(sql)) {
                 plan = PhoenixRuntime.getOptimizedQueryPlan(stmt);
-                assertEquals(0, plan.getOrderBy().getOrderByExpressions().size());
+                assertEquals(0, plan.getOrderBy().getOrderByExpressions()
+                        .size());
             }
         }
     }
 
-    private void assertPKs(ResultSet rs, String[] expectedPKs) throws SQLException {
+    private void assertPKs(ResultSet rs, String[] expectedPKs)
+            throws SQLException {
         List<String> pkCols = newArrayListWithExpectedSize(expectedPKs.length);
         while (rs.next()) {
             pkCols.add(rs.getString("COLUMN_NAME"));
@@ -588,54 +559,75 @@ public class ViewIT extends SplitSystemCatalogIT {
     @Test
     public void testCompositeDescPK() throws Exception {
         Properties props = new Properties();
-        try (Connection globalConn = DriverManager.getConnection(getUrl(), props)) {
-            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String viewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-            String viewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-            String viewName3 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String viewName4 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        try (Connection globalConn = DriverManager.getConnection(getUrl(),
+                props)) {
+            String tableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String viewName1 = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
+            String viewName2 = SchemaUtil.getTableName(SCHEMA3,
+                    generateUniqueName());
+            String viewName3 = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String viewName4 = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
 
             String myTableDDLOptions = tableDDLOptions;
             if (myTableDDLOptions.length() != 0) myTableDDLOptions += ",";
-            myTableDDLOptions += "VERSIONS=1, MULTI_TENANT=true, IMMUTABLE_ROWS=TRUE, "
-                    + "REPLICATION_SCOPE=1";
+            myTableDDLOptions += "VERSIONS=1, MULTI_TENANT=true, "
+                    + "IMMUTABLE_ROWS=TRUE, REPLICATION_SCOPE=1";
 
             // create global base table
             try (Statement stmt = globalConn.createStatement()) {
                 stmt.execute("CREATE TABLE " + tableName
-                        + " (TENANT_ID CHAR(15) NOT NULL, KEY_PREFIX CHAR(3) NOT NULL, "
-                        + "CREATED_DATE DATE, CREATED_BY CHAR(15), SYSTEM_MODSTAMP DATE CONSTRAINT "
-                        + "PK PRIMARY KEY (TENANT_ID, KEY_PREFIX)) " + myTableDDLOptions);
+                        + " (TENANT_ID CHAR(15) NOT NULL, KEY_PREFIX CHAR(3)"
+                        + " NOT NULL, "
+                        + "CREATED_DATE DATE, CREATED_BY CHAR(15), "
+                        + "SYSTEM_MODSTAMP DATE CONSTRAINT "
+                        + "PK PRIMARY KEY (TENANT_ID, KEY_PREFIX)) "
+                        + myTableDDLOptions);
             }
 
             String tenantId = "tenantId";
             Properties tenantProps = new Properties();
             tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
             // create a tenant specific view
-            try (Connection tenantConn = DriverManager.getConnection(getUrl(), tenantProps);
+            try (Connection tenantConn = DriverManager.getConnection(getUrl(),
+                    tenantProps);
                     Statement tenantStmt = tenantConn.createStatement()) {
                 // create various tenant specific views
-                // view with composite PK with multiple PK values of VARCHAR values DESC
+                // view with composite PK with multiple PK values of VARCHAR
+                // values DESC
                 tenantStmt.execute("CREATE VIEW " + viewName1
-                        + " (pk1 VARCHAR(10) NOT NULL, pk2 VARCHAR(10) NOT NULL, col1 DATE,"
-                        + " col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC)) "
-                        + "AS SELECT * FROM " + tableName + " WHERE KEY_PREFIX = 'abc' ");
+                        + " (pk1 VARCHAR(10) NOT NULL, pk2 VARCHAR(10) "
+                        + "NOT NULL, col1 DATE,"
+                        + " col3 DECIMAL CONSTRAINT PK PRIMARY KEY "
+                        + "(pk1 DESC, pk2 DESC)) "
+                        + "AS SELECT * FROM " + tableName
+                        + " WHERE KEY_PREFIX = 'abc' ");
                 // view with composite PK with single pk value DESC
                 tenantStmt.execute("CREATE VIEW " + viewName2
-                        + " (pk1 VARCHAR(10) NOT NULL, pk2 VARCHAR(10) NOT NULL, col1 DATE,"
-                        + " col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC)) "
-                        + "AS SELECT * FROM " + tableName + " WHERE KEY_PREFIX = 'abc' ");
+                        + " (pk1 VARCHAR(10) NOT NULL, pk2 VARCHAR(10) "
+                        + "NOT NULL, col1 DATE,"
+                        + " col3 DECIMAL CONSTRAINT PK PRIMARY KEY "
+                        + "(pk1 DESC, pk2 DESC)) "
+                        + "AS SELECT * FROM " + tableName
+                        + " WHERE KEY_PREFIX = 'abc' ");
 
                 // view with composite PK with multiple Date PK values DESC
                 tenantStmt.execute("CREATE VIEW " + viewName3
-                        + " (pk1 DATE(10) NOT NULL, pk2 DATE(10) NOT NULL, "
-                        + "col1 VARCHAR(10), col3 DECIMAL CONSTRAINT PK PRIMARY KEY "
+                        + " (pk1 DATE(10) NOT NULL, pk2 DATE(10) "
+                        + "NOT NULL, "
+                        + "col1 VARCHAR(10), col3 DECIMAL CONSTRAINT PK "
+                        + "PRIMARY KEY "
                         + "(pk1 DESC, pk2 DESC)) AS SELECT * FROM "
                         + tableName + " WHERE KEY_PREFIX = 'ab3' ");
                 
                 tenantStmt.execute("CREATE VIEW " + viewName4
-                        + " (pk1 DATE(10) NOT NULL, pk2 DECIMAL NOT NULL, pk3 VARCHAR(10) NOT NULL,"
-                        + " col3 DECIMAL CONSTRAINT PK PRIMARY KEY (pk1 DESC, pk2 DESC, pk3 DESC)) "
+                        + " (pk1 DATE(10) NOT NULL, pk2 DECIMAL NOT NULL,"
+                        + " pk3 VARCHAR(10) NOT NULL,"
+                        + " col3 DECIMAL CONSTRAINT PK PRIMARY KEY "
+                        + "(pk1 DESC, pk2 DESC, pk3 DESC)) "
                         + "AS SELECT * FROM "
                         + tableName + " WHERE KEY_PREFIX = 'ab4' ");
 
@@ -652,72 +644,105 @@ public class ViewIT extends SplitSystemCatalogIT {
                 validate(viewName2, tenantConn, whereClauses, expectedArray);
 
                 tenantStmt.execute("UPSERT INTO " + viewName3
-                        + " (pk1, pk2, col1, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
-                        + "'yyyy-MM-dd HH:mm:ss'), TO_DATE('2017-10-16 21:00:00', "
+                        + " (pk1, pk2, col1, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
+                        + "'yyyy-MM-dd HH:mm:ss'), "
+                        + "TO_DATE('2017-10-16 21:00:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 'txt1', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName3
-                        + " (pk1, pk2, col1, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
-                        + "'yyyy-MM-dd HH:mm:ss'), TO_DATE('2017-10-16 21:01:00', "
+                        + " (pk1, pk2, col1, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
+                        + "'yyyy-MM-dd HH:mm:ss'), "
+                        + "TO_DATE('2017-10-16 21:01:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 'txt1', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName3
-                        + " (pk1, pk2, col1, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
-                        + "'yyyy-MM-dd HH:mm:ss'), TO_DATE('2017-10-16 21:02:00', "
+                        + " (pk1, pk2, col1, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
+                        + "'yyyy-MM-dd HH:mm:ss'), "
+                        + "TO_DATE('2017-10-16 21:02:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 'txt1', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName3
-                        + " (pk1, pk2, col1, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
-                        + "'yyyy-MM-dd HH:mm:ss'), TO_DATE('2017-10-16 21:03:00', "
+                        + " (pk1, pk2, col1, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
+                        + "'yyyy-MM-dd HH:mm:ss'), "
+                        + "TO_DATE('2017-10-16 21:03:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 'txt1', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName3
-                        + " (pk1, pk2, col1, col3) VALUES (TO_DATE('2017-10-16 23:00:00', "
-                        + "'yyyy-MM-dd HH:mm:ss'), TO_DATE('2017-10-16 21:04:00', "
+                        + " (pk1, pk2, col1, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 23:00:00', "
+                        + "'yyyy-MM-dd HH:mm:ss'), "
+                        + "TO_DATE('2017-10-16 21:04:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 'txt1', 10)");
                 tenantConn.commit();
 
                 String[] view3WhereClauses = new String[] {
-                        "pk1 = TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')", "",
-                        "pk1 >= TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 <= TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 > TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 < TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')" };
-                validate(viewName3, tenantConn, view3WhereClauses, expectedArray);
+                        "pk1 = TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')", "",
+                        "pk1 >= TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 <= TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 > TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 < TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')" };
+                validate(viewName3, tenantConn, view3WhereClauses,
+                        expectedArray);
 
                 tenantStmt.execute("UPSERT INTO " + viewName4
-                        + " (pk1, pk2, pk3, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
+                        + " (pk1, pk2, pk3, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 1, 'txt1', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName4
-                        + " (pk1, pk2, pk3, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
+                        + " (pk1, pk2, pk3, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 2, 'txt2', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName4
-                        + " (pk1, pk2, pk3, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
+                        + " (pk1, pk2, pk3, col3) VALUES"
+                        + " (TO_DATE('2017-10-16 22:00:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 3, 'txt3', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName4
-                        + " (pk1, pk2, pk3, col3) VALUES (TO_DATE('2017-10-16 22:00:00', "
+                        + " (pk1, pk2, pk3, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 22:00:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 4, 'txt4', 10)");
                 tenantStmt.execute("UPSERT INTO " + viewName4
-                        + " (pk1, pk2, pk3, col3) VALUES (TO_DATE('2017-10-16 23:00:00', "
+                        + " (pk1, pk2, pk3, col3) VALUES "
+                        + "(TO_DATE('2017-10-16 23:00:00', "
                         + "'yyyy-MM-dd HH:mm:ss'), 1, 'txt1', 10)");
                 tenantConn.commit();
 
                 String[] view4WhereClauses = new String[] {
-                        "pk1 = TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 = TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss') AND pk2 = 2",
-                        "pk1 = TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss') AND pk2 > 2",
-                        "", "pk1 >= TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 <= TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 > TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')",
-                        "pk1 < TO_DATE('2017-10-16 22:00:00', 'yyyy-MM-dd HH:mm:ss')" };
-                long[] view4ExpectedArray = new long[] { 4, 1, 2, 5, 5, 4, 1, 0 };
-                validate(viewName4, tenantConn, view4WhereClauses, view4ExpectedArray);
-            }
-        }
-    }
-
-    private void validate(String viewName, Connection tenantConn, String[] whereClauseArray,
+                        "pk1 = TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 = TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss') AND pk2 = 2",
+                        "pk1 = TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss') AND pk2 > 2",
+                        "", "pk1 >= TO_DATE('2017-10-16 22:00:00', "
+                        + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 <= TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 > TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')",
+                        "pk1 < TO_DATE('2017-10-16 22:00:00', "
+                                + "'yyyy-MM-dd HH:mm:ss')" };
+                long[] view4ExpectedArray = new long[]
+                        { 4, 1, 2, 5, 5, 4, 1, 0 };
+                validate(viewName4, tenantConn, view4WhereClauses,
+                        view4ExpectedArray);
+            }
+        }
+    }
+
+    private void validate(String viewName, Connection tenantConn,
+            String[] whereClauseArray,
             long[] expectedArray) throws SQLException {
         for (int i = 0; i < whereClauseArray.length; ++i) {
-            String where = !whereClauseArray[i].isEmpty() ? (" WHERE " + whereClauseArray[i]) : "";
+            String where = !whereClauseArray[i].isEmpty() ?
+                    (" WHERE " + whereClauseArray[i]) : "";
             try (Statement stmt = tenantConn.createStatement()) {
-                ResultSet rs = stmt.executeQuery("SELECT count(*) FROM " + viewName + where);
+                ResultSet rs = stmt.executeQuery("SELECT count(*) FROM "
+                        + viewName + where);
                 assertTrue(rs.next());
                 assertEquals(expectedArray[i], rs.getLong(1));
                 assertFalse(rs.next());
@@ -725,30 +750,39 @@ public class ViewIT extends SplitSystemCatalogIT {
         }
     }
 
-    private void upsertRows(String viewName, Connection tenantConn) throws SQLException {
+    private void upsertRows(String viewName, Connection tenantConn)
+            throws SQLException {
         try (Statement stmt = tenantConn.createStatement()) {
-            stmt.execute("UPSERT INTO " + viewName + " (pk1, pk2, col1, col3) VALUES "
+            stmt.execute("UPSERT INTO " + viewName
+                    + " (pk1, pk2, col1, col3) VALUES "
                     + "('testa', 'testb', TO_DATE('2017-10-16 22:00:00', "
                     + "'yyyy-MM-dd HH:mm:ss'), 10)");
-            stmt.execute("UPSERT INTO " + viewName + " (pk1, pk2, col1, col3) VALUES "
+            stmt.execute("UPSERT INTO " + viewName
+                    + " (pk1, pk2, col1, col3) VALUES "
                     + "('testa', 'testc', TO_DATE('2017-10-16 22:00:00', "
                     + "'yyyy-MM-dd HH:mm:ss'), 10)");
-            stmt.execute("UPSERT INTO " + viewName + " (pk1, pk2, col1, col3) "
-                    + "VALUES ('testa', 'testd', TO_DATE('2017-10-16 22:00:00', "
+            stmt.execute("UPSERT INTO " + viewName
+                    + " (pk1, pk2, col1, col3) "
+                    + "VALUES ('testa', 'testd', "
+                    + "TO_DATE('2017-10-16 22:00:00', "
                     + "'yyyy-MM-dd HH:mm:ss'), 10)");
-            stmt.execute("UPSERT INTO " + viewName + " (pk1, pk2, col1, col3) "
-                    + "VALUES ('testa', 'teste', TO_DATE('2017-10-16 22:00:00', "
+            stmt.execute("UPSERT INTO " + viewName
+                    + " (pk1, pk2, col1, col3) "
+                    + "VALUES ('testa', 'teste', "
+                    + "TO_DATE('2017-10-16 22:00:00', "
                     + "'yyyy-MM-dd HH:mm:ss'), 10)");
-            stmt.execute("UPSERT INTO " + viewName + " (pk1, pk2, col1, col3) "
-                    + "VALUES ('testb', 'testa', TO_DATE('2017-10-16 22:00:00', "
+            stmt.execute("UPSERT INTO " + viewName
+                    + " (pk1, pk2, col1, col3) "
+                    + "VALUES ('testb', 'testa', "
+                    + "TO_DATE('2017-10-16 22:00:00', "
                     + "'yyyy-MM-dd HH:mm:ss'), 10)");
             tenantConn.commit();
         }
     }
     
-    public static void testUpdatableView(String fullTableName, String fullViewName,
-            String fullChildViewName, String childViewDDL, Integer saltBuckets,
-            String tableDDLOptions) throws Exception {
+    public static void testUpdatableView(String fullTableName,
+            String fullViewName, String fullChildViewName, String childViewDDL,
+            Integer saltBuckets, String tableDDLOptions) throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
             if (saltBuckets != null) {
@@ -756,20 +790,25 @@ public class ViewIT extends SplitSystemCatalogIT {
                 tableDDLOptions += (" SALT_BUCKETS=" + saltBuckets);
             }
             String ddl = "CREATE TABLE " + fullTableName
-                    + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL, s VARCHAR "
-                    + "CONSTRAINT pk PRIMARY KEY (k1, k2, k3))" + tableDDLOptions;
+                    + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, k3 DECIMAL,"
+                    + " s VARCHAR "
+                    + "CONSTRAINT pk PRIMARY KEY (k1, k2, k3))"
+                    + tableDDLOptions;
             stmt.execute(ddl);
-            ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName
+            ddl = "CREATE VIEW " + fullViewName + " AS SELECT * FROM "
+                    + fullTableName
                     + " WHERE k1 = 1";
             stmt.execute(ddl);
-            ArrayList<String> splitPoints = Lists.newArrayList(fullTableName, fullViewName);
+            ArrayList<String> splitPoints = Lists.newArrayList(fullTableName,
+                    fullViewName);
             if (fullChildViewName != null) {
                 stmt.execute(childViewDDL);
                 splitPoints.add(fullChildViewName);
             }
 
             for (int i = 0; i < 10; i++) {
-                stmt.execute("UPSERT INTO " + fullTableName + " VALUES(" + (i % 4) + ","
+                stmt.execute("UPSERT INTO " + fullTableName
+                        + " VALUES(" + (i % 4) + ","
                         + (i + 100) + "," + (i > 5 ? 2 : 1) + ")");
             }
             conn.commit();
@@ -796,11 +835,14 @@ public class ViewIT extends SplitSystemCatalogIT {
             assertEquals(2, rs.getInt(3));
             assertFalse(rs.next());
 
-            stmt.execute("UPSERT INTO " + fullViewName + "(k2,S,k3) VALUES(120,'foo',50.0)");
-            stmt.execute("UPSERT INTO " + fullViewName + "(k2,S,k3) VALUES(121,'bar',51.0)");
+            stmt.execute("UPSERT INTO " + fullViewName
+                    + "(k2,S,k3) VALUES(120,'foo',50.0)");
+            stmt.execute("UPSERT INTO " + fullViewName
+                    + "(k2,S,k3) VALUES(121,'bar',51.0)");
             conn.commit();
 
-            rs = stmt.executeQuery("SELECT k1, k2 FROM " + fullViewName + " WHERE k2 >= 120");
+            rs = stmt.executeQuery("SELECT k1, k2 FROM " + fullViewName
+                    + " WHERE k2 >= 120");
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(120, rs.getInt(2));
@@ -811,20 +853,25 @@ public class ViewIT extends SplitSystemCatalogIT {
         }
     }
 
-    public static Pair<String, Scan> testUpdatableViewIndex(String fullTableName, Integer saltBuckets,
+    public static Pair<String, Scan> testUpdatableViewIndex(
+            String fullTableName, Integer saltBuckets,
             boolean localIndex, String viewName) throws Exception {
         ResultSet rs;
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
             String viewIndexName1 = "I_" + generateUniqueName();
-            String viewIndexPhysicalName = MetaDataUtil.getViewIndexPhysicalName(fullTableName);
+            String viewIndexPhysicalName = MetaDataUtil.
+                    getViewIndexPhysicalName(fullTableName);
             if (localIndex) {
-                stmt.execute("CREATE LOCAL INDEX " + viewIndexName1 + " on " + viewName + "(k3)");
+                stmt.execute("CREATE LOCAL INDEX " + viewIndexName1 + " on "
+                        + viewName + "(k3)");
             } else {
                 stmt.execute(
-                        "CREATE INDEX " + viewIndexName1 + " on " + viewName + "(k3) include (s)");
+                        "CREATE INDEX " + viewIndexName1 + " on " + viewName
+                                + "(k3) include (s)");
             }
-            stmt.execute("UPSERT INTO " + viewName + "(k2,S,k3) " + "VALUES(120,'foo',50.0)");
+            stmt.execute("UPSERT INTO " + viewName + "(k2,S,k3) "
+                    + "VALUES(120,'foo',50.0)");
             conn.commit();
 
             analyzeTable(conn, viewName);
@@ -833,35 +880,45 @@ public class ViewIT extends SplitSystemCatalogIT {
             // buckets
             assertEquals(saltBuckets == null ? 6 : 8, splits.size());
 
-            String query = "SELECT k1, k2, k3, s FROM " + viewName + " WHERE k3 = 51.0";
+            String query = "SELECT k1, k2, k3, s FROM " + viewName
+                    + " WHERE k3 = 51.0";
             rs = stmt.executeQuery(query);
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(121, rs.getInt(2));
-            assertEquals(0, BigDecimal.valueOf(51.0).compareTo(rs.getBigDecimal(3)));
+            assertEquals(0, BigDecimal.valueOf(51.0).compareTo(
+                    rs.getBigDecimal(3)));
             assertEquals("bar", rs.getString(4));
             assertFalse(rs.next());
             rs = stmt.executeQuery("EXPLAIN " + query);
             String queryPlan = QueryUtil.getExplainPlan(rs);
             if (localIndex) {
-                assertEquals("CLIENT PARALLEL " + (saltBuckets == null ? 1 : saltBuckets)
+                assertEquals("CLIENT PARALLEL "
+                        + (saltBuckets == null ? 1 : saltBuckets)
                         + "-WAY RANGE SCAN OVER " + fullTableName + " [1,51]\n"
-                        + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT", queryPlan);
+                        + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                        + "CLIENT MERGE SORT", queryPlan);
             } else {
                 assertEquals(saltBuckets == null ?
-                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName
+                        "CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
+                                + viewIndexPhysicalName
                                 + " [" + Short.MIN_VALUE + ",51]" :
-                        "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER "
+                        "CLIENT PARALLEL " + saltBuckets
+                                + "-WAY RANGE SCAN OVER "
                                 + viewIndexPhysicalName + " [0,"
-                                + Short.MIN_VALUE + ",51] - [" + (saltBuckets - 1) + ","
-                                + Short.MIN_VALUE + ",51]\nCLIENT MERGE SORT", queryPlan);
+                                + Short.MIN_VALUE + ",51] - ["
+                                + (saltBuckets - 1) + ","
+                                + Short.MIN_VALUE + ",51]\nCLIENT MERGE SORT",
+                        queryPlan);
             }
 
             String viewIndexName2 = "I_" + generateUniqueName();
             if (localIndex) {
-                stmt.execute("CREATE LOCAL INDEX " + viewIndexName2 + " on " + viewName + "(s)");
+                stmt.execute("CREATE LOCAL INDEX " + viewIndexName2 + " on "
+                        + viewName + "(s)");
             } else {
-                stmt.execute("CREATE INDEX " + viewIndexName2 + " on " + viewName + "(s)");
+                stmt.execute("CREATE INDEX " + viewIndexName2 + " on "
+                        + viewName + "(s)");
             }
 
             // new index hasn't been analyzed yet
@@ -875,7 +932,8 @@ public class ViewIT extends SplitSystemCatalogIT {
 
             query = "SELECT k1, k2, s FROM " + viewName + " WHERE s = 'foo'";
             rs = stmt.executeQuery(query);
-            Scan scan = stmt.unwrap(PhoenixStatement.class).getQueryPlan().getContext().getScan();
+            Scan scan = stmt.unwrap(PhoenixStatement.class).getQueryPlan()
+                    .getContext().getScan();
             assertTrue(rs.next());
             assertEquals(1, rs.getInt(1));
             assertEquals(120, rs.getInt(2));
@@ -885,21 +943,31 @@ public class ViewIT extends SplitSystemCatalogIT {
             String physicalTableName;
             if (localIndex) {
                 physicalTableName = fullTableName;
-                assertEquals("CLIENT PARALLEL " + (saltBuckets == null ? 1 : saltBuckets)
-                        + "-WAY RANGE SCAN OVER " + fullTableName + " [" + (2) + ",'foo']\n"
-                        + "    SERVER FILTER BY FIRST KEY ONLY\n" + "CLIENT MERGE SORT",
+                assertEquals("CLIENT PARALLEL "
+                                + (saltBuckets == null ? 1 : saltBuckets)
+                                + "-WAY RANGE SCAN OVER " + fullTableName
+                                + " [" + (2) + ",'foo']\n"
+                                + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                                + "CLIENT MERGE SORT",
                         QueryUtil.getExplainPlan(rs));
             } else {
                 physicalTableName = viewIndexPhysicalName;
                 assertEquals(saltBuckets == null ?
-                                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER " + viewIndexPhysicalName
-                                        + " [" + (Short.MIN_VALUE + 1) + ",'foo']\n"
-                                        + "    SERVER FILTER BY FIRST KEY ONLY" :
-                                "CLIENT PARALLEL " + saltBuckets + "-WAY RANGE SCAN OVER "
-                                        + viewIndexPhysicalName + " [0," + (Short.MIN_VALUE + 1)
-                                        + ",'foo'] - [" + (saltBuckets - 1) + ","
-                                        + (Short.MIN_VALUE + 1) + ",'foo']\n"
-                                        + "    SERVER FILTER BY FIRST KEY ONLY\n"
+                                "CLIENT PARALLEL 1-WAY RANGE SCAN OVER "
+                                        + viewIndexPhysicalName
+                                        + " [" + (Short.MIN_VALUE + 1)
+                                        + ",'foo']\n"
+                                        + "    SERVER FILTER BY FIRST KEY ONLY"
+                                :
+                                "CLIENT PARALLEL " + saltBuckets
+                                        + "-WAY RANGE SCAN OVER "
+                                        + viewIndexPhysicalName + " [0,"
+                                        + (Short.MIN_VALUE + 1)
+                                        + ",'foo'] - [" + (saltBuckets - 1)
+                                        + "," + (Short.MIN_VALUE + 1)
+                                        + ",'foo']\n"
+                                        + "    SERVER FILTER BY FIRST KEY ONLY"
+                                        + "\n"
                                         + "CLIENT MERGE SORT",
                         QueryUtil.getExplainPlan(rs));
             }
@@ -908,389 +976,19 @@ public class ViewIT extends SplitSystemCatalogIT {
     }
     
     @Test
-    public void testChildViewCreationFails() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Statement stmt = conn.createStatement()) {
-
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 = FAILED_VIEWNAME;
-            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-
-            String tableDdl = "CREATE TABLE " + fullTableName
-                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
-            stmt.execute(tableDdl);
-            String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k > 5";
-            try {
-                stmt.execute(ddl);
-                fail();
-            } catch (PhoenixIOException ignored) {
-            }
-            ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName
-                    + " WHERE k > 10";
-            stmt.execute(ddl);
-
-            // the first child view should not exist
-            try {
-                PhoenixRuntime.getTableNoCache(conn, fullViewName1);
-                fail();
-            } catch (SQLException ignored) {
-            }
-
-            // we should be able to load the table
-            PhoenixRuntime.getTableNoCache(conn, fullTableName);
-            // we should be able to load the second view
-            PhoenixRuntime.getTableNoCache(conn, fullViewName2);
-        }
-    }
-
-    @Test
-    public void testConcurrentViewCreationAndTableDrop() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
-            latch1 = new CountDownLatch(1);
-            latch2 = new CountDownLatch(1);
-            String tableDdl =
-                    "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)"
-                            + tableDDLOptions;
-            stmt.execute(tableDdl);
-
-            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
-                @Override
-                public Thread newThread(Runnable r) {
-                    Thread t = Executors.defaultThreadFactory().newThread(r);
-                    t.setDaemon(true);
-                    t.setPriority(Thread.MIN_PRIORITY);
-                    return t;
-                }
-            });
-
-            // When dropping a table, we check the parent->child links in the SYSTEM.CHILD_LINK
-            // table and check that cascade is set, if it isn't, we throw an exception (see
-            // ViewUtil.hasChildViews). After PHOENIX-4810, we first send a client-server RPC to add
-            // parent->child links to SYSTEM.CHILD_LINK and then add metadata for the view in
-            // SYSTEM.CATALOG, so we must delay link creation so that the drop table does not fail
-            slowDownAddingChildLink = true;
-            // create the view in a separate thread (which will take some time
-            // to complete)
-            Future<Exception> future =
-                    executorService.submit(new CreateViewRunnable(fullTableName, fullViewName1));
-            // wait till the thread makes the rpc to create the view
-            latch1.await();
-            tableDdl = "DROP TABLE " + fullTableName;
-
-            // Revert this flag since we don't want to wait in preDropTable
-            slowDownAddingChildLink = false;
-            // drop table goes through first and so the view creation should fail
-            stmt.execute(tableDdl);
-            latch2.countDown();
-
-            Exception e = future.get();
-            assertTrue("Expected TableNotFoundException since drop table goes through first",
-                    e instanceof TableNotFoundException &&
-                            fullTableName.equals(((TableNotFoundException) e).getTableName()));
-
-        }
-    }
-
-    @Test
-    public void testChildLinkCreationFailThrowsException() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-            // create base table
-            String tableDdl = "CREATE TABLE " + fullTableName
-                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
-            stmt.execute(tableDdl);
-
-            // Throw an exception in ChildLinkMetaDataEndpoint while adding parent->child links
-            // to simulate a failure
-            throwExceptionInChildLinkPreHook = true;
-            // create a view
-            String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k = 6";
-            try {
-                stmt.execute(ddl);
-                fail("Should have thrown an exception");
-            } catch(SQLException sqlE) {
-                assertEquals("Expected a different Error code",
-                        SQLExceptionCode.UNABLE_TO_CREATE_CHILD_LINK.getErrorCode(),
-                        sqlE.getErrorCode());
-            }
-        }
-    }
-
-    @Test
-    public void testConcurrentAddSameColumnDifferentType() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
-            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-            // create base table
-            String tableDdl = "CREATE TABLE " + fullTableName
-                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
-            stmt.execute(tableDdl);
-            // create a view
-            String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k = 6";
-            stmt.execute(ddl);
-
-            latch1 = new CountDownLatch(1);
-            latch2 = new CountDownLatch(1);
-            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
-                @Override
-                public Thread newThread(Runnable r) {
-                    Thread t = Executors.defaultThreadFactory().newThread(r);
-                    t.setDaemon(true);
-                    t.setPriority(Thread.MIN_PRIORITY);
-                    return t;
-                }
-            });
-
-            // add a column with the same name and different type to the view in a separate thread
-            // (which will take some time to complete)
-            Future<Exception> future = executorService.submit(
-                    new AddColumnRunnable(fullViewName1, null));
-            // wait till the thread makes the rpc to add the column
-            boolean result = latch1.await(2, TimeUnit.MINUTES);
-            if (!result) {
-                fail("The create view rpc look too long");
-            }
-            tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
-            try {
-                // add the same column to the base table with a different type
-                stmt.execute(tableDdl);
-                fail("Adding a column of a to a base table should fail when the same column of a "
-                        + "different type is being added to a child view");
-            } catch (ConcurrentTableMutationException ignored) {
-            }
-            latch2.countDown();
-
-            Exception e = future.get();
-            assertNull(e);
-
-            // add the same column to the another view  to ensure that the cell used
-            // to prevent concurrent modifications was removed
-            ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " 
-                    + fullTableName + " WHERE k = 6";
-            stmt.execute(ddl);
-            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
-            stmt.execute(tableDdl);
-        }
-    }
-
-    // Test that we do a checkAndPut even in case of tenant-specific connections (see PHOENIX-6075)
-    @Test
-    public void testConcurrentAddSameColumnDifferentTypeTenantView() throws Exception {
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
-        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-        String tenantId = "t001";
-        String tableDdl = "CREATE TABLE " + fullTableName
-                + " (TENANT_ID VARCHAR NOT NULL, k INTEGER NOT NULL, v1 DATE CONSTRAINT PK"
-                + " PRIMARY KEY (TENANT_ID, k)) MULTI_TENANT=true"
-                + (!tableDDLOptions.isEmpty() ? ", " : "") + tableDDLOptions;
-        String viewDdl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
-                + fullTableName + " WHERE k = 6";
-
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Statement stmt = conn.createStatement()) {
-            // create a multi-tenant base table
-            stmt.execute(tableDdl);
-
-            Properties props = new Properties();
-            props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
-            try (Connection tenantConn = DriverManager.getConnection(getUrl(), props);
-                    Statement tenantStmt = tenantConn.createStatement()) {
-                // create a tenant-specific view
-                tenantStmt.execute(viewDdl);
-            }
-
-            latch1 = new CountDownLatch(1);
-            latch2 = new CountDownLatch(1);
-            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
-                @Override
-                public Thread newThread(Runnable r) {
-                    Thread t = Executors.defaultThreadFactory().newThread(r);
-                    t.setDaemon(true);
-                    t.setPriority(Thread.MIN_PRIORITY);
-                    return t;
-                }
-            });
-
-            // add a column with the same name and different type to the tenant-specific view in a
-            // separate thread (which will take some time to complete)
-            Future<Exception> future = executorService.submit(
-                    new AddColumnRunnable(fullViewName1, tenantId));
-            // wait till the thread makes the rpc to add the column
-            boolean result = latch1.await(2, TimeUnit.MINUTES);
-            if (!result) {
-                fail("The tenant-specific view creation rpc look too long");
-            }
-            tableDdl = "ALTER TABLE " + fullTableName + " ADD v3 INTEGER";
-            try {
-                // add the same column to the base table with a different type
-                stmt.execute(tableDdl);
-                fail("Adding a column of a to a base table should fail when the same column of a "
-                        + "different type is being added to a child view");
-            } catch (ConcurrentTableMutationException ignored) {
-            }
-            latch2.countDown();
-
-            Exception e = future.get();
-            assertNull(e);
-
-            // add the same column to the another view  to ensure that the cell used
-            // to prevent concurrent modifications was removed
-            viewDdl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k = 6";
-            stmt.execute(viewDdl);
-            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
-            stmt.execute(tableDdl);
-        }
-    }
-    
-    @Test
-    public void testConcurrentAddDifferentColumn() throws Exception {
-        try (Connection conn = DriverManager.getConnection(getUrl());
-                Statement stmt = conn.createStatement()) {
-            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
-            String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-            String fullViewName3 = SchemaUtil.getTableName(SCHEMA4, generateUniqueName());
-            // create base table
-            String tableDdl = "CREATE TABLE " + fullTableName
-                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
-            stmt.execute(tableDdl);
-            // create two views
-            String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k = 6";
-            stmt.execute(ddl);
-            ddl = "CREATE VIEW " + fullViewName3 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k = 7";
-            stmt.execute(ddl);
-
-            latch1 = new CountDownLatch(1);
-            latch2 = new CountDownLatch(1);
-            ExecutorService executorService = Executors.newFixedThreadPool(1, new ThreadFactory() {
-                @Override
-                public Thread newThread(Runnable r) {
-                    Thread t = Executors.defaultThreadFactory().newThread(r);
-                    t.setDaemon(true);
-                    t.setPriority(Thread.MIN_PRIORITY);
-                    return t;
-                }
-            });
-
-            // add a column to a view in a separate thread (we slow this operation down)
-            Future<Exception> future = executorService.submit(
-                    new AddColumnRunnable(fullViewName1, null));
-            // wait till the thread makes the rpc to add the column
-            boolean result = latch1.await(2, TimeUnit.MINUTES);
-            if (!result) {
-                fail("The alter view rpc look too long");
-            }
-            tableDdl = "ALTER VIEW " + fullViewName3 + " ADD v4 INTEGER";
-            try {
-                // add a column to another view 
-                stmt.execute(tableDdl);
-                if (columnEncoded) {
-                    // this should fail as the previous add column is still not complete
-                    fail("Adding columns to two different views concurrently where the base table"
-                            + " uses encoded column should fail");
-                }
-            } catch (ConcurrentTableMutationException e) {
-                if (!columnEncoded) {
-                    // this should not fail as we don't need to update the parent table for non
-                    // column encoded tables
-                    fail("Adding columns to two different views concurrently where the base table"
-                            + " does not use encoded columns should succeed");
-                }
-            }
-            latch2.countDown();
-
-            Exception e = future.get();
-            // if the base table uses column encoding then the add column operation for
-            // fullViewName1 fails
-            assertNull(e);
-
-            // add the same column to the another view  to ensure that the cell used
-            // to prevent concurrent modifications was removed
-            ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM "
-                    + fullTableName + " WHERE k = 6";
-            stmt.execute(ddl);
-            tableDdl = "ALTER VIEW " + fullViewName2 + " ADD v3 INTEGER";
-            stmt.execute(tableDdl);
-        }
-    }
-
-    @Test
     public void testDisallowCreatingViewsOnSystemTable() throws SQLException {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            String viewDDL = "CREATE VIEW " + generateUniqueName() + " AS SELECT * FROM " +
-                    "SYSTEM.CATALOG";
+            String viewDDL = "CREATE VIEW " + generateUniqueName()
+                    + " AS SELECT * FROM SYSTEM.CATALOG";
             try (Statement stmt = conn.createStatement()){
                 stmt.execute(viewDDL);
                 fail("Should have thrown an exception");
             } catch (SQLException sqlE) {
                 assertEquals("Expected a different Error code",
-                        SQLExceptionCode.CANNOT_CREATE_VIEWS_ON_SYSTEM_TABLES.getErrorCode(),
-                        sqlE.getErrorCode());
-            }
-        }
-    }
-
-    private static class CreateViewRunnable implements Callable<Exception> {
-        private final String fullTableName;
-        private final String fullViewName;
-
-        public CreateViewRunnable(String fullTableName, String fullViewName) {
-            this.fullTableName = fullTableName;
-            this.fullViewName = fullViewName;
-        }
-
-        @Override
-        public Exception call() throws Exception {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            try (Connection conn = DriverManager.getConnection(getUrl(), props);
-                    Statement stmt = conn.createStatement()) {
-                String ddl = "CREATE VIEW " + fullViewName + " (v2 VARCHAR) AS SELECT * FROM "
-                        + fullTableName + " WHERE k = 5";
-                stmt.execute(ddl);
-            } catch (SQLException e) {
-                return e;
+                        SQLExceptionCode.CANNOT_CREATE_VIEWS_ON_SYSTEM_TABLES
+                                .getErrorCode(), sqlE.getErrorCode());
             }
-            return null;
         }
     }
 
-    private static class AddColumnRunnable implements Callable<Exception> {
-        private final String fullViewName;
-        private final String tenantId;
-
-        public AddColumnRunnable(String fullViewName, String tenantId) {
-            this.fullViewName = fullViewName;
-            this.tenantId = tenantId;
-        }
-
-        @Override
-        public Exception call() throws Exception {
-            Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
-            if (this.tenantId != null) {
-                props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, this.tenantId);
-            }
-            try (Connection conn = DriverManager.getConnection(getUrl(), props);
-                    Statement stmt = conn.createStatement()) {
-                String ddl = "ALTER VIEW " + fullViewName + " ADD v3 CHAR(15)";
-                stmt.execute(ddl);
-            } catch (SQLException e) {
-                return e;
-            }
-            return null;
-        }
-    }
 }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java
index 51fc812..260b7b3 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewMetadataIT.java
@@ -18,18 +18,27 @@
 package org.apache.phoenix.end2end;
 
 import static com.google.common.collect.Lists.newArrayListWithExpectedSize;
+import static org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost
+        .PHOENIX_META_DATA_COPROCESSOR_CONF_KEY;
 import static org.apache.phoenix.coprocessor.TaskRegionObserver.TASK_DETAILS;
-import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MODIFY_VIEW_PK;
+import static org.apache.phoenix.exception.SQLExceptionCode
+        .CANNOT_MODIFY_VIEW_PK;
 import static org.apache.phoenix.exception.SQLExceptionCode.CANNOT_MUTATE_TABLE;
-import static org.apache.phoenix.exception.SQLExceptionCode.NOT_NULLABLE_COLUMN_IN_ROW_KEY;
+import static org.apache.phoenix.exception.SQLExceptionCode
+        .NOT_NULLABLE_COLUMN_IN_ROW_KEY;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_LINK_HBASE_TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData
+        .SYSTEM_LINK_HBASE_TABLE_NAME;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData
+        .SYSTEM_TASK_HBASE_TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_TASK_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TASK_TYPE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
+import static org.apache.phoenix.query.QueryServicesOptions
+        .DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS;
 import static org.apache.phoenix.schema.PTable.TaskType.DROP_CHILD_VIEWS;
 import static org.apache.phoenix.util.ByteUtil.EMPTY_BYTE_ARRAY;
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
@@ -63,16 +72,12 @@ import org.apache.hadoop.hbase.client.Table;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
-import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost;
 import org.apache.phoenix.coprocessor.TableInfo;
 import org.apache.phoenix.coprocessor.TaskRegionObserver;
-import org.apache.phoenix.end2end.ViewIT.TestMetaDataRegionObserver;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.QueryServices;
-import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PName;
@@ -93,6 +98,9 @@ import com.google.common.base.Predicate;
 import com.google.common.collect.Collections2;
 import com.google.common.collect.Maps;
 
+/**
+ * Test suite related to view metadata
+ */
 public class ViewMetadataIT extends SplitSystemCatalogIT {
 
     private static RegionCoprocessorEnvironment TaskRegionEnvironment;
@@ -101,13 +109,17 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     private final static String CHILD_VIEW_LEVEL_2_SCHEMA = "S2";
     private final static String CHILD_VIEW_LEVEL_3_SCHEMA = "S3";
     final static String CREATE_BASE_TABLE_DDL =
-            "CREATE TABLE %s.%s (A INTEGER NOT NULL PRIMARY KEY, B INTEGER, C INTEGER)";
+            "CREATE TABLE %s.%s (A INTEGER NOT NULL PRIMARY KEY, "
+                    + "B INTEGER, C INTEGER)";
     final static String CREATE_CHILD_VIEW_LEVEL_1_DDL =
-            "CREATE VIEW %s.%s (NEW_COL1 INTEGER) AS SELECT * FROM %s.%s WHERE B > 10";
+            "CREATE VIEW %s.%s (NEW_COL1 INTEGER) AS SELECT * FROM %s.%s "
+                    + "WHERE B > 10";
     final static String CREATE_CHILD_VIEW_LEVEL_2_DDL =
-            "CREATE VIEW %s.%s (NEW_COL2 INTEGER) AS SELECT * FROM %s.%s WHERE NEW_COL1=5";
+            "CREATE VIEW %s.%s (NEW_COL2 INTEGER) AS SELECT * FROM %s.%s "
+                    + "WHERE NEW_COL1=5";
     final static String CREATE_CHILD_VIEW_LEVEL_3_DDL =
-            "CREATE VIEW %s.%s (NEW_COL3 INTEGER) AS SELECT * FROM %s.%s WHERE NEW_COL2=10";
+            "CREATE VIEW %s.%s (NEW_COL3 INTEGER) AS SELECT * FROM %s.%s "
+                    + "WHERE NEW_COL2=10";
 
     @BeforeClass
     public static synchronized void doSetup() throws Exception {
@@ -117,11 +129,13 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         boolean splitSystemCatalog = (driver == null);
         Map<String, String> serverProps = Maps.newHashMapWithExpectedSize(1);
         serverProps.put(QueryServices.PHOENIX_ACLS_ENABLED, "true");
-        serverProps.put(PhoenixMetaDataCoprocessorHost.PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
-                TestMetaDataRegionObserver.class.getName());
+        serverProps.put(PHOENIX_META_DATA_COPROCESSOR_CONF_KEY,
+                ViewConcurrencyAndFailureIT.TestMetaDataRegionObserver.class
+                        .getName());
         serverProps.put("hbase.coprocessor.abortonerror", "false");
-        // Set this in server properties too since we get a connection on the server and pass in
-        // server-side properties when running the drop child views tasks
+        // Set this in server properties too since we get a connection on the
+        // server and pass in server-side properties when running the drop
+        // child views tasks
         serverProps.put(DROP_METADATA_ATTRIB, Boolean.TRUE.toString());
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()),
                 new ReadOnlyProps(clientProps.entrySet().iterator()));
@@ -132,13 +146,11 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
             splitSystemCatalog();
         }
 
-        TaskRegionEnvironment =
-                (RegionCoprocessorEnvironment)getUtility()
-                        .getRSForFirstRegionInTable(
-                                PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
-                        .getOnlineRegions(PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
-                        .get(0).getCoprocessorHost()
-                        .findCoprocessorEnvironment(TaskRegionObserver.class.getName());
+        TaskRegionEnvironment = (RegionCoprocessorEnvironment)getUtility()
+                .getRSForFirstRegionInTable(SYSTEM_TASK_HBASE_TABLE_NAME)
+                .getOnlineRegions(SYSTEM_TASK_HBASE_TABLE_NAME)
+                .get(0).getCoprocessorHost()
+                .findCoprocessorEnvironment(TaskRegionObserver.class.getName());
     }
 
     @Test
@@ -149,10 +161,13 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
       String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
       String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
       conn1.createStatement().execute(
-        "CREATE TABLE "+tableName+" (k VARCHAR PRIMARY KEY, v1 VARCHAR, v2 VARCHAR) UPDATE_CACHE_FREQUENCY=1000000");
-      conn1.createStatement().execute("upsert into "+tableName+" values ('row1', 'value1', 'key1')");
+        "CREATE TABLE "+tableName+" (k VARCHAR PRIMARY KEY, "
+                + "v1 VARCHAR, v2 VARCHAR) UPDATE_CACHE_FREQUENCY=1000000");
+      conn1.createStatement().execute("upsert into "+tableName
+              +" values ('row1', 'value1', 'key1')");
       conn1.createStatement().execute(
-        "CREATE VIEW "+viewName+" (v43 VARCHAR) AS SELECT * FROM "+tableName+" WHERE v1 = 'value1'");
+        "CREATE VIEW "+viewName+" (v43 VARCHAR) AS SELECT * FROM "+tableName
+                +" WHERE v1 = 'value1'");
 
       ResultSet rs = conn1.createStatement()
           .executeQuery("SELECT * FROM "+tableName+" WHERE v1 = 'value1'");
@@ -167,54 +182,67 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         HBaseAdmin admin = driver.getConnectionQueryServices(getUrl(),
                 TestUtil.TEST_PROPERTIES).getAdmin();
 
-        HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(tableNameStr));
+        HTableDescriptor desc = new HTableDescriptor(TableName.valueOf(
+                tableNameStr));
         desc.addFamily(new HColumnDescriptor(familyNameStr));
         admin.createTable(desc);
         Connection conn = DriverManager.getConnection(getUrl());
 
         //PK is not specified, without where clause
         try {
-            conn.createStatement().executeUpdate("CREATE VIEW \"" + tableNameStr +
+            conn.createStatement().executeUpdate("CREATE VIEW \""
+                    + tableNameStr +
                     "\" (ROWKEY VARCHAR, \"" + familyNameStr + "\".a VARCHAR)");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.PRIMARY_KEY_MISSING.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.PRIMARY_KEY_MISSING.getErrorCode(),
+                    e.getErrorCode());
         }
 
         // No error, as PK is specified
         conn.createStatement().executeUpdate("CREATE VIEW \"" + tableNameStr +
-                "\" (ROWKEY VARCHAR PRIMARY KEY, \"" + familyNameStr + "\".a VARCHAR)");
+                "\" (ROWKEY VARCHAR PRIMARY KEY, \"" + familyNameStr
+                + "\".a VARCHAR)");
 
-        conn.createStatement().executeUpdate("DROP VIEW \"" + tableNameStr + "\"");
+        conn.createStatement().executeUpdate("DROP VIEW \"" + tableNameStr
+                + "\"");
 
         //PK is not specified, with where clause
         try {
-            conn.createStatement().executeUpdate("CREATE VIEW \"" + tableNameStr +
-                    "\" (ROWKEY VARCHAR, \"" + familyNameStr + "\".a VARCHAR) AS SELECT * FROM \""
+            conn.createStatement().executeUpdate("CREATE VIEW \""
+                    + tableNameStr +
+                    "\" (ROWKEY VARCHAR, \"" + familyNameStr + "\".a VARCHAR)"
+                    + " AS SELECT * FROM \""
                     + tableNameStr + "\" WHERE ROWKEY = '1'");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.PRIMARY_KEY_MISSING.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.PRIMARY_KEY_MISSING.getErrorCode(),
+                    e.getErrorCode());
         }
 
         conn.createStatement().executeUpdate("CREATE VIEW \"" + tableNameStr +
-                "\" (ROWKEY VARCHAR PRIMARY KEY, \"" + familyNameStr + "\".a VARCHAR) AS SELECT " +
+                "\" (ROWKEY VARCHAR PRIMARY KEY, \"" + familyNameStr
+                + "\".a VARCHAR) AS SELECT " +
                 "* FROM \"" + tableNameStr + "\" WHERE ROWKEY = '1'");
 
-        conn.createStatement().executeUpdate("DROP VIEW \"" + tableNameStr + "\"");
+        conn.createStatement().executeUpdate("DROP VIEW \"" + tableNameStr
+                + "\"");
     }
 
     @Test
-    public void testCreateViewMappedToExistingHbaseTableWithNamespaceMappingEnabled() throws Exception {
+    public void testCreateViewMappedToExistingHbaseTableWithNSMappingEnabled()
+            throws Exception {
         final String NS = "NS_" + generateUniqueName();
         final String TBL = "TBL_" + generateUniqueName();
         final String CF = "CF";
 
         Properties props = new Properties();
-        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.TRUE.toString());
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED,
+                Boolean.TRUE.toString());
 
         try (Connection conn = DriverManager.getConnection(getUrl(), props);
-                Admin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin()) {
+                Admin admin = conn.unwrap(PhoenixConnection.class)
+                        .getQueryServices().getAdmin()) {
 
             conn.createStatement().execute("CREATE SCHEMA " + NS);
 
@@ -227,10 +255,12 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
                 String view1 = NS + "." + TBL;
                 conn.createStatement().execute(
-                        "CREATE VIEW " + view1 + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
+                        "CREATE VIEW " + view1 + " (PK VARCHAR PRIMARY KEY, "
+                                + CF + ".COL VARCHAR)");
 
                 assertTrue(QueryUtil.getExplainPlan(
-                        conn.createStatement().executeQuery("explain select * from " + view1))
+                        conn.createStatement().executeQuery(
+                                "explain select * from " + view1))
                         .contains(NS + ":" + TBL));
 
                 conn.createStatement().execute("DROP VIEW " + view1);
@@ -238,7 +268,8 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
                 admin.deleteTable(tableName);
             }
 
-            // test for a view whose name contains a dot (e.g. "AAA.BBB") in default schema (for backward compatibility)
+            // test for a view whose name contains a dot (e.g. "AAA.BBB") in
+            // default schema (for backward compatibility)
             {
                 TableName tableName = TableName.valueOf(NS + "." + TBL);
                 HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -247,11 +278,12 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
                 String view2 = "\"" + NS + "." + TBL + "\"";
                 conn.createStatement().execute(
-                        "CREATE VIEW " + view2 + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
+                        "CREATE VIEW " + view2 + " (PK VARCHAR PRIMARY KEY, "
+                                + CF + ".COL VARCHAR)");
 
-                assertTrue(QueryUtil
-                        .getExplainPlan(
-                                conn.createStatement().executeQuery("explain select * from " + view2))
+                assertTrue(QueryUtil.getExplainPlan(
+                        conn.createStatement().executeQuery(
+                                "explain select * from " + view2))
                         .contains(NS + "." + TBL));
 
                 conn.createStatement().execute("DROP VIEW " + view2);
@@ -259,7 +291,8 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
                 admin.deleteTable(tableName);
             }
 
-            // test for a view whose name contains a dot (e.g. "AAA.BBB") in non-default schema
+            // test for a view whose name contains a dot (e.g. "AAA.BBB") in
+            // non-default schema
             {
                 TableName tableName = TableName.valueOf(NS, NS + "." + TBL);
                 HTableDescriptor desc = new HTableDescriptor(tableName);
@@ -268,10 +301,12 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
                 String view3 = NS + ".\"" + NS + "." + TBL + "\"";
                 conn.createStatement().execute(
-                        "CREATE VIEW " + view3 + " (PK VARCHAR PRIMARY KEY, " + CF + ".COL VARCHAR)");
+                        "CREATE VIEW " + view3 + " (PK VARCHAR PRIMARY KEY, "
+                                + CF + ".COL VARCHAR)");
 
                 assertTrue(QueryUtil.getExplainPlan(
-                        conn.createStatement().executeQuery("explain select * from " + view3))
+                        conn.createStatement().executeQuery(
+                                "explain select * from " + view3))
                         .contains(NS + ":" + NS + "." + TBL));
 
                 conn.createStatement().execute("DROP VIEW " + view3);
@@ -285,22 +320,30 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     @Test
     public void testRecreateDroppedTableWithChildViews() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-
-        String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
+
+        String tableDdl = "CREATE TABLE " + fullTableName
+                + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(tableDdl);
-        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        String ddl = "CREATE VIEW " + fullViewName1
+                + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         String indexName = generateUniqueName();
         ddl = "CREATE INDEX " + indexName + " on " + fullViewName1 + "(v2)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
+        ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM "
+                + fullTableName + " WHERE k > 10";
         conn.createStatement().execute(ddl);
 
         // drop table cascade should succeed
-        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName
+                + " CASCADE");
         runDropChildViewsTask();
 
         validateViewDoesNotExist(conn, fullViewName1);
@@ -312,12 +355,12 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         try {
             PhoenixRuntime.getTableNoCache(conn, fullViewName1);
             fail();
-        } catch (SQLException e) {
+        } catch (SQLException ignored) {
         }
         try {
             PhoenixRuntime.getTableNoCache(conn, fullViewName2);
             fail();
-        } catch (SQLException e) {
+        } catch (SQLException ignored) {
         }
     }
 
@@ -326,23 +369,26 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         final String parent1TableName = generateUniqueName();
         final String parent2TableName = generateUniqueName();
         final String viewName = "V_" + generateUniqueName();
-        // Note that this column name is the same as the new column on the child view
+        // Note that this column name is the same as the new column on the
+        // child view
         final String alterTableDDL = "ALTER TABLE %s ADD NEW_COL1 VARCHAR";
-        createOrphanLink(SCHEMA1, parent1TableName, parent2TableName, SCHEMA2, viewName);
+        createOrphanLink(SCHEMA1, parent1TableName, parent2TableName, SCHEMA2,
+                viewName);
 
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            // Should not fail since this table is unrelated to the view in spite of
-            // the orphan parent->child link
+            // Should not fail since this table is unrelated to the view
+            // in spite of the orphan parent->child link
             stmt.execute(String.format(alterTableDDL,
                     SchemaUtil.getTableName(SCHEMA1, parent2TableName)));
             try {
                 stmt.execute(String.format(alterTableDDL,
                         SchemaUtil.getTableName(SCHEMA1, parent1TableName)));
-                fail("Adding column should be disallowed since there is a conflicting column type "
-                        + "on the child view");
+                fail("Adding column should be disallowed since there is a "
+                        + "conflicting column type on the child view");
             } catch (SQLException sqlEx) {
-                assertEquals(CANNOT_MUTATE_TABLE.getErrorCode(), sqlEx.getErrorCode());
+                assertEquals(CANNOT_MUTATE_TABLE.getErrorCode(),
+                        sqlEx.getErrorCode());
             }
         }
     }
@@ -353,20 +399,23 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         final String parent2TableName = generateUniqueName();
         final String viewName = "V_" + generateUniqueName();
         final String dropTableNoCascadeDDL = "DROP TABLE %s ";
-        createOrphanLink(SCHEMA1, parent1TableName, parent2TableName, SCHEMA2, viewName);
+        createOrphanLink(SCHEMA1, parent1TableName, parent2TableName, SCHEMA2,
+                viewName);
 
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            // Should not fail since this table is unrelated to the view in spite of
-            // the orphan parent->child link
+            // Should not fail since this table is unrelated to the view
+            // in spite of the orphan parent->child link
             stmt.execute(String.format(dropTableNoCascadeDDL,
                     SchemaUtil.getTableName(SCHEMA1, parent2TableName)));
             try {
                 stmt.execute(String.format(dropTableNoCascadeDDL,
                         SchemaUtil.getTableName(SCHEMA1, parent1TableName)));
-                fail("Drop table without cascade should fail since there is a child view");
+                fail("Drop table without cascade should fail since there is a"
+                        + " child view");
             } catch (SQLException sqlEx) {
-                assertEquals(CANNOT_MUTATE_TABLE.getErrorCode(), sqlEx.getErrorCode());
+                assertEquals(CANNOT_MUTATE_TABLE.getErrorCode(),
+                        sqlEx.getErrorCode());
             }
         }
     }
@@ -374,20 +423,31 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     /**
      * Create a view hierarchy:
      *
-     *              _____ parent1 ____                                 _____ parent2 ____
-     *             /         |        \                               /         |        \
-     *    level1view1   level1view3  level1view4            level1view2   level1view5  level1view6
-     *         |                                                 |
-     *  t001.level2view1                                 t001.level2view2
-     *                                                           |
-     *                                                   t001.level3view1
+     *              _____ parent1 ____
+     *             /         |        \
+     *    level1view1   level1view3  level1view4
+     *         |
+     *  t001.level2view1
+     *
+     *                      And
      *
-     * We induce orphan links by recreating the same view names on top of different parents
+     *              _____ parent2 ____
+     *             /         |        \
+     *    level1view2   level1view5  level1view6
+     *         |
+     *  t001.level2view2
+     *         |
+     *  t001.level3view1
+     *
+     * We induce orphan links by recreating the same view names on top of
+     * different parents
      */
     @Test
     public void testViewHierarchyWithOrphanLinks() throws Exception {
-        final List<TableInfo> expectedLegitChildViewsListForParent1 = new ArrayList<>();
-        final List<TableInfo> expectedLegitChildViewsListForParent2 = new ArrayList<>();
+        final List<TableInfo> expectedLegitChildViewsListForParent1 =
+                new ArrayList<>();
+        final List<TableInfo> expectedLegitChildViewsListForParent2 =
+                new ArrayList<>();
         final String tenantId = "t001";
         final String parent1TableName = "P1_" + generateUniqueName();
         final String parent2TableName = "P2_" + generateUniqueName();
@@ -408,36 +468,45 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
         // Create other legit views on top of parent1 and parent2
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_1_DDL,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName3,
                     BASE_TABLE_SCHEMA, parent1TableName));
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_1_DDL,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName4,
                     BASE_TABLE_SCHEMA, parent1TableName));
 
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_1_DDL,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName2,
                     BASE_TABLE_SCHEMA, parent2TableName));
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_1_DDL,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName5,
                     BASE_TABLE_SCHEMA, parent2TableName));
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_1_DDL,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName6,
                     BASE_TABLE_SCHEMA, parent2TableName));
         }
         Properties props = new Properties();
         props.put(TENANT_ID_ATTRIB, tenantId);
         try (Connection conn = DriverManager.getConnection(getUrl(), props)) {
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_2_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_2_DDL,
                     CHILD_VIEW_LEVEL_2_SCHEMA, level2ViewName1,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName1));
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_2_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_2_DDL,
                     CHILD_VIEW_LEVEL_2_SCHEMA, level2ViewName2,
                     CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName2));
 
-            // Try to recreate the same view on a different global view to create an orphan link
+            // Try to recreate the same view on a different global view to
+            // create an orphan link
             try {
-                conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_2_DDL,
+                conn.createStatement().execute(String.format(
+                        CREATE_CHILD_VIEW_LEVEL_2_DDL,
                         CHILD_VIEW_LEVEL_2_SCHEMA, level2ViewName2,
                         CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName1));
                 fail("Creating the same view again should have failed");
@@ -445,33 +514,47 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
                 // expected
             }
             // Create a third level view
-            conn.createStatement().execute(String.format(CREATE_CHILD_VIEW_LEVEL_3_DDL,
+            conn.createStatement().execute(String.format(
+                    CREATE_CHILD_VIEW_LEVEL_3_DDL,
                     CHILD_VIEW_LEVEL_3_SCHEMA, level3ViewName1,
                     CHILD_VIEW_LEVEL_2_SCHEMA, level2ViewName2));
         }
         // Populate all expected legitimate views in depth-first order
         expectedLegitChildViewsListForParent1.add(new TableInfo(null,
-                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName1.getBytes()));
-        expectedLegitChildViewsListForParent1.add(new TableInfo(tenantId.getBytes(),
-                CHILD_VIEW_LEVEL_2_SCHEMA.getBytes(), level2ViewName1.getBytes()));
+                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                level1ViewName1.getBytes()));
+        expectedLegitChildViewsListForParent1.add(
+                new TableInfo(tenantId.getBytes(),
+                CHILD_VIEW_LEVEL_2_SCHEMA.getBytes(),
+                level2ViewName1.getBytes()));
         expectedLegitChildViewsListForParent1.add(new TableInfo(null,
-                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName3.getBytes()));
+                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                level1ViewName3.getBytes()));
         expectedLegitChildViewsListForParent1.add(new TableInfo(null,
-                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName4.getBytes()));
+                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                level1ViewName4.getBytes()));
 
         expectedLegitChildViewsListForParent2.add(new TableInfo(null,
-                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName2.getBytes()));
-        expectedLegitChildViewsListForParent2.add(new TableInfo(tenantId.getBytes(),
-                CHILD_VIEW_LEVEL_2_SCHEMA.getBytes(), level2ViewName2.getBytes()));
-        expectedLegitChildViewsListForParent2.add(new TableInfo(tenantId.getBytes(),
-                CHILD_VIEW_LEVEL_3_SCHEMA.getBytes(), level3ViewName1.getBytes()));
+                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                level1ViewName2.getBytes()));
+        expectedLegitChildViewsListForParent2.add(
+                new TableInfo(tenantId.getBytes(),
+                CHILD_VIEW_LEVEL_2_SCHEMA.getBytes(),
+                        level2ViewName2.getBytes()));
+        expectedLegitChildViewsListForParent2.add(
+                new TableInfo(tenantId.getBytes(),
+                CHILD_VIEW_LEVEL_3_SCHEMA.getBytes(),
+                        level3ViewName1.getBytes()));
         expectedLegitChildViewsListForParent2.add(new TableInfo(null,
-                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName5.getBytes()));
+                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                level1ViewName5.getBytes()));
         expectedLegitChildViewsListForParent2.add(new TableInfo(null,
-                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName6.getBytes()));
+                CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                level1ViewName6.getBytes()));
 
         /*
-            After this setup, SYSTEM.CHILD_LINK parent->child linking rows will look like this:
+            After this setup, SYSTEM.CHILD_LINK parent->child linking rows
+            will look like this:
             parent1->level1view1
             parent1->level1view3
             parent1->level1view4
@@ -489,71 +572,92 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
          */
 
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            ConnectionQueryServices cqs = conn.unwrap(PhoenixConnection.class).getQueryServices();
+            ConnectionQueryServices cqs = conn.unwrap(PhoenixConnection.class)
+                    .getQueryServices();
             try (Table childLinkTable = cqs.getTable(SchemaUtil.getPhysicalName(
-                    SYSTEM_LINK_HBASE_TABLE_NAME.toBytes(), cqs.getProps()).getName())) {
+                    SYSTEM_LINK_HBASE_TABLE_NAME.toBytes(),
+                    cqs.getProps()).getName())) {
                 Pair<List<PTable>, List<TableInfo>> allDescendants =
-                        ViewUtil.findAllDescendantViews(childLinkTable, cqs.getConfiguration(),
+                        ViewUtil.findAllDescendantViews(childLinkTable,
+                                cqs.getConfiguration(),
                                 EMPTY_BYTE_ARRAY, BASE_TABLE_SCHEMA.getBytes(),
-                                parent1TableName.getBytes(), HConstants.LATEST_TIMESTAMP, false);
+                                parent1TableName.getBytes(),
+                                HConstants.LATEST_TIMESTAMP, false);
                 List<PTable> legitChildViews = allDescendants.getFirst();
                 List<TableInfo> orphanViews = allDescendants.getSecond();
-                // All of the orphan links are legit views of the other parent so they don't count
-                // as orphan views for this parent
+                // All of the orphan links are legit views of the other parent
+                // so they don't count as orphan views for this parent
                 assertTrue(orphanViews.isEmpty());
-                assertLegitChildViews(expectedLegitChildViewsListForParent1, legitChildViews);
+                assertLegitChildViews(expectedLegitChildViewsListForParent1,
+                        legitChildViews);
 
                 allDescendants = ViewUtil.findAllDescendantViews(childLinkTable,
-                        cqs.getConfiguration(), EMPTY_BYTE_ARRAY, BASE_TABLE_SCHEMA.getBytes(),
-                        parent2TableName.getBytes(), HConstants.LATEST_TIMESTAMP, false);
+                        cqs.getConfiguration(), EMPTY_BYTE_ARRAY,
+                        BASE_TABLE_SCHEMA.getBytes(),
+                        parent2TableName.getBytes(),
+                        HConstants.LATEST_TIMESTAMP, false);
                 legitChildViews = allDescendants.getFirst();
                 orphanViews = allDescendants.getSecond();
-                // All of the orphan links are legit views of the other parent so they don't count
-                // as orphan views for this parent
+                // All of the orphan links are legit views of the other parent
+                // so they don't count as orphan views for this parent
                 assertTrue(orphanViews.isEmpty());
-                assertLegitChildViews(expectedLegitChildViewsListForParent2, legitChildViews);
-
-                // Drop one of the legitimate level 1 views that was on top of parent1
-                conn.createStatement().execute(String.format("DROP VIEW %s.%s CASCADE",
-                        CHILD_VIEW_LEVEL_1_SCHEMA, level1ViewName1));
-                // The view hierarchy rooted at this view is 2 levels deep so we must run the
-                // DropChildViewsTask twice to clear out views level by level
+                assertLegitChildViews(expectedLegitChildViewsListForParent2,
+                        legitChildViews);
+
+                // Drop one of the legitimate level 1 views that was on top of
+                // parent1
+                conn.createStatement().execute(String.format(
+                        "DROP VIEW %s.%s CASCADE", CHILD_VIEW_LEVEL_1_SCHEMA,
+                        level1ViewName1));
+                // The view hierarchy rooted at this view is 2 levels deep so
+                // we must run the DropChildViewsTask twice to clear out views
+                // level by level
                 runDropChildViewsTask();
                 runDropChildViewsTask();
 
                 expectedLegitChildViewsListForParent1.clear();
                 expectedLegitChildViewsListForParent1.add(new TableInfo(
-                        null, CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName3.getBytes()));
+                        null, CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                        level1ViewName3.getBytes()));
                 expectedLegitChildViewsListForParent1.add(new TableInfo(
-                        null, CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(), level1ViewName4.getBytes()));
+                        null, CHILD_VIEW_LEVEL_1_SCHEMA.getBytes(),
+                        level1ViewName4.getBytes()));
 
                 allDescendants = ViewUtil.findAllDescendantViews(childLinkTable,
-                        cqs.getConfiguration(), EMPTY_BYTE_ARRAY, BASE_TABLE_SCHEMA.getBytes(),
-                        parent1TableName.getBytes(), HConstants.LATEST_TIMESTAMP, false);
+                        cqs.getConfiguration(), EMPTY_BYTE_ARRAY,
+                        BASE_TABLE_SCHEMA.getBytes(),
+                        parent1TableName.getBytes(),
+                        HConstants.LATEST_TIMESTAMP, false);
                 legitChildViews = allDescendants.getFirst();
                 orphanViews = allDescendants.getSecond();
-                assertLegitChildViews(expectedLegitChildViewsListForParent1, legitChildViews);
+                assertLegitChildViews(expectedLegitChildViewsListForParent1,
+                        legitChildViews);
                 assertTrue(orphanViews.isEmpty());
 
                 allDescendants = ViewUtil.findAllDescendantViews(childLinkTable,
-                        cqs.getConfiguration(), EMPTY_BYTE_ARRAY, BASE_TABLE_SCHEMA.getBytes(),
-                        parent2TableName.getBytes(), HConstants.LATEST_TIMESTAMP, false);
+                        cqs.getConfiguration(), EMPTY_BYTE_ARRAY,
+                        BASE_TABLE_SCHEMA.getBytes(),
+                        parent2TableName.getBytes(),
+                        HConstants.LATEST_TIMESTAMP, false);
                 legitChildViews = allDescendants.getFirst();
                 orphanViews = allDescendants.getSecond();
 
-                // We prune orphan branches and so we will not explore any more orphan links that
-                // stem from the first found orphan
+                // We prune orphan branches and so we will not explore any
+                // more orphan links that stem from the first found orphan
                 assertEquals(1, orphanViews.size());
                 assertEquals(0, orphanViews.get(0).getTenantId().length);
                 assertEquals(CHILD_VIEW_LEVEL_1_SCHEMA,
                         Bytes.toString(orphanViews.get(0).getSchemaName()));
-                assertEquals(level1ViewName1, Bytes.toString(orphanViews.get(0).getTableName()));
-                assertLegitChildViews(expectedLegitChildViewsListForParent2, legitChildViews);
+                assertEquals(level1ViewName1, Bytes.toString(
+                        orphanViews.get(0).getTableName()));
+                assertLegitChildViews(expectedLegitChildViewsListForParent2,
+                        legitChildViews);
             }
         }
     }
 
-    private void assertLegitChildViews(List<TableInfo> expectedList, List<PTable> actualList) {
+    private void assertLegitChildViews(List<TableInfo> expectedList,
+            List<PTable> actualList) {
         assertEquals(expectedList.size(), actualList.size());
         for (int i=0; i<expectedList.size(); i++) {
             TableInfo expectedChild = expectedList.get(i);
@@ -561,7 +665,8 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
             PName actualTenantId = actualList.get(i).getTenantId();
             assertTrue((expectedTenantId == null && actualTenantId == null) ||
                     ((actualTenantId != null && expectedTenantId != null) &&
-                            Arrays.equals(actualTenantId.getBytes(), expectedTenantId)));
+                            Arrays.equals(actualTenantId.getBytes(),
+                                    expectedTenantId)));
             assertEquals(Bytes.toString(expectedChild.getSchemaName()),
                     actualList.get(i).getSchemaName().getString());
             assertEquals(Bytes.toString(expectedChild.getTableName()),
@@ -569,71 +674,87 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         }
     }
 
-    // Create 2 base tables and attempt to create the same view on top of both. The second view
-    // creation will fail, however an orphan parent->child link will be created inside
-    // SYSTEM.CHILD_LINK between parent2 and the view
-    static void createOrphanLink(String parentSchema, String parent1, String parent2,
-            String viewSchema, String viewName) throws SQLException {
+    // Create 2 base tables and attempt to create the same view on top of both.
+    // The second view creation will fail, however an orphan parent->child link
+    // will be created inside SYSTEM.CHILD_LINK between parent2 and the view
+    static void createOrphanLink(String parentSchema, String parent1,
+            String parent2, String viewSchema, String viewName)
+            throws SQLException {
 
         final String querySysChildLink =
                 "SELECT * FROM SYSTEM.CHILD_LINK WHERE TABLE_SCHEM='%s' AND "
-                        + "TABLE_NAME='%s' AND COLUMN_FAMILY='%s' AND " + LINK_TYPE + " = " +
+                        + "TABLE_NAME='%s' AND COLUMN_FAMILY='%s' AND "
+                        + LINK_TYPE + " = " +
                         PTable.LinkType.CHILD_TABLE.getSerializedValue();
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement stmt = conn.createStatement()) {
-            stmt.execute(String.format(CREATE_BASE_TABLE_DDL, parentSchema, parent1));
-            stmt.execute(String.format(CREATE_BASE_TABLE_DDL, parentSchema, parent2));
-            stmt.execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL, viewSchema, viewName,
-                    parentSchema, parent1));
+            stmt.execute(String.format(CREATE_BASE_TABLE_DDL, parentSchema,
+                    parent1));
+            stmt.execute(String.format(CREATE_BASE_TABLE_DDL, parentSchema,
+                    parent2));
+            stmt.execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+                    viewSchema, viewName,  parentSchema, parent1));
             try {
-                stmt.execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL, viewSchema, viewName,
-                        parentSchema, parent2));
+                stmt.execute(String.format(CREATE_CHILD_VIEW_LEVEL_1_DDL,
+                        viewSchema, viewName, parentSchema, parent2));
                 fail("Creating the same view again should have failed");
             } catch (TableAlreadyExistsException ignored) {
                 // expected
             }
 
-            // Confirm that the orphan parent->child link exists after the second view creation
-            ResultSet rs = stmt.executeQuery(String.format(querySysChildLink, parentSchema,
-                    parent2, SchemaUtil.getTableName(viewSchema, viewName)));
+            // Confirm that the orphan parent->child link exists after the
+            // second view creation
+            ResultSet rs = stmt.executeQuery(String.format(querySysChildLink,
+                    parentSchema,  parent2, SchemaUtil.getTableName(
+                            viewSchema, viewName)));
             assertTrue(rs.next());
         }
     }
 
     void runDropChildViewsTask() {
         // Run DropChildViewsTask to complete the tasks for dropping child views
-        TaskRegionObserver.SelfHealingTask task = new TaskRegionObserver.SelfHealingTask(
-                TaskRegionEnvironment, QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS);
+        TaskRegionObserver.SelfHealingTask task =
+                new TaskRegionObserver.SelfHealingTask(TaskRegionEnvironment,
+                        DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS);
         task.run();
     }
 
     @Test
     public void testRecreateIndexWhoseAncestorWasDropped() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullTableName2 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-
-        String tableDdl = "CREATE TABLE " + fullTableName1 + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullTableName2 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+
+        String tableDdl = "CREATE TABLE " + fullTableName1
+                + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(tableDdl);
-        tableDdl = "CREATE TABLE " + fullTableName2 + "  (k INTEGER NOT NULL PRIMARY KEY, v3 DATE)";
+        tableDdl = "CREATE TABLE " + fullTableName2
+                + "  (k INTEGER NOT NULL PRIMARY KEY, v3 DATE)";
         conn.createStatement().execute(tableDdl);
-        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
+        String ddl = "CREATE VIEW " + fullViewName1
+                + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         String indexName = generateUniqueName();
         ddl = "CREATE INDEX " + indexName + " on " + fullViewName1 + "(v2)";
         conn.createStatement().execute(ddl);
         try {
                 // this should fail because an index with this name is present
-            ddl = "CREATE INDEX " + indexName + " on " + fullTableName2 + "(v1)";
+            ddl = "CREATE INDEX " + indexName + " on " + fullTableName2
+                    + "(v1)";
             conn.createStatement().execute(ddl);
             fail();
         }
-        catch(SQLException e) {
+        catch(SQLException ignored) {
         }
 
         // drop table cascade should succeed
-        conn.createStatement().execute("DROP TABLE " + fullTableName1 + " CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName1
+                + " CASCADE");
         runDropChildViewsTask();
 
         // should be able to reuse the index name 
@@ -649,23 +770,33 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     @Test
     public void testRecreateViewWhoseParentWasDropped() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullTableName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-
-        String tableDdl = "CREATE TABLE " + fullTableName1 + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String fullTableName1 = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullTableName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
+
+        String tableDdl = "CREATE TABLE " + fullTableName1
+                + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(tableDdl);
-        tableDdl = "CREATE TABLE " + fullTableName2 + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        tableDdl = "CREATE TABLE " + fullTableName2
+                + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(tableDdl);
-        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
+        String ddl = "CREATE VIEW " + fullViewName1
+                + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
 
         // drop table cascade should succeed
-        conn.createStatement().execute("DROP TABLE " + fullTableName1 + " CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName1
+                + " CASCADE");
         runDropChildViewsTask();
 
         // should be able to reuse the view name 
-        ddl = "CREATE VIEW " + fullViewName1 + " (v3 VARCHAR) AS SELECT * FROM " + fullTableName2 + " WHERE k > 5";
+        ddl = "CREATE VIEW " + fullViewName1
+                + " (v3 VARCHAR) AS SELECT * FROM " + fullTableName2
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
 
         PTable view = PhoenixRuntime.getTableNoCache(conn, fullViewName1);
@@ -678,75 +809,97 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     public void testRepeatedCreateAndDropCascadeTableWorks() throws Exception {
         String tableName = generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(SCHEMA1, tableName);
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
 
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            createTableViewAndDropCascade(conn, fullTableName, fullViewName, false);
+            createTableViewAndDropCascade(conn, fullTableName, fullViewName,
+                    false);
             validateViewDoesNotExist(conn, fullViewName);
-            validateSystemTaskContainsCompletedDropChildViewsTasks(conn, SCHEMA1, tableName, 1);
+            validateSystemTaskContainsCompletedDropChildViewsTasks(conn,
+                    SCHEMA1, tableName, 1);
 
             // Repeat this and check that the view still doesn't exist
-            createTableViewAndDropCascade(conn, fullTableName, fullViewName, false);
+            createTableViewAndDropCascade(conn, fullTableName, fullViewName,
+                    false);
             validateViewDoesNotExist(conn, fullViewName);
-            validateSystemTaskContainsCompletedDropChildViewsTasks(conn, SCHEMA1, tableName, 2);
+            validateSystemTaskContainsCompletedDropChildViewsTasks(conn,
+                    SCHEMA1, tableName, 2);
         }
     }
 
-    // We set DROP_METADATA_ATTRIB to true and check that this does not fail dropping child views
-    // that have an index, though their underlying physical table was already dropped.
+    // We set DROP_METADATA_ATTRIB to true and check that this does not fail
+    // dropping child views that have an index, though their underlying
+    // physical table was already dropped.
     // See PHOENIX-5545.
     @Test
-    public void testDropTableCascadeWithChildViewWithIndex() throws SQLException {
+    public void testDropTableCascadeWithChildViewWithIndex()
+            throws SQLException {
         String tableName = generateUniqueName();
         String fullTableName = SchemaUtil.getTableName(SCHEMA1, tableName);
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            createTableViewAndDropCascade(conn, fullTableName, fullViewName, true);
+            createTableViewAndDropCascade(conn, fullTableName, fullViewName,
+                    true);
             validateViewDoesNotExist(conn, fullViewName);
-            validateSystemTaskContainsCompletedDropChildViewsTasks(conn, SCHEMA1, tableName, 1);
+            validateSystemTaskContainsCompletedDropChildViewsTasks(conn,
+                    SCHEMA1, tableName, 1);
         }
     }
 
-    private void createTableViewAndDropCascade(Connection conn, String fullTableName,
-            String fullViewName, boolean createViewIndex) throws SQLException {
+    private void createTableViewAndDropCascade(Connection conn,
+            String fullTableName, String fullViewName, boolean createViewIndex)
+            throws SQLException {
         String tableDdl = "CREATE TABLE " + fullTableName +
                 "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(tableDdl);
         String ddl = "CREATE VIEW " + fullViewName +
-                " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+                " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         if (createViewIndex) {
-            conn.createStatement().execute("CREATE INDEX " + "INDEX_" + generateUniqueName() +
+            conn.createStatement().execute("CREATE INDEX " + "INDEX_"
+                    + generateUniqueName() +
                     " ON " + fullViewName + "(v2)");
         }
         // drop table cascade should succeed
-        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName
+                + " CASCADE");
         runDropChildViewsTask();
     }
 
-    private void validateSystemTaskContainsCompletedDropChildViewsTasks(Connection conn,
-            String schemaName, String tableName, int numTasks) throws SQLException {
-        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM " + SYSTEM_TASK_NAME +
-                " WHERE " + TASK_TYPE + "=" + DROP_CHILD_VIEWS.getSerializedValue() +
+    private void validateSystemTaskContainsCompletedDropChildViewsTasks(
+            Connection conn,  String schemaName, String tableName,
+            int numTasks) throws SQLException {
+        ResultSet rs = conn.createStatement().executeQuery("SELECT * FROM "
+                + SYSTEM_TASK_NAME +
+                " WHERE " + TASK_TYPE + "=" +
+                DROP_CHILD_VIEWS.getSerializedValue() +
                 " AND " + TENANT_ID + " IS NULL" +
                 " AND " + TABLE_SCHEM + "='" + schemaName +
                 "' AND " + TABLE_NAME + "='" + tableName + "'");
         assertTrue(rs.next());
         for (int i = 0; i < numTasks; i++) {
             Timestamp maxTs = new Timestamp(HConstants.LATEST_TIMESTAMP);
-            assertNotEquals("Should have got a valid timestamp", maxTs, rs.getTimestamp(2));
-            assertEquals("Task should be completed", PTable.TaskStatus.COMPLETED.toString(),
+            assertNotEquals("Should have got a valid timestamp", maxTs,
+                    rs.getTimestamp(2));
+            assertEquals("Task should be completed",
+                    PTable.TaskStatus.COMPLETED.toString(),
                     rs.getString(6));
-            assertNotNull("Task end time should not be null", rs.getTimestamp(7));
+            assertNotNull("Task end time should not be null",
+                    rs.getTimestamp(7));
             String taskData = rs.getString(9);
-            assertTrue("Task data should contain final status", taskData != null &&
+            assertTrue("Task data should contain final status",
+                    taskData != null &&
                     taskData.contains(TASK_DETAILS) &&
                     taskData.contains(PTable.TaskStatus.COMPLETED.toString()));
         }
     }
 
     @Test
-    public void testViewAndTableInDifferentSchemasWithNamespaceMappingEnabled() throws Exception {
+    public void testViewAndTableInDifferentSchemasWithNamespaceMappingEnabled()
+            throws Exception {
         testViewAndTableInDifferentSchemas(true);
     }
 
@@ -756,31 +909,43 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
     }
 
-    private void testViewAndTableInDifferentSchemas(boolean isNamespaceMapped) throws Exception {
+    private void testViewAndTableInDifferentSchemas(boolean isNamespaceMapped)
+            throws Exception {
         Properties props = new Properties();
-        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED,
+                Boolean.toString(isNamespaceMapped));
         Connection conn = DriverManager.getConnection(getUrl(),props);
         String tableName = "T_" + generateUniqueName();
         String schemaName1 = SCHEMA1;
         String fullTableName1 = SchemaUtil.getTableName(schemaName1, tableName);
         String viewName1 = "V_" + generateUniqueName();
         String viewSchemaName = SCHEMA2;
-        String fullViewName1 = SchemaUtil.getTableName(viewSchemaName, viewName1);
+        String fullViewName1 = SchemaUtil.getTableName(viewSchemaName,
+                viewName1);
         String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, viewName1);
 
         if (isNamespaceMapped) {
-            conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS " + schemaName1);
+            conn.createStatement().execute("CREATE SCHEMA IF NOT EXISTS "
+                    + schemaName1);
         }
-        String ddl = "CREATE TABLE " + fullTableName1 + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
-        HBaseAdmin admin = conn.unwrap(PhoenixConnection.class).getQueryServices().getAdmin();
+        String ddl = "CREATE TABLE " + fullTableName1
+                + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        HBaseAdmin admin = conn.unwrap(PhoenixConnection.class)
+                .getQueryServices().getAdmin();
         conn.createStatement().execute(ddl);
-        assertTrue(admin.tableExists(SchemaUtil.getPhysicalTableName(SchemaUtil.normalizeIdentifier(fullTableName1),
-                conn.unwrap(PhoenixConnection.class).getQueryServices().getProps())));
-
-        ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
+        assertTrue(admin.tableExists(SchemaUtil.getPhysicalTableName(
+                SchemaUtil.normalizeIdentifier(fullTableName1),
+                conn.unwrap(PhoenixConnection.class).getQueryServices()
+                        .getProps())));
+
+        ddl = "CREATE VIEW " + fullViewName1
+                + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
 
-        ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1 + " WHERE k > 5";
+        ddl = "CREATE VIEW " + fullViewName2
+                + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName1
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
 
         conn.createStatement().executeQuery("SELECT * FROM " + fullViewName1);
@@ -793,7 +958,8 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         }
         ddl = "DROP VIEW " + fullViewName1;
         conn.createStatement().execute(ddl);
-        ddl = "DROP VIEW " + SchemaUtil.getTableName(viewSchemaName, generateUniqueName());
+        ddl = "DROP VIEW " + SchemaUtil.getTableName(viewSchemaName,
+                generateUniqueName());
         try {
             conn.createStatement().execute(ddl);
             fail();
@@ -809,18 +975,26 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     @Test
     public void testViewAndTableAndDropCascade() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-
-        String tableDdl = "CREATE TABLE " + fullTableName + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
+
+        String tableDdl = "CREATE TABLE " + fullTableName
+                + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(tableDdl);
-        String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        String ddl = "CREATE VIEW " + fullViewName1
+                + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName
+                + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         String indexName = generateUniqueName();
-        ddl = "CREATE LOCAL INDEX " + indexName + " on " + fullViewName1 + "(v2)";
+        ddl = "CREATE LOCAL INDEX " + indexName + " on " + fullViewName1
+                + "(v2)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
+        ddl = "CREATE VIEW " + fullViewName2 + "(v2 VARCHAR) AS SELECT * FROM "
+                + fullTableName + " WHERE k > 10";
         conn.createStatement().execute(ddl);
 
         // dropping base table without cascade should fail
@@ -833,7 +1007,8 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         }
 
         // drop table cascade should succeed
-        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName
+                + " CASCADE");
         runDropChildViewsTask();
 
         validateViewDoesNotExist(conn, fullViewName1);
@@ -843,78 +1018,107 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
     @Test
     public void testUpdatingPropertyOnBaseTable() throws Exception {
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
         try (Connection conn = DriverManager.getConnection(getUrl())) {
             conn.createStatement()
                     .execute("create table " + fullTableName
-                            + "(tenantId CHAR(15) NOT NULL, pk1 integer NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
+                            + "(tenantId CHAR(15) NOT NULL, pk1 integer "
+                            + "NOT NULL, v varchar CONSTRAINT PK PRIMARY KEY "
                             + "(tenantId, pk1)) MULTI_TENANT=true");
-            conn.createStatement().execute("CREATE VIEW " + fullViewName + " AS SELECT * FROM " + fullTableName);
+            conn.createStatement().execute("CREATE VIEW " + fullViewName
+                    + " AS SELECT * FROM " + fullTableName);
 
             conn.createStatement()
-                    .execute("ALTER TABLE " + fullTableName + " set IMMUTABLE_ROWS = true");
+                    .execute("ALTER TABLE " + fullTableName
+                            + " set IMMUTABLE_ROWS = true");
 
             // fetch the latest tables
             PTable table = PhoenixRuntime.getTableNoCache(conn, fullTableName);
             PTable view = PhoenixRuntime.getTableNoCache(conn, fullViewName);
-            assertEquals("IMMUTABLE_ROWS property set incorrectly", true, table.isImmutableRows());
-            assertEquals("IMMUTABLE_ROWS property set incorrectly", true, view.isImmutableRows());
+            assertTrue("IMMUTABLE_ROWS property set incorrectly",
+                    table.isImmutableRows());
+            assertTrue("IMMUTABLE_ROWS property set incorrectly",
+                    view.isImmutableRows());
         }
     }
 
     @Test
-    public void testViewAddsPKColumnWhoseParentsLastPKIsVarLength() throws Exception {
+    public void testViewAddsPKColumnWhoseParentsLastPKIsVarLength()
+            throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 VARCHAR NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 VARCHAR NOT NULL, v1 DECIMAL, "
+                + "CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM "
+                + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
 
-        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
+        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY,"
+                + " k4 VARCHAR PRIMARY KEY, v2 INTEGER";
         try {
             conn.createStatement().execute(ddl);
-            fail("View cannot extend PK if parent's last PK is variable length. See https://issues.apache.org/jira/browse/PHOENIX-978.");
+            fail("View cannot extend PK if parent's last PK is variable length."
+                    + " See "
+                    + "https://issues.apache.org/jira/browse/PHOENIX-978.");
         } catch (SQLException e) {
-            assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(), e.getErrorCode());
+            assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(),
+                    e.getErrorCode());
         }
         String fullViewName2 = "V_" + generateUniqueName();
-        ddl = "CREATE VIEW " + fullViewName2 + " (k3 VARCHAR PRIMARY KEY)  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName2 + " (k3 VARCHAR PRIMARY KEY)  "
+                + "AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         try {
             conn.createStatement().execute(ddl);
         } catch (SQLException e) {
-            assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(), e.getErrorCode());
+            assertEquals(CANNOT_MODIFY_VIEW_PK.getErrorCode(),
+                    e.getErrorCode());
         }
     }
 
     @Test(expected=ColumnAlreadyExistsException.class)
     public void testViewAddsClashingPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL,"
+                + " CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM "
+                + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
 
-        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, k2 VARCHAR PRIMARY KEY, v2 INTEGER";
+        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, "
+                + "k2 VARCHAR PRIMARY KEY, v2 INTEGER";
         conn.createStatement().execute(ddl);
     }
 
     @Test
-    public void testQueryWithSeparateConnectionForViewOnTableThatHasIndex() throws SQLException {
+    public void testQueryWithSeparateConnectionForViewOnTableThatHasIndex()
+            throws SQLException {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Connection conn2 = DriverManager.getConnection(getUrl());
                 Statement s = conn.createStatement();
                 Statement s2 = conn2.createStatement()) {
-            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String tableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
             String indexName = generateUniqueName();
-            helpTestQueryForViewOnTableThatHasIndex(s, s2, tableName, viewName, indexName);
+            helpTestQueryForViewOnTableThatHasIndex(s, s2, tableName, viewName,
+                    indexName);
         }
     }
 
@@ -922,27 +1126,34 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     public void testQueryForViewOnTableThatHasIndex() throws SQLException {
         try (Connection conn = DriverManager.getConnection(getUrl());
                 Statement s = conn.createStatement()) {
-            String tableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-            String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
+            String tableName = SchemaUtil.getTableName(SCHEMA1,
+                    generateUniqueName());
+            String viewName = SchemaUtil.getTableName(SCHEMA2,
+                    generateUniqueName());
             String indexName = generateUniqueName();
-            helpTestQueryForViewOnTableThatHasIndex(s, s, tableName, viewName, indexName);
+            helpTestQueryForViewOnTableThatHasIndex(s, s, tableName, viewName,
+                    indexName);
         }
     }
 
-    private void helpTestQueryForViewOnTableThatHasIndex(Statement s1, Statement s2, String tableName, String viewName, String indexName)
+    private void helpTestQueryForViewOnTableThatHasIndex(Statement s1,
+            Statement s2, String tableName, String viewName, String indexName)
             throws SQLException {
         // Create a table
-        s1.execute("create table " + tableName + " (col1 varchar primary key, col2 varchar)");
+        s1.execute("create table " + tableName
+                + " (col1 varchar primary key, col2 varchar)");
 
         // Create a view on the table
-        s1.execute("create view " + viewName + " (col3 varchar) as select * from " + tableName);
+        s1.execute("create view " + viewName
+                + " (col3 varchar) as select * from " + tableName);
         s1.executeQuery("select * from " + viewName);
         // Create a index on the table
-        s1.execute("create index " + indexName + " ON " + tableName + " (col2)");
+        s1.execute("create index " + indexName + " ON " + tableName
+                + " (col2)");
 
-        try (ResultSet rs =
-                s2.executeQuery("explain select /*+ INDEX(" + viewName + " " + indexName
-                        + ") */ * from " + viewName + " where col2 = 'aaa'")) {
+        try (ResultSet rs = s2.executeQuery("explain select /*+ INDEX("
+                + viewName + " " + indexName + ") */ * from "
+                + viewName + " where col2 = 'aaa'")) {
             String explainPlan = QueryUtil.getExplainPlan(rs);
 
             // check if the query uses the index
@@ -954,31 +1165,39 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     public void testViewAndTableAndDropCascadeWithIndexes() throws Exception {
         // Setup - Tables and Views with Indexes
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String ddl = "CREATE TABLE " + fullTableName + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)";
         conn.createStatement().execute(ddl);
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
         String indexName1 = "I_" + generateUniqueName();
         String indexName2 = "I_" + generateUniqueName();
         String indexName3 = "I_" + generateUniqueName();
 
         ddl = "CREATE INDEX " + indexName1 + " ON " + fullTableName + " (v1)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 5";
+        ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
+                + fullTableName + " WHERE k > 5";
         conn.createStatement().execute(ddl);
         ddl = "CREATE INDEX " + indexName2 + " ON " + fullViewName1 + " (v2)";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE k > 10";
+        ddl = "CREATE VIEW " + fullViewName2 + " (v2 VARCHAR) AS SELECT * FROM "
+                + fullTableName + " WHERE k > 10";
         conn.createStatement().execute(ddl);
         ddl = "CREATE INDEX " + indexName3 + " ON " + fullViewName2 + " (v2)";
         conn.createStatement().execute(ddl);
 
         // Execute DROP...CASCADE
-        conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        conn.createStatement().execute("DROP TABLE " + fullTableName
+                + " CASCADE");
         runDropChildViewsTask();
 
-        // Validate Views were deleted - Try and delete child views, should throw TableNotFoundException
+        // Validate Views were deleted - Try and delete child views,
+        // should throw TableNotFoundException
         validateViewDoesNotExist(conn, fullViewName1);
         validateViewDoesNotExist(conn, fullViewName2);
     }
@@ -986,34 +1205,47 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     @Test
     public void testViewAddsNotNullPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL,"
+                + " CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM "
+                + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
 
         try {
-            ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR NOT NULL PRIMARY KEY"; 
+            ddl = "ALTER VIEW " + fullViewName
+                    + " ADD k3 VARCHAR NOT NULL PRIMARY KEY";
             conn.createStatement().execute(ddl);
             fail("can only add nullable PKs via ALTER VIEW/TABLE");
         } catch (SQLException e) {
-            assertEquals(NOT_NULLABLE_COLUMN_IN_ROW_KEY.getErrorCode(), e.getErrorCode());
+            assertEquals(NOT_NULLABLE_COLUMN_IN_ROW_KEY.getErrorCode(),
+                    e.getErrorCode());
         }
     }
 
     @Test
     public void testDisallowDropOfColumnOnParentTable() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String viewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String viewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, "
+                + "CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + viewName + "(v2 VARCHAR, v3 VARCHAR) "
+                + "AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
 
         try {
-            conn.createStatement().execute("ALTER TABLE " + fullTableName + " DROP COLUMN v1");
+            conn.createStatement().execute("ALTER TABLE " + fullTableName
+                    + " DROP COLUMN v1");
             fail();
         } catch (SQLException e) {
             assertEquals(CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
@@ -1023,71 +1255,94 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
     @Test
     public void testDisallowDropOfReferencedColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
-
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName1 = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+        String fullViewName2 = SchemaUtil.getTableName(SCHEMA3,
+                generateUniqueName());
+
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, "
+                + "CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName1 + "(v2 VARCHAR, v3 VARCHAR) AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName1 + "(v2 VARCHAR, v3 VARCHAR)"
+                + " AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM " + fullViewName1 + " WHERE v2 != 'foo'";
+        ddl = "CREATE VIEW " + fullViewName2 + " AS SELECT * FROM "
+                + fullViewName1 + " WHERE v2 != 'foo'";
         conn.createStatement().execute(ddl);
 
         try {
-            conn.createStatement().execute("ALTER VIEW " + fullViewName1 + " DROP COLUMN v1");
+            conn.createStatement().execute("ALTER VIEW " + fullViewName1
+                    + " DROP COLUMN v1");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL.getErrorCode(),
-                    e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL
+                            .getErrorCode(), e.getErrorCode());
         }
 
         try {
-            conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v1");
+            conn.createStatement().execute("ALTER VIEW " + fullViewName2
+                    + " DROP COLUMN v1");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL.getErrorCode(),
-                    e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL
+                            .getErrorCode(), e.getErrorCode());
         }
         try {
-            conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v2");
+            conn.createStatement().execute("ALTER VIEW " + fullViewName2
+                    + " DROP COLUMN v2");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL.getErrorCode(),
-                    e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL
+                            .getErrorCode(), e.getErrorCode());
         }
-        conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v3");
+        conn.createStatement().execute("ALTER VIEW " + fullViewName2
+                + " DROP COLUMN v3");
     }
 
     @Test
     public void testViewAddsPKColumn() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
         String viewName = generateUniqueName();
         String fullViewName = SchemaUtil.getTableName(SCHEMA2, viewName);
 
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, "
+                + "CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM " + fullTableName + " WHERE v1 = 1.0";
+        ddl = "CREATE VIEW " + fullViewName + "  AS SELECT * FROM "
+                + fullTableName + " WHERE v1 = 1.0";
         conn.createStatement().execute(ddl);
 
-        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, k4 VARCHAR PRIMARY KEY, v2 INTEGER";
+        ddl = "ALTER VIEW " + fullViewName + " ADD k3 VARCHAR PRIMARY KEY, "
+                + "k4 VARCHAR PRIMARY KEY, v2 INTEGER";
         conn.createStatement().execute(ddl);
 
         // assert PK metadata
-        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, SCHEMA2, viewName);
+        ResultSet rs = conn.getMetaData().getPrimaryKeys(null, SCHEMA2,
+                viewName);
         assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
     }
 
     @Test
     public void testCreateViewDefinesPKConstraint() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
-        String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
-        String fullViewName = SchemaUtil.getTableName(SCHEMA2, generateUniqueName());
-
-        String ddl = "CREATE TABLE " + fullTableName + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL, CONSTRAINT pk PRIMARY KEY (k1, k2))";
+        String fullTableName = SchemaUtil.getTableName(SCHEMA1,
+                generateUniqueName());
+        String fullViewName = SchemaUtil.getTableName(SCHEMA2,
+                generateUniqueName());
+
+        String ddl = "CREATE TABLE " + fullTableName
+                + " (k1 INTEGER NOT NULL, k2 INTEGER NOT NULL, v1 DECIMAL,"
+                + " CONSTRAINT pk PRIMARY KEY (k1, k2))";
         conn.createStatement().execute(ddl);
-        ddl = "CREATE VIEW " + fullViewName + "(v2 VARCHAR, k3 VARCHAR, k4 INTEGER NOT NULL, CONSTRAINT PKVEW PRIMARY KEY (k3, k4)) AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
+        ddl = "CREATE VIEW " + fullViewName + "(v2 VARCHAR, k3 VARCHAR, "
+                + "k4 INTEGER NOT NULL, CONSTRAINT PKVEW PRIMARY KEY (k3, k4))"
+                + " AS SELECT * FROM " + fullTableName + " WHERE K1 = 1";
         conn.createStatement().execute(ddl);
 
         PhoenixRuntime.getTableNoCache(conn, fullViewName);
@@ -1100,7 +1355,8 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         assertPKs(rs, new String[] {"K1", "K2", "K3", "K4"});
     }
 
-    private void assertPKs(ResultSet rs, String[] expectedPKs) throws SQLException {
+    private void assertPKs(ResultSet rs, String[] expectedPKs)
+            throws SQLException {
         List<String> pkCols = newArrayListWithExpectedSize(expectedPKs.length);
         while (rs.next()) {
             pkCols.add(rs.getString("COLUMN_NAME"));
@@ -1109,11 +1365,13 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         assertArrayEquals(expectedPKs, actualPKs);
     }
 
-    private void validateViewDoesNotExist(Connection conn, String fullViewName)    throws SQLException {
+    private void validateViewDoesNotExist(Connection conn, String fullViewName)
+            throws SQLException {
         try {
             String ddl1 = "DROP VIEW " + fullViewName;
             conn.createStatement().execute(ddl1);
-            fail("View " + fullViewName + " should have been deleted when parent was dropped");
+            fail("View " + fullViewName + " should have been deleted when "
+                    + "parent was dropped");
         } catch (TableNotFoundException e) {
             //Expected
         }
@@ -1129,8 +1387,10 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
             }
         };
         List<PColumn> colList = table.getColumns();
-        Collection<PColumn> filteredCols = Collections2.filter(colList, predicate);
+        Collection<PColumn> filteredCols = Collections2.filter(colList,
+                predicate);
         assertEquals(1, filteredCols.size());
-        assertEquals(prefix + "V3", filteredCols.iterator().next().getName().getString());
+        assertEquals(prefix + "V3", filteredCols.iterator().next().getName()
+                .getString());
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
index 0a45981..4e20997 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataClient.java
@@ -2610,6 +2610,9 @@ public class MetaDataClient {
 
             Map<String, Integer> changedCqCounters = new HashMap<>(colDefs.size());
             boolean wasPKDefined = false;
+            // Keep track of all columns that are newly added to a view
+            Set<Integer> viewNewColumnPositions =
+                    Sets.newHashSetWithExpectedSize(colDefs.size());
             for (ColumnDef colDef : colDefs) {
                 rowTimeStampColumnAlreadyFound = checkAndValidateRowTimestampCol(colDef, pkConstraint, rowTimeStampColumnAlreadyFound, tableType);
                 if (colDef.isPK()) { // i.e. the column is declared as CREATE TABLE COLNAME DATATYPE PRIMARY KEY...
@@ -2675,6 +2678,8 @@ public class MetaDataClient {
                 }
                 if (columns.put(column, column) != null) {
                     throw new ColumnAlreadyExistsException(schemaName, tableName, column.getName().getString());
+                } else if (tableType == VIEW) {
+                    viewNewColumnPositions.add(column.getPosition());
                 }
                 if ((colDef.getDataType() == PVarbinary.INSTANCE || colDef.getDataType().isArrayType())
                         && SchemaUtil.isPKColumn(column)
@@ -2866,13 +2871,24 @@ public class MetaDataClient {
                         }
 
                         // if the base table column is referenced in the view
-                        if (isViewColumnReferenced.get(columnPosition)) {
-                            // acquire the mutex using the global physical table name to
-                            // prevent this column from being dropped while the view is being created
-                            boolean acquiredMutex = writeCell(null, parentPhysicalSchemaName, parentPhysicalTableName,
+                        // or if we are adding a new column during view creation
+                        if (isViewColumnReferenced.get(columnPosition) ||
+                                viewNewColumnPositions.contains(
+                                        columnPosition)) {
+                            // acquire the mutex using the global physical table
+                            // name to prevent this column from being dropped
+                            // while the view is being created or to prevent
+                            // a conflicting column from being added to a parent
+                            // in case the view creation adds new columns
+                            boolean acquiredMutex = writeCell(
+                                    null,
+                                    parentPhysicalSchemaName,
+                                    parentPhysicalTableName,
                                     column.toString());
                             if (!acquiredMutex) {
-                                throw new ConcurrentTableMutationException(parentPhysicalSchemaName, parentPhysicalTableName);
+                                throw new ConcurrentTableMutationException(
+                                        parentPhysicalSchemaName,
+                                        parentPhysicalTableName);
                             }
                             acquiredColumnMutexSet.add(column.toString());
                         }