You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by td...@apache.org on 2019/07/19 23:29:50 UTC

[phoenix] branch 4.x-HBase-1.4 updated: PHOENIX-4893 Move parent column combining logic of view and view indexes from server to client

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

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


The following commit(s) were added to refs/heads/4.x-HBase-1.4 by this push:
     new 47d8c9c  PHOENIX-4893 Move parent column combining logic of view and view indexes from server to client
47d8c9c is described below

commit 47d8c9cfc0586c075b4ba7e46019fe5ec0d6664e
Author: Thomas D'Silva <td...@apache.org>
AuthorDate: Tue Nov 20 12:10:05 2018 -0800

    PHOENIX-4893 Move parent column combining logic of view and view indexes from server to client
---
 .../org/apache/phoenix/end2end/AlterTableIT.java   |    1 -
 .../apache/phoenix/end2end/AppendOnlySchemaIT.java |   13 +-
 .../phoenix/end2end/DropTableWithViewsIT.java      |    8 +-
 .../phoenix/end2end/MetaDataEndpointImplIT.java    |   12 +-
 .../phoenix/end2end/ParallelStatsEnabledIT.java    |   13 +
 .../phoenix/end2end/TenantSpecificTablesDDLIT.java |   31 +-
 .../org/apache/phoenix/end2end/ViewMetadataIT.java |   38 +-
 .../apache/phoenix/end2end/index/DropColumnIT.java |    9 +-
 .../index/InvalidIndexStateClientSideIT.java       |    2 -
 .../java/org/apache/phoenix/rpc/UpdateCacheIT.java |    6 +-
 .../phoenix/coprocessor/AddColumnMutator.java      |  427 ++++
 .../apache/phoenix/coprocessor/ColumnMutator.java  |   63 +
 .../phoenix/coprocessor/DropColumnMutator.java     |  277 +++
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  | 2564 ++++++--------------
 .../org/apache/phoenix/coprocessor/ViewFinder.java |  144 --
 .../phoenix/coprocessor/WhereConstantParser.java   |    2 +-
 .../coprocessor/generated/MetaDataProtos.java      | 1321 +++++-----
 .../coprocessor/generated/PTableProtos.java        |  198 +-
 .../coprocessor/tasks/DropChildViewsTask.java      |   11 +-
 .../apache/phoenix/exception/SQLExceptionCode.java |    3 +
 .../phoenix/query/ConnectionQueryServices.java     |   26 +-
 .../phoenix/query/ConnectionQueryServicesImpl.java |   46 +-
 .../query/ConnectionlessQueryServicesImpl.java     |   21 +-
 .../query/DelegateConnectionQueryServices.java     |   32 +-
 .../org/apache/phoenix/schema/MetaDataClient.java  |  639 +++--
 .../apache/phoenix/schema/MetaDataSplitPolicy.java |    6 +-
 .../java/org/apache/phoenix/schema/PTableImpl.java |   28 +-
 .../schema/ParentTableNotFoundException.java       |   47 -
 .../java/org/apache/phoenix/util/MetaDataUtil.java |   30 +-
 .../java/org/apache/phoenix/util/SchemaUtil.java   |    3 +-
 .../TableViewFinderResult.java                     |    3 +-
 .../java/org/apache/phoenix/util/UpgradeUtil.java  |    6 +-
 .../java/org/apache/phoenix/util/ViewUtil.java     |  578 +++++
 .../apache/phoenix/compile/ViewCompilerTest.java   |   14 +-
 .../java/org/apache/phoenix/query/BaseTest.java    |    1 +
 phoenix-protocol/src/main/MetaDataService.proto    |    7 +-
 phoenix-protocol/src/main/PTable.proto             |    2 +
 37 files changed, 3606 insertions(+), 3026 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
index 669f1c1..163be71 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AlterTableIT.java
@@ -599,7 +599,6 @@ public class AlterTableIT extends ParallelStatsDisabledIT {
 
     @Test
     public void testDropColumnsWithImutability() throws Exception {
-
         Properties props = PropertiesUtil.deepCopy(TEST_PROPERTIES);
         Connection conn = DriverManager.getConnection(getUrl(), props);
         conn.setAutoCommit(false);
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
index e1c56ea..dc06f62 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/AppendOnlySchemaIT.java
@@ -109,14 +109,14 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT {
             }
             
             // verify getTable rpcs
-            verify(connectionQueryServices, sameClient ? never() : times(1)).getTable(
-                (PName) isNull(), eq(new byte[0]), eq(Bytes.toBytes(viewName)), anyLong(),
-                anyLong(), eq(false), eq(false), (PTable) isNull());
+            verify(connectionQueryServices, sameClient ? never() : times(1))
+                    .getTable((PName) isNull(), eq(new byte[0]),
+                            eq(Bytes.toBytes(viewName)), anyLong(), anyLong());
             
             // verify no create table rpcs
             verify(connectionQueryServices, never()).createTable(anyListOf(Mutation.class),
                 any(byte[].class), any(PTableType.class), anyMap(), anyList(), any(byte[][].class),
-                eq(false), eq(false), eq(false));
+                eq(false), eq(false), eq(false), any(PTable.class));
             reset(connectionQueryServices);
             
             // execute alter table ddl that adds the same column
@@ -135,7 +135,10 @@ public class AppendOnlySchemaIT extends ParallelStatsDisabledIT {
             
             // if not verify exists is true one call to add column table with empty mutation list (which does not make a rpc) 
             // else verify no add column calls
-            verify(connectionQueryServices, notExists ? times(1) : never() ).addColumn(eq(Collections.<Mutation>emptyList()), any(PTable.class), anyMap(), anySetOf(String.class), anyListOf(PColumn.class));
+            verify(connectionQueryServices, notExists ? times(1) : never() )
+                    .addColumn(eq(Collections.<Mutation>emptyList()), any(PTable.class),
+                            any(PTable.class), anyMap(), anySetOf(String.class),
+                            anyListOf(PColumn.class));
 
             // upsert one row
             conn2.createStatement().execute("UPSERT INTO " + viewName + "(hostName, metricVal) VALUES('host2', 2.0)");
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
index bf2633a..5836c56 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/DropTableWithViewsIT.java
@@ -19,7 +19,6 @@ package org.apache.phoenix.end2end;
 
 import static org.apache.phoenix.util.PhoenixRuntime.TENANT_ID_ATTRIB;
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertFalse;
 
@@ -27,16 +26,15 @@ import java.sql.Connection;
 import java.sql.DriverManager;
 import java.sql.ResultSet;
 import java.sql.SQLException;
-import java.sql.Timestamp;
 import java.util.Arrays;
 import java.util.Collection;
 
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
-import org.apache.phoenix.coprocessor.TableViewFinderResult;
+import org.apache.phoenix.util.TableViewFinderResult;
 import org.apache.phoenix.coprocessor.TaskRegionObserver;
-import org.apache.phoenix.coprocessor.ViewFinder;
+import org.apache.phoenix.util.ViewUtil;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 
 import org.apache.phoenix.query.QueryServicesOptions;
@@ -144,7 +142,7 @@ public class DropTableWithViewsIT extends SplitSystemCatalogIT {
             // Views should be dropped by now
             TableName linkTable = TableName.valueOf(PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES);
             TableViewFinderResult childViewsResult = new TableViewFinderResult();
-            ViewFinder.findAllRelatives(getUtility().getConnection().getTable(linkTable),
+            ViewUtil.findAllRelatives(getUtility().getConnection().getTable(linkTable),
                     HConstants.EMPTY_BYTE_ARRAY,
                     SchemaUtil.getSchemaNameFromFullName(baseTable).getBytes(),
                     SchemaUtil.getTableNameFromFullName(baseTable).getBytes(),
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java
index f14af9e..21ab6f8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/MetaDataEndpointImplIT.java
@@ -15,9 +15,8 @@ import java.util.Map;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.HTable;
-import org.apache.phoenix.coprocessor.TableViewFinderResult;
-import org.apache.phoenix.coprocessor.ViewFinder;
-import org.apache.phoenix.end2end.ParallelStatsDisabledIT;
+import org.apache.phoenix.util.TableViewFinderResult;
+import org.apache.phoenix.util.ViewUtil;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.schema.PColumn;
@@ -82,13 +81,14 @@ public class MetaDataEndpointImplIT extends ParallelStatsDisabledIT {
         System.err.println(rightChildTable);
 
         TableViewFinderResult childViews = new TableViewFinderResult();
-        ViewFinder.findAllRelatives(getTable(linkTable), HConstants.EMPTY_BYTE_ARRAY, table.getSchemaName().getBytes(),
-            table.getTableName().getBytes(), PTable.LinkType.CHILD_TABLE, childViews);
+        ViewUtil.findAllRelatives(getTable(linkTable), HConstants.EMPTY_BYTE_ARRAY,
+                table.getSchemaName().getBytes(), table.getTableName().getBytes(),
+                PTable.LinkType.CHILD_TABLE, childViews);
         assertEquals(3, childViews.getLinks().size());
 
         PTable childMostView = PhoenixRuntime.getTable(conn , leftGrandChild.toUpperCase());
         TableViewFinderResult parentViews = new TableViewFinderResult();
-        ViewFinder
+        ViewUtil
             .findAllRelatives(getTable(catalogTable), HConstants.EMPTY_BYTE_ARRAY, childMostView.getSchemaName().getBytes(),
                 childMostView.getTableName().getBytes(), PTable.LinkType.PARENT_TABLE, parentViews);
         // returns back everything but the parent table - should only return back the left_child and not the right child
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
index a383ea1..d890ebd 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ParallelStatsEnabledIT.java
@@ -20,6 +20,9 @@ package org.apache.phoenix.end2end;
 
 import java.util.Map;
 
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.phoenix.coprocessor.TaskRegionObserver;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.BaseTest;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.util.ReadOnlyProps;
@@ -37,6 +40,8 @@ import com.google.common.collect.Maps;
 @Category(ParallelStatsEnabledTest.class)
 public abstract class ParallelStatsEnabledIT extends BaseTest {
 
+    protected static RegionCoprocessorEnvironment TaskRegionEnvironment;
+
     @BeforeClass
     public static final void doSetup() throws Exception {
         Map<String, String> props = Maps.newHashMapWithExpectedSize(1);
@@ -45,6 +50,14 @@ public abstract class ParallelStatsEnabledIT extends BaseTest {
         props.put(QueryServices.MAX_SERVER_METADATA_CACHE_TIME_TO_LIVE_MS_ATTRIB, Long.toString(5));
         props.put(QueryServices.USE_STATS_FOR_PARALLELIZATION, Boolean.toString(true));
         setUpTestDriver(new ReadOnlyProps(props.entrySet().iterator()));
+
+        TaskRegionEnvironment =
+                (RegionCoprocessorEnvironment)getUtility()
+                        .getRSForFirstRegionInTable(
+                                PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
+                        .getOnlineRegions(PhoenixDatabaseMetaData.SYSTEM_TASK_HBASE_TABLE_NAME)
+                        .get(0).getCoprocessorHost()
+                        .findCoprocessorEnvironment(TaskRegionObserver.class.getName());
     }
 
     @AfterClass
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
index 85c9128..e147225 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TenantSpecificTablesDDLIT.java
@@ -46,9 +46,12 @@ import java.util.Properties;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.phoenix.coprocessor.TaskRegionObserver;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PTableType;
@@ -58,11 +61,12 @@ import org.apache.phoenix.util.PhoenixRuntime;
 import org.apache.phoenix.util.PropertiesUtil;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.StringUtil;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
 
 public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
-    
+
     @Test
     public void testCreateTenantSpecificTable() throws Exception {
         // ensure we didn't create a physical HBase table for the tenant-specific table
@@ -372,19 +376,25 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
             
 			// Drop Parent Table 
 			conn.createStatement().executeUpdate("DROP TABLE " + PARENT_TABLE_NAME + " CASCADE");
-		  
+            TaskRegionObserver.SelfHealingTask task = new TaskRegionObserver.SelfHealingTask
+                    (TaskRegionEnvironment,
+                            QueryServicesOptions.DEFAULT_TASK_HANDLING_MAX_INTERVAL_MS);
+            task.run();
+
 			// Validate Tenant Views are dropped
 			connTenant1 = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL, props);
 	        validateTenantViewIsDropped(connTenant1);
 			connTenant2 = DriverManager.getConnection(PHOENIX_JDBC_TENANT_SPECIFIC_URL2, props);
 	        validateTenantViewIsDropped(connTenant2);
 	        
-	        // TODO uncomment after PHOENIX-4764 is implemented
 	        // Validate Tenant Metadata is gone for the Tenant Table TENANT_TABLE_NAME
-//            rs = meta.getTables(null, "", StringUtil.escapeLike(TENANT_TABLE_NAME), new String[] {PTableType.VIEW.getValue().getString()});
-//            assertFalse(rs.next());
-//            rs = meta.getTables(null, "", StringUtil.escapeLike(tenantTable2), new String[] {PTableType.VIEW.getValue().getString()});
-//            assertFalse(rs.next());
+            rs = meta.getTables(null, "",
+                    StringUtil.escapeLike(TENANT_TABLE_NAME),
+                    new String[] {PTableType.VIEW.getValue().getString()});
+            assertFalse(rs.next());
+            rs = meta.getTables(null, "", StringUtil.escapeLike(tenantTable2)
+                    , new String[] {PTableType.VIEW.getValue().getString()});
+            assertFalse(rs.next());
             
             rs = meta.getTables(null, "", StringUtil.escapeLike(TENANT_TABLE_NAME_NO_TENANT_TYPE_ID), new String[] {PTableType.VIEW.getValue().getString()});
             assertTrue(rs.next());
@@ -406,6 +416,13 @@ public class TenantSpecificTablesDDLIT extends BaseTenantSpecificTablesIT {
     }
 
 	private void validateTenantViewIsDropped(Connection connTenant)	throws SQLException {
+        try {
+            PhoenixRuntime.getTableNoCache(connTenant, TENANT_TABLE_NAME);
+            fail("Tenant specific view " + TENANT_TABLE_NAME
+                    + " should have been dropped when parent was dropped");
+        } catch (TableNotFoundException e) {
+            //Expected
+        }
 		// Try and drop tenant view, should throw TableNotFoundException
 		try {
 			String ddl = "DROP VIEW " + TENANT_TABLE_NAME;
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 1419b06..3d46393 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
@@ -41,11 +41,15 @@ import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Admin;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
 import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost;
+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.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
 import org.apache.phoenix.schema.ColumnAlreadyExistsException;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PTable;
@@ -64,6 +68,9 @@ import com.google.common.collect.Collections2;
 import com.google.common.collect.Maps;
 
 public class ViewMetadataIT extends SplitSystemCatalogIT {
+
+    private static RegionCoprocessorEnvironment TaskRegionEnvironment;
+
     @BeforeClass
     public static void doSetup() throws Exception {
         NUM_SLAVES_BASE = 6;
@@ -72,13 +79,21 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         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());
+                TestMetaDataRegionObserver.class.getName());
         serverProps.put("hbase.coprocessor.abortonerror", "false");
         setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(props.entrySet().iterator()));
         // Split SYSTEM.CATALOG once after the mini-cluster is started
         if (splitSystemCatalog) {
             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());
     }
 
     @Test
@@ -232,6 +247,7 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
         // drop table cascade should succeed
         conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        runDropChildViewsTask();
 
         validateViewDoesNotExist(conn, fullViewName1);
         validateViewDoesNotExist(conn, fullViewName2);
@@ -251,6 +267,13 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
         }
     }
 
+    private 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);
+        task.run();
+    }
+
     @Test
     public void testRecreateIndexWhoseAncestorWasDropped() throws Exception {
         Connection conn = DriverManager.getConnection(getUrl());
@@ -278,6 +301,7 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
         // drop table cascade should succeed
         conn.createStatement().execute("DROP TABLE " + fullTableName1 + " CASCADE");
+        runDropChildViewsTask();
 
         // should be able to reuse the index name 
         ddl = "CREATE INDEX " + indexName + " on " + fullTableName2 + "(v3)";
@@ -305,6 +329,7 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
         // drop table cascade should succeed
         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";
@@ -404,6 +429,7 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
         // drop table cascade should succeed
         conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        runDropChildViewsTask();
 
         validateViewDoesNotExist(conn, fullViewName1);
         validateViewDoesNotExist(conn, fullViewName2);
@@ -545,6 +571,7 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
 
         // Execute DROP...CASCADE
         conn.createStatement().execute("DROP TABLE " + fullTableName + " CASCADE");
+        runDropChildViewsTask();
 
         // Validate Views were deleted - Try and delete child views, should throw TableNotFoundException
         validateViewDoesNotExist(conn, fullViewName1);
@@ -606,20 +633,23 @@ public class ViewMetadataIT extends SplitSystemCatalogIT {
             conn.createStatement().execute("ALTER VIEW " + fullViewName1 + " DROP COLUMN v1");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL.getErrorCode(),
+                    e.getErrorCode());
         }
 
         try {
             conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v1");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL.getErrorCode(),
+                    e.getErrorCode());
         }
         try {
             conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v2");
             fail();
         } catch (SQLException e) {
-            assertEquals(SQLExceptionCode.CANNOT_MUTATE_TABLE.getErrorCode(), e.getErrorCode());
+            assertEquals(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL.getErrorCode(),
+                    e.getErrorCode());
         }
         conn.createStatement().execute("ALTER VIEW " + fullViewName2 + " DROP COLUMN v3");
     }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
index 3ed2f11..9ab625f 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/DropColumnIT.java
@@ -343,7 +343,8 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
             PhoenixConnection pconn = conn.unwrap(PhoenixConnection.class);
             PTable dataTable = pconn.getTable(new PTableKey(null, dataTableFullName));
             assertEquals("Unexpected number of indexes ", 3, dataTable.getIndexes().size());
-            PName localIndexTablePhysicalName = dataTable.getIndexes().get(1).getPhysicalName();
+            byte[] indexTablePhysicalName = indexTableName.getBytes();
+            byte[] localIndexTablePhysicalName = dataTableFullName.getBytes();
             
             // drop v2 which causes the regular index and first local index to be dropped
             conn.createStatement().execute(
@@ -379,13 +380,15 @@ public class DropColumnIT extends ParallelStatsDisabledIT {
             }
             
             // verify that the local index physical table was *not* dropped
-            conn.unwrap(PhoenixConnection.class).getQueryServices().getTableDescriptor(localIndexTablePhysicalName.getBytes());
+            conn.unwrap(PhoenixConnection.class).getQueryServices()
+                    .getTableDescriptor(localIndexTablePhysicalName);
             PTable localIndex2 = conn.unwrap(PhoenixConnection.class).getTable(new PTableKey(null, localIndexTableName2));
             
             // there should be a single row belonging to localIndexTableName2 
             Scan scan = new Scan();
             scan.addFamily(QueryConstants.DEFAULT_LOCAL_INDEX_COLUMN_FAMILY_BYTES);
-            HTable table = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices().getTable(localIndexTablePhysicalName.getBytes());
+            HTable table = (HTable) conn.unwrap(PhoenixConnection.class).getQueryServices()
+                    .getTable(localIndexTablePhysicalName);
             ResultScanner results = table.getScanner(scan);
             Result result = results.next();
             assertNotNull(result);
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/InvalidIndexStateClientSideIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/InvalidIndexStateClientSideIT.java
index aea9759..60a07a7 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/InvalidIndexStateClientSideIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/index/InvalidIndexStateClientSideIT.java
@@ -111,8 +111,6 @@ public class InvalidIndexStateClientSideIT extends ParallelStatsDisabledIT {
                         builder.setClientTimestamp(resolvedTimestamp);
                         builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION,
                             13, PHOENIX_PATCH_NUMBER));
-                        builder.setSkipAddingParentColumns(false);
-                        builder.setSkipAddingIndexes(false);
                         instance.getTable(controller, builder.build(), rpcCallback);
                         if (controller.getFailedOn() != null) {
                             throw controller.getFailedOn();
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
index a1bdad7..e21b933 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/rpc/UpdateCacheIT.java
@@ -177,7 +177,7 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
         // Even the indexes should now have the modified value of UPDATE_CACHE_FREQUENCY
         // Note that when we query the base table, during query plan generation, we make 2 getTable
         // requests (to retrieve the base table) for each index of the base table
-        helpTestUpdateCache(fullTableName, new int[] {1, 15}, false);
+        helpTestUpdateCache(fullTableName, new int[] {1, 18}, false);
         helpTestUpdateCache(INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + localIndex,
                 new int[] {3}, true);
         helpTestUpdateCache(INDEX_DATA_SCHEMA + QueryConstants.NAME_SEPARATOR + globalIndex,
@@ -209,7 +209,7 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
                 // verify only 0 or 1 rpc to fetch table metadata,
                 verify(connectionQueryServices, times(numUpsertRpcs)).getTable((PName) isNull(),
                         eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)),
-                        anyLong(), anyLong(), eq(false), eq(false), (PTable)isNull());
+                        anyLong(), anyLong());
                 reset(connectionQueryServices);
             }
             validateSelectRowKeyCols(conn, selectSql, skipUpsertForIndexes);
@@ -223,7 +223,7 @@ public class UpdateCacheIT extends ParallelStatsDisabledIT {
             int numRpcs = skipUpsertForIndexes ? expectedRPCs[0] : expectedRPCs[1];
             verify(connectionQueryServices, times(numRpcs)).getTable((PName) isNull(),
                 eq(PVarchar.INSTANCE.toBytes(schemaName)), eq(PVarchar.INSTANCE.toBytes(tableName)),
-                anyLong(), anyLong(), eq(false), eq(false), (PTable)isNull());
+                anyLong(), anyLong());
 		}
         finally {
         	conn.close();
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/AddColumnMutator.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/AddColumnMutator.java
new file mode 100644
index 0000000..71b4827
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/AddColumnMutator.java
@@ -0,0 +1,427 @@
+/*
+ * 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.coprocessor;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.hbase.Cell;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SortOrder;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.schema.types.PSmallint;
+import org.apache.phoenix.util.ByteUtil;
+import org.apache.phoenix.util.EncodedColumnsUtil;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.apache.phoenix.util.UpgradeUtil;
+import org.apache.phoenix.util.ViewUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DATA_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DECIMAL_DIGITS_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.MULTI_TENANT_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID_INDEX;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+public class AddColumnMutator implements ColumnMutator {
+
+    private static final Logger logger = LoggerFactory.getLogger(AddColumnMutator.class);
+
+    private int getInteger(Put p, byte[] family, byte[] qualifier) {
+        List<Cell> cells = p.get(family, qualifier);
+        if (cells != null && cells.size() > 0) {
+            Cell cell = cells.get(0);
+            return (Integer)PInteger.INSTANCE.toObject(cell.getValueArray(),
+                    cell.getValueOffset(), cell.getValueLength());
+        }
+        return 0;
+    }
+
+    @Override
+    public MutateColumnType getMutateColumnType() {
+        return MutateColumnType.ADD_COLUMN;
+    }
+
+    /**
+     * Validates that we can add the column to the base table by ensuring that if the same column
+     * already exists in any child view all of the column properties match
+     */
+    @Override
+    public MetaDataMutationResult validateWithChildViews(PTable table, List<PTable> childViews,
+                                                         List<Mutation> tableMetadata,
+                                                         byte[] schemaName, byte[] tableName)
+            throws SQLException {
+        // Disallow if trying to switch tenancy of a table that has views
+        if (!childViews.isEmpty() && switchAttribute(table.isMultiTenant(),
+                tableMetadata, MULTI_TENANT_BYTES)) {
+            return new MetaDataMutationResult(MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION
+                    , EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
+
+        List<Put> columnPutsForBaseTable =
+                Lists.newArrayListWithExpectedSize(tableMetadata.size());
+        boolean salted = table.getBucketNum()!=null;
+        // Isolate the puts relevant to adding columns
+        for (Mutation m : tableMetadata) {
+            if (m instanceof Put) {
+                byte[][] rkmd = new byte[5][];
+                int pkCount = getVarChars(m.getRow(), rkmd);
+                // check if this put is for adding a column
+                if (pkCount > COLUMN_NAME_INDEX && rkmd[COLUMN_NAME_INDEX] != null
+                        && rkmd[COLUMN_NAME_INDEX].length > 0
+                        && Bytes.compareTo(schemaName, rkmd[SCHEMA_NAME_INDEX]) == 0
+                        && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) == 0) {
+                    columnPutsForBaseTable.add((Put)m);
+                }
+            }
+        }
+        for (PTable view : childViews) {
+            /*
+             * Disallow adding columns to a base table with APPEND_ONLY_SCHEMA since this
+             * creates a gap in the column positions for every view (PHOENIX-4737).
+             */
+            if (!columnPutsForBaseTable.isEmpty() && view.isAppendOnlySchema()) {
+                return new MetaDataProtocol.MetaDataMutationResult(
+                        MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                        EnvironmentEdgeManager.currentTimeMillis(), null);
+            }
+
+            // add the new columns to the child view
+            List<PColumn> viewPkCols = new ArrayList<>(view.getPKColumns());
+            // remove salted column
+            if (salted) {
+                viewPkCols.remove(0);
+            }
+            // remove pk columns that are present in the parent
+            viewPkCols.removeAll(table.getPKColumns());
+            boolean addedPkColumn = false;
+            for (Put columnToBeAdded : columnPutsForBaseTable) {
+                PColumn existingViewColumn = null;
+                byte[][] rkmd = new byte[5][];
+                getVarChars(columnToBeAdded.getRow(), rkmd);
+                String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
+                String columnFamily =
+                        rkmd[FAMILY_NAME_INDEX] == null ? null
+                                : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
+                try {
+                    existingViewColumn =
+                            columnFamily == null ? view.getColumnForColumnName(columnName)
+                                    : view.getColumnFamily(columnFamily)
+                                    .getPColumnForColumnName(columnName);
+                } catch (ColumnFamilyNotFoundException e) {
+                    // ignore since it means that the column family is not present for the column to
+                    // be added.
+                } catch (ColumnNotFoundException e) {
+                    // ignore since it means the column is not present in the view
+                }
+
+                boolean isCurrColumnToBeAddPkCol = columnFamily == null;
+                addedPkColumn |= isCurrColumnToBeAddPkCol;
+                if (existingViewColumn != null) {
+                    if (EncodedColumnsUtil.usesEncodedColumnNames(table)
+                            && !SchemaUtil.isPKColumn(existingViewColumn)) {
+                        /*
+                         * If the column already exists in a view, then we cannot add the column to
+                         * the base table. The reason is subtle and is as follows: consider the case
+                         * where a table has two views where both the views have the same key value
+                         * column KV. Now, we dole out encoded column qualifiers for key value
+                         * columns in views by using the counters stored in the base physical table.
+                         * So the KV column can have different column qualifiers for the two views.
+                         * For example, 11 for VIEW1 and 12 for VIEW2. This naturally extends to
+                         * rows being inserted using the two views having different column
+                         * qualifiers for the column named KV. Now, when an attempt is made to add
+                         * column KV to the base table, we cannot decide which column qualifier
+                         * should that column be assigned. It cannot be a number different than 11
+                         * or 12 since a query like SELECT KV FROM BASETABLE would return null for
+                         * KV which is incorrect since column KV is present in rows inserted from
+                         * the two views. We cannot use 11 or 12 either because we will then
+                         * incorrectly return value of KV column inserted using only one view.
+                         */
+                        return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.
+                                MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+                    // Validate data type is same
+                    int baseColumnDataType =
+                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES);
+                    if (baseColumnDataType != existingViewColumn.getDataType().getSqlType()) {
+                        return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.
+                                MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+
+                    // Validate max length is same
+                    int maxLength =
+                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, COLUMN_SIZE_BYTES);
+                    int existingMaxLength =
+                            existingViewColumn.getMaxLength() == null ? 0
+                                    : existingViewColumn.getMaxLength();
+                    if (maxLength != existingMaxLength) {
+                        return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.
+                                MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+
+                    // Validate scale is same
+                    int scale =
+                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
+                    int existingScale =
+                            existingViewColumn.getScale() == null ? 0
+                                    : existingViewColumn.getScale();
+                    if (scale != existingScale) {
+                        return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.
+                                MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+
+                    // Validate sort order is same
+                    int sortOrder =
+                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, SORT_ORDER_BYTES);
+                    if (sortOrder != existingViewColumn.getSortOrder().getSystemValue()) {
+                        return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.
+                                MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+
+                    // if the column to be added to the base table is a pk column, then we need to
+                    // validate that the key slot position is the same
+                    if (isCurrColumnToBeAddPkCol) {
+                        List<Cell> keySeqCells =
+                                columnToBeAdded.get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                        PhoenixDatabaseMetaData.KEY_SEQ_BYTES);
+                        if (keySeqCells != null && keySeqCells.size() > 0) {
+                            Cell cell = keySeqCells.get(0);
+                            int keySeq =
+                                    PSmallint.INSTANCE.getCodec().decodeInt(cell.getValueArray(),
+                                            cell.getValueOffset(), SortOrder.getDefault());
+                            // we need to take into account the columns inherited from the base table
+                            // if the table is salted we don't include the salted column (which is
+                            // present in getPKColumns())
+                            int pkPosition = SchemaUtil.getPKPosition(view, existingViewColumn)
+                                    + 1 - (salted ? 1 : 0);
+                            if (pkPosition != keySeq) {
+                                return new MetaDataProtocol.MetaDataMutationResult(
+                                        MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                        EnvironmentEdgeManager.currentTimeMillis(),
+                                        table);
+                            }
+                        }
+                    }
+                }
+                if (existingViewColumn!=null && isCurrColumnToBeAddPkCol) {
+                    viewPkCols.remove(existingViewColumn);
+                }
+            }
+            /*
+             * Allow adding a pk columns to base table : 1. if all the view pk columns are exactly
+             * the same as the base table pk columns 2. if we are adding all the existing view pk
+             * columns to the base table
+             */
+            if (addedPkColumn && !viewPkCols.isEmpty()) {
+                return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.MutationCode
+                        .UNALLOWED_TABLE_MUTATION,
+                        EnvironmentEdgeManager.currentTimeMillis(), table);
+            }
+        }
+        return null;
+    }
+
+    private boolean switchAttribute(boolean currAttribute, List<Mutation> tableMetaData,
+                                    byte[] attrQualifier) {
+        for (Mutation m : tableMetaData) {
+            if (m instanceof Put) {
+                Put p = (Put)m;
+                List<Cell> cells = p.get(TABLE_FAMILY_BYTES, attrQualifier);
+                if (cells != null && cells.size() > 0) {
+                    Cell cell = cells.get(0);
+                    boolean newAttribute = (boolean)PBoolean.INSTANCE.toObject(cell.getValueArray(),
+                            cell.getValueOffset(), cell.getValueLength());
+                    return currAttribute != newAttribute;
+                }
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public MetaDataMutationResult validateAndAddMetadata(PTable table, byte[][] rowKeyMetaData,
+                                                         List<Mutation> tableMetaData,
+                                                         Region region,
+                                                         List<ImmutableBytesPtr> invalidateList,
+                                                         List<Region.RowLock> locks,
+                                                         long clientTimeStamp) {
+        byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
+        byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
+        byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
+        PTableType type = table.getType();
+        byte[] tableHeaderRowKey = SchemaUtil.getTableKey(tenantId,
+                schemaName, tableName);
+        List<Mutation> additionalTableMetadataMutations =
+                Lists.newArrayListWithExpectedSize(2);
+
+        boolean addingCol = false;
+        for (Mutation m : tableMetaData) {
+            byte[] key = m.getRow();
+            boolean addingPKColumn = false;
+            int pkCount = getVarChars(key, rowKeyMetaData);
+            // this means we have are adding a column
+            if (pkCount > COLUMN_NAME_INDEX
+                    && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
+                    && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
+                try {
+                    addingCol = true;
+                    byte[] familyName = null;
+                    byte[] colName = null;
+                    if (pkCount > FAMILY_NAME_INDEX) {
+                        familyName = rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX];
+                    }
+                    if (pkCount > COLUMN_NAME_INDEX) {
+                        colName = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
+                    }
+                    if (table.getExcludedColumns().contains(
+                            PColumnImpl.createExcludedColumn(MetaDataEndpointImpl.newPName(familyName),
+                                    MetaDataEndpointImpl.newPName(colName), 0l))) {
+                        // if this column was previously dropped in a view
+                        // do not allow adding the column back
+                        return new MetaDataProtocol.MetaDataMutationResult(
+                                MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                    if (familyName!=null && familyName.length > 0) {
+                        PColumnFamily family =
+                                table.getColumnFamily(familyName);
+                        family.getPColumnForColumnNameBytes(colName);
+                    } else if (colName!=null && colName.length > 0) {
+                        addingPKColumn = true;
+                        table.getPKColumn(new String(colName));
+                    } else {
+                        continue;
+                    }
+                    return new MetaDataProtocol.MetaDataMutationResult(
+                            MetaDataProtocol.MutationCode.COLUMN_ALREADY_EXISTS,
+                            EnvironmentEdgeManager.currentTimeMillis(), table);
+                } catch (ColumnFamilyNotFoundException e) {
+                    continue;
+                } catch (ColumnNotFoundException e) {
+                    if (addingPKColumn) {
+                        // We may be adding a DESC column, so if table is already
+                        // able to be rowKeyOptimized, it should continue to be so.
+                        if (table.rowKeyOrderOptimizable()) {
+                            UpgradeUtil.addRowKeyOrderOptimizableCell(
+                                    additionalTableMetadataMutations, tableHeaderRowKey,
+                                    clientTimeStamp);
+                        } else if (table.getType() == PTableType.VIEW){
+                            // Don't allow view PK to diverge from table PK as our upgrade code
+                            // does not handle this.
+                            return new MetaDataProtocol.MetaDataMutationResult(
+                                    MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                    EnvironmentEdgeManager.currentTimeMillis(), null);
+                        }
+                        // Add all indexes to invalidate list, as they will all be
+                        // adding the same PK column. No need to lock them, as we
+                        // have the parent table lock at this point.
+                        for (PTable index : table.getIndexes()) {
+                            invalidateList.add(new ImmutableBytesPtr(SchemaUtil
+                                    .getTableKey(tenantId, index.getSchemaName()
+                                            .getBytes(), index.getTableName()
+                                            .getBytes())));
+                            // We may be adding a DESC column, so if index is already
+                            // able to be rowKeyOptimized, it should continue to be so.
+                            if (index.rowKeyOrderOptimizable()) {
+                                byte[] indexHeaderRowKey =
+                                        SchemaUtil.getTableKey(index.getTenantId() == null ?
+                                                ByteUtil.EMPTY_BYTE_ARRAY :
+                                                index.getTenantId().getBytes(),
+                                                index.getSchemaName().getBytes(),
+                                                index.getTableName().getBytes());
+                                UpgradeUtil.addRowKeyOrderOptimizableCell(
+                                        additionalTableMetadataMutations, indexHeaderRowKey,
+                                        clientTimeStamp);
+                            }
+                        }
+                    }
+                    continue;
+                }
+            } else if (pkCount == COLUMN_NAME_INDEX &&
+                    ! (Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0 &&
+                            Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0 ) ) {
+                // Invalidate any table with mutations
+                // TODO: this likely means we don't need the above logic that
+                // loops through the indexes if adding a PK column, since we'd
+                // always have header rows for those.
+                invalidateList.add(new ImmutableBytesPtr(SchemaUtil
+                        .getTableKey(tenantId,
+                                rowKeyMetaData[SCHEMA_NAME_INDEX],
+                                rowKeyMetaData[TABLE_NAME_INDEX])));
+            }
+        }
+        tableMetaData.addAll(additionalTableMetadataMutations);
+        if (type == PTableType.VIEW) {
+            if ( EncodedColumnsUtil.usesEncodedColumnNames(table) && addingCol &&
+                    !table.isAppendOnlySchema()) {
+                // When adding a column to a view that uses encoded column name
+                // scheme, we need to modify the CQ counters stored in the view's
+                // physical table. So to make sure clients get the latest PTable, we
+                // need to invalidate the cache entry.
+                // If the table uses APPEND_ONLY_SCHEMA we use the position of the
+                // column as the encoded column qualifier and so we don't need to
+                // update the CQ counter in the view physical table (see
+                // PHOENIX-4737)
+                invalidateList.add(new ImmutableBytesPtr(
+                        MetaDataUtil.getPhysicalTableRowForView(table)));
+            }
+            // Pass in null as the parent PTable, since we always want to tag the cells
+            // in this case, irrespective of the property values of the parent
+            ViewUtil.addTagsToPutsForViewAlteredProperties(tableMetaData, null);
+        }
+        return null;
+    }
+
+    @Override
+    public List<Pair<PTable, PColumn>> getTableAndDroppedColumnPairs() {
+        return Collections.emptyList();
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ColumnMutator.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ColumnMutator.java
new file mode 100644
index 0000000..aa090aa
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ColumnMutator.java
@@ -0,0 +1,63 @@
+/*
+ * 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.coprocessor;
+
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+public interface ColumnMutator {
+
+    enum MutateColumnType {
+        ADD_COLUMN, DROP_COLUMN
+    }
+
+    /**
+     * Validates the column to be added or dropped against all child views
+     */
+    MetaDataProtocol.MetaDataMutationResult validateWithChildViews(PTable table, List<PTable> childViews,
+                                                                   List<Mutation> tableMetadata,
+                                                                   byte[] schemaName, byte[] tableName)
+            throws IOException, SQLException;
+
+    /**
+     * Validates that the column being added or dropped against the table or view itself
+     * Adds to  the list of mutations required to add or drop columns
+     */
+    MetaDataProtocol.MetaDataMutationResult validateAndAddMetadata(PTable table, byte[][] rowKeyMetaData,
+                                                                   List<Mutation> tableMetadata, Region region,
+                                                                   List<ImmutableBytesPtr> invalidateList,
+                                                                   List<Region.RowLock> locks,
+                                                                   long clientTimeStamp)
+            throws IOException, SQLException;
+
+    /**
+     * @return list of pair of table and column being dropped, used to drop any indexes that require the column
+     */
+    List<Pair<PTable, PColumn>> getTableAndDroppedColumnPairs();
+
+    MutateColumnType getMutateColumnType();
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DropColumnMutator.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DropColumnMutator.java
new file mode 100644
index 0000000..3d94253
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/DropColumnMutator.java
@@ -0,0 +1,277 @@
+/*
+ * 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.coprocessor;
+
+import com.google.common.collect.Lists;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.client.Delete;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.regionserver.Region;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.compile.ColumnResolver;
+import org.apache.phoenix.compile.FromCompiler;
+import org.apache.phoenix.compile.StatementContext;
+import org.apache.phoenix.compile.WhereCompiler;
+import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
+import org.apache.phoenix.expression.Expression;
+import org.apache.phoenix.hbase.index.util.ImmutableBytesPtr;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.jdbc.PhoenixStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.ColumnRef;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.TableRef;
+import org.apache.phoenix.schema.types.PInteger;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.QueryUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+import java.util.ListIterator;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID_INDEX;
+import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+public class DropColumnMutator implements ColumnMutator {
+
+    private List<Pair<PTable, PColumn>> tableAndDroppedColPairs;
+    private Configuration conf;
+
+    private static final Logger logger = LoggerFactory.getLogger(DropColumnMutator.class);
+
+    public DropColumnMutator(Configuration conf) {
+        this.tableAndDroppedColPairs = Lists.newArrayList();
+        this.conf = conf;
+    }
+
+    @Override
+    public MutateColumnType getMutateColumnType() {
+        return MutateColumnType.DROP_COLUMN;
+    }
+
+    /**
+     * Checks to see if the column being dropped is required by a child view
+     */
+    @Override
+    public MetaDataMutationResult validateWithChildViews(PTable table, List<PTable> childViews,
+                                                         List<Mutation> tableMetadata,
+                                                         byte[] schemaName, byte[] tableName)
+            throws IOException, SQLException {
+        List<Delete> columnDeletesForBaseTable = Lists.newArrayListWithExpectedSize(5);
+        for (Mutation m : tableMetadata) {
+            if (m instanceof Delete) {
+                byte[][] rkmd = new byte[5][];
+                int pkCount = getVarChars(m.getRow(), rkmd);
+                if (pkCount > COLUMN_NAME_INDEX
+                        && Bytes.compareTo(schemaName, rkmd[SCHEMA_NAME_INDEX]) == 0
+                        && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) == 0) {
+                    columnDeletesForBaseTable.add((Delete) m);
+                }
+            }
+        }
+        for (PTable view : childViews) {
+            for (Delete columnDeleteForBaseTable : columnDeletesForBaseTable) {
+                PColumn existingViewColumn = null;
+                byte[][] rkmd = new byte[5][];
+                getVarChars(columnDeleteForBaseTable.getRow(), rkmd);
+                String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
+                String columnFamily =
+                        rkmd[FAMILY_NAME_INDEX] == null ? null : Bytes
+                                .toString(rkmd[FAMILY_NAME_INDEX]);
+                try {
+                    existingViewColumn = columnFamily == null ?
+                            view.getColumnForColumnName(columnName) :
+                            view.getColumnFamily(columnFamily).getPColumnForColumnName(columnName);
+                } catch (ColumnFamilyNotFoundException e) {
+                    // ignore since it means that the column family is not present for the column to
+                    // be added.
+                } catch (ColumnNotFoundException e) {
+                    // ignore since it means the column is not present in the view
+                }
+
+                // check if the view where expression contains the column being dropped and prevent
+                // it
+                if (existingViewColumn != null && view.getViewStatement() != null) {
+                    ParseNode viewWhere =
+                            new SQLParser(view.getViewStatement()).parseQuery().getWhere();
+                    PhoenixConnection conn=null;
+                    try {
+                        conn = QueryUtil.getConnectionOnServer(conf).unwrap(
+                                PhoenixConnection.class);
+                    } catch (ClassNotFoundException e) {
+                        throw new IOException(e);
+                    }
+                    PhoenixStatement statement = new PhoenixStatement(conn);
+                    TableRef baseTableRef = new TableRef(view);
+                    ColumnResolver columnResolver = FromCompiler.getResolver(baseTableRef);
+                    StatementContext context = new StatementContext(statement, columnResolver);
+                    Expression whereExpression = WhereCompiler.compile(context, viewWhere);
+                    Expression colExpression =
+                            new ColumnRef(baseTableRef, existingViewColumn.getPosition())
+                                    .newColumnExpression();
+                    MetaDataEndpointImpl.ColumnFinder columnFinder =
+                            new MetaDataEndpointImpl.ColumnFinder(colExpression);
+                    whereExpression.accept(columnFinder);
+                    if (columnFinder.getColumnFound()) {
+                        return new MetaDataProtocol.MetaDataMutationResult(
+                                MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                EnvironmentEdgeManager.currentTimeMillis(), table);
+                    }
+                }
+
+                if (existingViewColumn != null) {
+                    tableAndDroppedColPairs.add(new Pair(view, existingViewColumn));
+                }
+            }
+
+        }
+        return null;
+    }
+
+    @Override
+    public MetaDataMutationResult validateAndAddMetadata(PTable table,
+                                                         byte[][] rowKeyMetaData,
+                                                         List<Mutation> tableMetaData,
+                                                         Region region,
+                                                         List<ImmutableBytesPtr> invalidateList,
+                                                         List<Region.RowLock> locks,
+                                                         long clientTimeStamp)
+                    throws SQLException {
+
+        byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
+        byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
+        byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
+        boolean isView = table.getType() == PTableType.VIEW;
+        boolean deletePKColumn = false;
+
+        List<Mutation> additionalTableMetaData = Lists.newArrayList();
+        ListIterator<Mutation> iterator = tableMetaData.listIterator();
+        while (iterator.hasNext()) {
+            Mutation mutation = iterator.next();
+            byte[] key = mutation.getRow();
+            int pkCount = getVarChars(key, rowKeyMetaData);
+            if (isView && mutation instanceof Put) {
+                PColumn column = MetaDataUtil.getColumn(pkCount, rowKeyMetaData, table);
+                if (column == null)
+                    continue;
+                // ignore any puts that modify the ordinal positions of columns
+                iterator.remove();
+            }
+            else if (mutation instanceof Delete) {
+                if (pkCount > COLUMN_NAME_INDEX
+                        && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
+                        && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
+                    PColumn columnToDelete = null;
+                    try {
+                        columnToDelete = MetaDataUtil.getColumn(pkCount, rowKeyMetaData, table);
+                        if (columnToDelete == null)
+                            continue;
+                        deletePKColumn = columnToDelete.getFamilyName() == null;
+                        if (isView) {
+                            // if we are dropping a derived column add it to the excluded
+                            // column list
+                            if (columnToDelete.isDerived()) {
+                                mutation = MetaDataUtil.cloneDeleteToPutAndAddColumn((Delete)
+                                                mutation, TABLE_FAMILY_BYTES, LINK_TYPE_BYTES,
+                                        PTable.LinkType.EXCLUDED_COLUMN.
+                                                getSerializedValueAsByteArray());
+                                iterator.set(mutation);
+                            }
+
+                            if (table.getBaseColumnCount() != DIVERGED_VIEW_BASE_COLUMN_COUNT
+                                    && columnToDelete.isDerived()) {
+                                // If the column being dropped is inherited from the base table,
+                                // then the view is about to diverge itself from the base table.
+                                // The consequence of this divergence is that that any further
+                                // meta-data changes made to the base table will not be
+                                // propagated to the hierarchy of views where this view is the root.
+                                byte[] viewKey = SchemaUtil.getTableKey(tenantId, schemaName,
+                                        tableName);
+                                Put updateBaseColumnCountPut = new Put(viewKey);
+                                byte[] baseColumnCountPtr =
+                                        new byte[PInteger.INSTANCE.getByteSize()];
+                                PInteger.INSTANCE.getCodec().encodeInt(
+                                        DIVERGED_VIEW_BASE_COLUMN_COUNT, baseColumnCountPtr, 0);
+                                updateBaseColumnCountPut.addColumn(
+                                        PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
+                                        PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES,
+                                        clientTimeStamp,
+                                        baseColumnCountPtr);
+                                additionalTableMetaData.add(updateBaseColumnCountPut);
+                            }
+                        }
+                        if (columnToDelete.isViewReferenced()) {
+                            // Disallow deletion of column referenced in WHERE clause of view
+                            return new MetaDataProtocol.MetaDataMutationResult(
+                                    MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                    EnvironmentEdgeManager.currentTimeMillis(), table,
+                                    columnToDelete);
+                        }
+                        // drop any indexes that need the column that is going to be dropped
+                        tableAndDroppedColPairs.add(new Pair(table, columnToDelete));
+                    } catch (ColumnFamilyNotFoundException e) {
+                        return new MetaDataProtocol.MetaDataMutationResult(
+                                MetaDataProtocol.MutationCode.COLUMN_NOT_FOUND,
+                                EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
+                    } catch (ColumnNotFoundException e) {
+                        return new MetaDataProtocol.MetaDataMutationResult(
+                                MetaDataProtocol.MutationCode.COLUMN_NOT_FOUND,
+                                EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
+                    }
+                }
+            }
+
+        }
+        tableMetaData.addAll(additionalTableMetaData);
+        if (deletePKColumn) {
+            if (table.getPKColumns().size() == 1) {
+                return new MetaDataProtocol.MetaDataMutationResult(
+                        MetaDataProtocol.MutationCode.NO_PK_COLUMNS,
+                        EnvironmentEdgeManager.currentTimeMillis(), null);
+            }
+        }
+        long currentTime = MetaDataUtil.getClientTimeStamp(tableMetaData);
+        return new MetaDataProtocol.MetaDataMutationResult(
+                MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS, currentTime, null);
+    }
+
+    @Override
+    public List<Pair<PTable, PColumn>> getTableAndDroppedColumnPairs() {
+        return tableAndDroppedColPairs;
+    }
+}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
index a49beb8..679c6b0 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
@@ -24,7 +24,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.AUTO_PARTITION_SEQ
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.CLASS_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_COUNT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_DEF_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_QUALIFIER_COUNTER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_SIZE_BYTES;
@@ -35,7 +34,6 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_COLUMN_FAM
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DEFAULT_VALUE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.DISABLE_WAL_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ENCODING_SCHEME_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.IMMUTABLE_ROWS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.INDEX_STATE_BYTES;
@@ -56,15 +54,12 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.ORDINAL_POSITION_B
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.RETURN_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SALT_BUCKETS_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SORT_ORDER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORAGE_SCHEME_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SEQ_NUM_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TENANT_ID_INDEX;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTIONAL_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TRANSACTION_PROVIDER_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TYPE_BYTES;
@@ -75,11 +70,8 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_BYTE
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_STATEMENT_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_TYPE_BYTES;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.VIEW_INDEX_ID_DATA_TYPE_BYTES;
-import static org.apache.phoenix.query.QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
 import static org.apache.phoenix.query.QueryConstants.VIEW_MODIFIED_PROPERTY_TAG_TYPE;
 import static org.apache.phoenix.schema.PTableType.INDEX;
-import static org.apache.phoenix.schema.PTableType.TABLE;
-import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
@@ -95,14 +87,17 @@ import java.util.Comparator;
 import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
-import java.util.ListIterator;
-import java.util.Map;
-import java.util.Map.Entry;
 import java.util.NavigableMap;
 import java.util.Properties;
 import java.util.Set;
 
+import com.google.common.cache.Cache;
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.protobuf.ByteString;
+import com.google.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
 import org.apache.hadoop.hbase.CellUtil;
@@ -138,13 +133,8 @@ import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.phoenix.cache.GlobalCache;
 import org.apache.phoenix.cache.GlobalCache.FunctionBytesPtr;
-import org.apache.phoenix.compile.ColumnNameTrackingExpressionCompiler;
-import org.apache.phoenix.compile.ColumnResolver;
-import org.apache.phoenix.compile.FromCompiler;
 import org.apache.phoenix.compile.QueryPlan;
 import org.apache.phoenix.compile.ScanRanges;
-import org.apache.phoenix.compile.StatementContext;
-import org.apache.phoenix.compile.WhereCompiler;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
@@ -165,6 +155,8 @@ import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionRequest
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.GetVersionResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.UpdateIndexStateRequest;
+import org.apache.phoenix.exception.SQLExceptionCode;
+import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.expression.Expression;
 import org.apache.phoenix.expression.KeyValueColumnExpression;
 import org.apache.phoenix.expression.LiteralExpression;
@@ -182,23 +174,16 @@ import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.jdbc.PhoenixResultSet;
 import org.apache.phoenix.jdbc.PhoenixStatement;
 import org.apache.phoenix.metrics.Metrics;
-import org.apache.phoenix.parse.DropTableStatement;
 import org.apache.phoenix.parse.LiteralParseNode;
 import org.apache.phoenix.parse.PFunction;
 import org.apache.phoenix.parse.PFunction.FunctionArgument;
 import org.apache.phoenix.parse.PSchema;
-import org.apache.phoenix.parse.ParseNode;
-import org.apache.phoenix.parse.SQLParser;
 import org.apache.phoenix.protobuf.ProtobufUtil;
 import org.apache.phoenix.query.ConnectionQueryServices;
 import org.apache.phoenix.query.KeyRange;
 import org.apache.phoenix.query.QueryConstants;
 import org.apache.phoenix.query.QueryServices;
 import org.apache.phoenix.query.QueryServicesOptions;
-import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
-import org.apache.phoenix.schema.ColumnNotFoundException;
-import org.apache.phoenix.schema.ColumnRef;
-import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.MetaDataSplitPolicy;
 import org.apache.phoenix.schema.PColumn;
 import org.apache.phoenix.schema.PColumnFamily;
@@ -215,25 +200,20 @@ import org.apache.phoenix.schema.PTable.LinkType;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.PTableImpl;
-import org.apache.phoenix.schema.PTableKey;
 import org.apache.phoenix.schema.PTableType;
-import org.apache.phoenix.schema.ParentTableNotFoundException;
 import org.apache.phoenix.schema.RowKeySchema;
-import org.apache.phoenix.schema.SaltingUtil;
 import org.apache.phoenix.schema.SequenceAllocation;
 import org.apache.phoenix.schema.SequenceAlreadyExistsException;
 import org.apache.phoenix.schema.SequenceKey;
 import org.apache.phoenix.schema.SequenceNotFoundException;
 import org.apache.phoenix.schema.SortOrder;
 import org.apache.phoenix.schema.TableNotFoundException;
-import org.apache.phoenix.schema.TableRef;
 import org.apache.phoenix.schema.task.Task;
 import org.apache.phoenix.schema.types.PBinary;
 import org.apache.phoenix.schema.types.PBoolean;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PInteger;
 import org.apache.phoenix.schema.types.PLong;
-import org.apache.phoenix.schema.types.PSmallint;
 import org.apache.phoenix.schema.types.PTinyint;
 import org.apache.phoenix.schema.types.PVarbinary;
 import org.apache.phoenix.schema.types.PVarchar;
@@ -242,7 +222,6 @@ import org.apache.phoenix.transaction.TransactionFactory;
 import org.apache.phoenix.util.ByteUtil;
 import org.apache.phoenix.util.EncodedColumnsUtil;
 import org.apache.phoenix.util.EnvironmentEdgeManager;
-import org.apache.phoenix.util.IndexUtil;
 import org.apache.phoenix.util.KeyValueUtil;
 import org.apache.phoenix.util.MetaDataUtil;
 import org.apache.phoenix.util.PhoenixRuntime;
@@ -250,18 +229,12 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.ServerUtil;
+import org.apache.phoenix.util.TableViewFinderResult;
 import org.apache.phoenix.util.UpgradeUtil;
+import org.apache.phoenix.util.ViewUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import com.google.common.cache.Cache;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.protobuf.ByteString;
-import com.google.protobuf.RpcCallback;
-import com.google.protobuf.RpcController;
-import com.google.protobuf.Service;
-
 /**
  * Endpoint co-processor through which all Phoenix metadata mutations flow.
  * Phoenix metadata is stored in SYSTEM.CATALOG. The table specific information
@@ -271,7 +244,7 @@ import com.google.protobuf.Service;
  * parent->child links are stored in a separate SYSTEM.CHILD_LINK table.
  * Metadata for all tables/views/indexes in the same schema are stored in a
  * single region which is enforced using the {@link MetaDataSplitPolicy}.
- * 
+ * <p>
  * While creating child views we only store columns added by the view. When
  * resolving a view we resolve all its parents and add their columns to the
  * PTable that is returned. We lock the parent table while creating an index to
@@ -288,7 +261,7 @@ import com.google.protobuf.Service;
  * been removed yet, we delete the child view metadata. When resolving a view,
  * we resolve all its parents, if any of them are dropped the child view is
  * considered to be dropped and we throw a TableNotFoundException.
- * 
+ * <p>
  * We only allow mutations to the latest version of a Phoenix table (i.e. the
  * timeStamp must be increasing). For adding/dropping columns we use a sequence
  * number on the table to ensure that the client has the latest version.
@@ -303,9 +276,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     public static final String ROW_KEY_ORDER_OPTIMIZABLE = "ROW_KEY_ORDER_OPTIMIZABLE";
     public static final byte[] ROW_KEY_ORDER_OPTIMIZABLE_BYTES = Bytes.toBytes(ROW_KEY_ORDER_OPTIMIZABLE);
 
-    private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
+    private static final byte[] CHILD_TABLE_BYTES = new byte[]{PTable.LinkType.CHILD_TABLE.getSerializedValue()};
     private static final byte[] PHYSICAL_TABLE_BYTES =
-            new byte[] { PTable.LinkType.PHYSICAL_TABLE.getSerializedValue() };
+            new byte[]{PTable.LinkType.PHYSICAL_TABLE.getSerializedValue()};
 
     // KeyValues for Table
     private static final KeyValue TABLE_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
@@ -324,7 +297,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue VIEW_INDEX_ID_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_INDEX_ID_BYTES);
     /**
      * A designator for choosing the right type for viewIndex (Short vs Long) to be backward compatible.
-     * **/
+     **/
     private static final KeyValue VIEW_INDEX_ID_DATA_TYPE_BYTES_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, VIEW_INDEX_ID_DATA_TYPE_BYTES);
     private static final KeyValue INDEX_TYPE_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_TYPE_BYTES);
     private static final KeyValue INDEX_DISABLE_TIMESTAMP_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES);
@@ -342,7 +315,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue STORAGE_SCHEME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, STORAGE_SCHEME_BYTES);
     private static final KeyValue ENCODING_SCHEME_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, ENCODING_SCHEME_BYTES);
     private static final KeyValue USE_STATS_FOR_PARALLELIZATION_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, USE_STATS_FOR_PARALLELIZATION_BYTES);
-    
+
     private static final List<KeyValue> TABLE_KV_COLUMNS = Arrays.<KeyValue>asList(
             EMPTY_KEYVALUE_KV,
             TABLE_TYPE_KV,
@@ -374,7 +347,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             STORAGE_SCHEME_KV,
             ENCODING_SCHEME_KV,
             USE_STATS_FOR_PARALLELIZATION_KV
-            );
+    );
+
     static {
         Collections.sort(TABLE_KV_COLUMNS, KeyValue.COMPARATOR);
     }
@@ -442,10 +416,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             IS_ROW_TIMESTAMP_KV,
             COLUMN_QUALIFIER_KV,
             LINK_TYPE_KV
-            );
+    );
+
     static {
         Collections.sort(COLUMN_KV_COLUMNS, KeyValue.COMPARATOR);
     }
+
     private static final KeyValue QUALIFIER_COUNTER_KV = KeyValue.createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, COLUMN_QUALIFIER_COUNTER_BYTES);
     private static final int DECIMAL_DIGITS_INDEX = COLUMN_KV_COLUMNS.indexOf(DECIMAL_DIGITS_KV);
     private static final int COLUMN_SIZE_INDEX = COLUMN_KV_COLUMNS.indexOf(COLUMN_SIZE_KV);
@@ -482,12 +458,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final KeyValue IS_ARRAY_KV = createFirstOnRow(ByteUtil.EMPTY_BYTE_ARRAY, TABLE_FAMILY_BYTES, IS_ARRAY_BYTES);
 
     private static final List<KeyValue> FUNCTION_KV_COLUMNS = Arrays.<KeyValue>asList(
-        EMPTY_KEYVALUE_KV,
-        CLASS_NAME_KV,
-        JAR_PATH_KV,
-        RETURN_TYPE_KV,
-        NUM_ARGS_KV
-        );
+            EMPTY_KEYVALUE_KV,
+            CLASS_NAME_KV,
+            JAR_PATH_KV,
+            RETURN_TYPE_KV,
+            NUM_ARGS_KV
+    );
+
     static {
         Collections.sort(FUNCTION_KV_COLUMNS, KeyValue.COMPARATOR);
     }
@@ -498,13 +475,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int NUM_ARGS_INDEX = FUNCTION_KV_COLUMNS.indexOf(NUM_ARGS_KV);
 
     private static final List<KeyValue> FUNCTION_ARG_KV_COLUMNS = Arrays.<KeyValue>asList(
-        TYPE_KV,
-        IS_ARRAY_KV,
-        IS_CONSTANT_KV,
-        DEFAULT_VALUE_KV,
-        MIN_VALUE_KV,
-        MAX_VALUE_KV
-        );
+            TYPE_KV,
+            IS_ARRAY_KV,
+            IS_CONSTANT_KV,
+            DEFAULT_VALUE_KV,
+            MIN_VALUE_KV,
+            MAX_VALUE_KV
+    );
+
     static {
         Collections.sort(FUNCTION_ARG_KV_COLUMNS, KeyValue.COMPARATOR);
     }
@@ -514,12 +492,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int DEFAULT_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(DEFAULT_VALUE_KV);
     private static final int MIN_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MIN_VALUE_KV);
     private static final int MAX_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MAX_VALUE_KV);
-    
-    private static PName newPName(byte[] buffer) {
-        return buffer==null ? null : newPName(buffer, 0, buffer.length);
+
+    public static PName newPName(byte[] buffer) {
+        return buffer == null ? null : newPName(buffer, 0, buffer.length);
     }
 
-    private static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) {
+    public static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) {
         if (keyLength <= 0) {
             return null;
         }
@@ -538,7 +516,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     // this flag denotes that we will continue to write parent table column metadata while creating
     // a child view and also block metadata changes that were previously propagated to children
     // before 4.15, so that we can rollback the upgrade to 4.15 if required
-    private boolean allowSystemCatalogRollback;
+    private boolean allowSplittableSystemCatalogRollback;
 
     /**
      * Stores a reference to the coprocessor environment provided by the
@@ -546,9 +524,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
      * coprocessor is loaded. Since this is a coprocessor endpoint, it always expects to be loaded
      * on a table region, so always expects this to be an instance of
      * {@link RegionCoprocessorEnvironment}.
+     *
      * @param env the environment provided by the coprocessor host
      * @throws IOException if the provided environment is not an instance of
-     *             {@code RegionCoprocessorEnvironment}
+     *                     {@code RegionCoprocessorEnvironment}
      */
     @Override
     public void start(CoprocessorEnvironment env) throws IOException {
@@ -557,18 +536,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } else {
             throw new CoprocessorException("Must be loaded on a table region!");
         }
-        
+
         phoenixAccessCoprocessorHost = new PhoenixMetaDataCoprocessorHost(this.env);
         Configuration config = env.getConfiguration();
         this.accessCheckEnabled = config.getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
                 QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
-        this.blockWriteRebuildIndex  = config.getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
+        this.blockWriteRebuildIndex = config.getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE,
                 QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
         this.maxIndexesPerTable = config.getInt(QueryServices.MAX_INDEXES_PER_TABLE,
-                    QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
+                QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
         this.isTablesMappingEnabled = SchemaUtil.isNamespaceMappingEnabled(PTableType.TABLE,
                 new ReadOnlyProps(config.iterator()));
-        this.allowSystemCatalogRollback = config.getBoolean(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK,
+        this.allowSplittableSystemCatalogRollback = config.getBoolean(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK,
                 QueryServicesOptions.DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
 
         LOGGER.info("Starting Tracing-Metrics Systems");
@@ -589,7 +568,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     @Override
     public void getTable(RpcController controller, GetTableRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                         RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         byte[] tenantId = request.getTenantId().toByteArray();
         byte[] schemaName = request.getSchemaName().toByteArray();
@@ -609,9 +588,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             long currentTime = EnvironmentEdgeManager.currentTimeMillis();
             PTable table =
                     doGetTable(tenantId, schemaName, tableName, request.getClientTimestamp(),
-                        null, request.getClientVersion(), request.getSkipAddingIndexes(),
-                        request.getSkipAddingParentColumns(),
-                        PTableImpl.createFromProto(request.getLockedAncestorTable()));
+                            null, request.getClientVersion());
             if (table == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                 builder.setMutationTime(currentTime);
@@ -630,8 +607,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     disableIndexTimestamp = index.getIndexDisableTimestamp();
                     if (disableIndexTimestamp > 0
                             && (index.getIndexState() == PIndexState.ACTIVE
-                                    || index.getIndexState() == PIndexState.PENDING_ACTIVE
-                                    || index.getIndexState() == PIndexState.PENDING_DISABLE)
+                            || index.getIndexState() == PIndexState.PENDING_ACTIVE
+                            || index.getIndexState() == PIndexState.PENDING_DISABLE)
                             && disableIndexTimestamp < minNonZerodisableIndexTimestamp) {
                         minNonZerodisableIndexTimestamp = disableIndexTimestamp;
                     }
@@ -646,404 +623,50 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             // the PTable of views and indexes on views might get updated because a column is added to one of
             // their parents (this won't change the timestamp)
-            if (table.getType()!=PTableType.TABLE || table.getTimeStamp() != tableTimeStamp) {
+            if (table.getType() != PTableType.TABLE || table.getTimeStamp() != tableTimeStamp) {
                 builder.setTable(PTableImpl.toProto(table));
             }
             done.run(builder.build());
         } catch (Throwable t) {
             LOGGER.error("getTable failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
-        }
-    }
-
-    /**
-     * Used to add the columns present the ancestor hierarchy to the PTable of the given view or
-     * view index
-     * @param table PTable of the view or view index
-     * @param skipAddingIndexes if true the returned PTable won't include indexes
-     * @param skipAddingParentColumns if true the returned PTable won't include columns derived from
-     *            ancestor tables
-     * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to
-     *            resolve this table as its locked)
-     */
-    private Pair<PTable, MetaDataProtos.MutationCode> combineColumns(PTable table, long timestamp,
-            int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns,
-            PTable lockedAncestorTable) throws SQLException, IOException {
-        boolean hasIndexId = table.getViewIndexId() != null;
-        if (table.getType() != PTableType.VIEW && !hasIndexId) {
-            return new Pair<PTable, MetaDataProtos.MutationCode>(table,
-                    MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
-        }
-        if (!skipAddingParentColumns) {
-            table =
-                    addDerivedColumnsFromAncestors(table, timestamp, clientVersion,
-                        lockedAncestorTable);
-            if (table==null) {
-                return new Pair<PTable, MetaDataProtos.MutationCode>(table,
-                        MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
-            }
-            // we need to resolve the indexes of views (to get ensure they also have all the columns
-            // derived from their ancestors) 
-            if (!skipAddingIndexes && !table.getIndexes().isEmpty()) {
-                List<PTable> indexes = Lists.newArrayListWithExpectedSize(table.getIndexes().size());
-                for (PTable index : table.getIndexes()) {
-                    byte[] tenantIdBytes =
-                            index.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY
-                                    : index.getTenantId().getBytes();
-                    PTable latestIndex =
-                            doGetTable(tenantIdBytes, index.getSchemaName().getBytes(),
-                                index.getTableName().getBytes(), timestamp, null, clientVersion, true,
-                                false, lockedAncestorTable);
-                    if (latestIndex == null) {
-                        throw new TableNotFoundException(
-                                "Could not find index table while combining columns "
-                                        + index.getTableName().getString() + " with tenant id "
-                                        + index.getTenantId());
-                    }
-                    indexes.add(latestIndex);
-                }
-                table = PTableImpl.builderWithColumns(table, getColumnsToClone(table))
-                        .setIndexes(indexes == null ? Collections.<PTable>emptyList() : indexes)
-                        .build();
-            }
+                    ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
-        
-        MetaDataProtos.MutationCode mutationCode =
-                table != null ? MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS
-                        : MetaDataProtos.MutationCode.TABLE_NOT_FOUND;
-        return new Pair<PTable, MetaDataProtos.MutationCode>(table, mutationCode);
-    }
-
-    
-    private PTable addDerivedColumnsFromAncestors(PTable table, long timestamp,
-            int clientVersion, PTable lockedAncestorTable) throws IOException, SQLException, TableNotFoundException {
-        // combine columns for view and view indexes
-        byte[] tenantId =
-                table.getTenantId() != null ? table.getTenantId().getBytes()
-                        : ByteUtil.EMPTY_BYTE_ARRAY;
-        byte[] schemaName = table.getSchemaName().getBytes();
-        byte[] tableName = table.getTableName().getBytes();
-        String fullTableName = SchemaUtil.getTableName(table.getSchemaName().getString(),
-                table.getTableName().getString());
-        boolean hasIndexId = table.getViewIndexId() != null;
-        boolean isSalted = table.getBucketNum() != null;
-        if (table.getType() != PTableType.VIEW && !hasIndexId) {
-            return table;
-        }
-        boolean isDiverged = isDivergedView(table);
-        // here you combine columns from the parent tables the logic is as follows, if the PColumn
-        // is in the EXCLUDED_COLUMNS remove it, otherwise priority of keeping duplicate columns is
-        // child -> parent
-        List<TableInfo> ancestorList = Lists.newArrayList();
-        TableViewFinderResult viewFinderResult = new TableViewFinderResult();
-        if (PTableType.VIEW == table.getType()) {
-            findAncestorViews(tenantId, schemaName, tableName, viewFinderResult,
-                table.isNamespaceMapped());
-        } else { // is a view index
-            findAncestorViewsOfIndex(tenantId, schemaName, tableName, viewFinderResult,
-                table.isNamespaceMapped());
-        }
-        List<TableInfo> tableViewInfoList = viewFinderResult.getLinks();
-        if (tableViewInfoList.isEmpty()) {
-            // no need to combine columns for local indexes on regular tables
-            return table;
-        }
-        ancestorList.addAll(tableViewInfoList);
-        List<PColumn> allColumns = Lists.newArrayList();
-        List<PColumn> excludedColumns = Lists.newArrayList();
-        // add my own columns first in reverse order
-        List<PColumn> myColumns = table.getColumns();
-        // skip salted column as it will be created automatically
-        myColumns = myColumns.subList(isSalted ? 1 : 0, myColumns.size());
-        for (int i = myColumns.size() - 1; i >= 0; i--) {
-            PColumn pColumn = myColumns.get(i);
-            if (pColumn.isExcluded()) {
-                excludedColumns.add(pColumn);
-            }
-            allColumns.add(pColumn);
-        }
-
-        // initialize map from with indexed expression to list of required data columns
-        // then remove the data columns that have not been dropped, so that we get the columns that
-        // have been dropped
-        Map<PColumn, List<String>> indexRequiredDroppedDataColMap =
-                Maps.newHashMapWithExpectedSize(table.getColumns().size());
-        if (hasIndexId) {
-            int indexPosOffset = (isSalted ? 1 : 0) + (table.isMultiTenant() ? 1 : 0) + 1;
-            ColumnNameTrackingExpressionCompiler expressionCompiler =
-                    new ColumnNameTrackingExpressionCompiler();
-            for (int i = indexPosOffset; i < table.getPKColumns().size(); i++) {
-                PColumn indexColumn = table.getPKColumns().get(i);
-                try {
-                    expressionCompiler.reset();
-                    String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
-                    ParseNode parseNode = SQLParser.parseCondition(expressionStr);
-                    parseNode.accept(expressionCompiler);
-                    indexRequiredDroppedDataColMap.put(indexColumn,
-                        Lists.newArrayList(expressionCompiler.getDataColumnNames()));
-                } catch (SQLException e) {
-                    throw new RuntimeException(e); // Impossible
-                }
-            }
-        }
-
-        // now go up from child to parent all the way to the base table:
-        PTable baseTable = null;
-        PTable immediateParent = null;
-        long maxTableTimestamp = -1;
-        int numPKCols = table.getPKColumns().size();
-        for (int i = 0; i < ancestorList.size(); i++) {
-            TableInfo parentTableInfo = ancestorList.get(i);
-            PTable pTable;
-            String fullParentTableName = SchemaUtil.getTableName(parentTableInfo.getSchemaName(),
-                parentTableInfo.getTableName());
-            PName parentTenantId =
-                    (parentTableInfo.getTenantId() != null && parentTableInfo.getTenantId().length!=0)
-                            ? PNameFactory.newName(parentTableInfo.getTenantId()) : null;
-            PTableKey pTableKey = new PTableKey(parentTenantId, fullParentTableName);
-            // if we already have the PTable of an ancestor that has been locked, no need to look up
-            // the table
-            if (lockedAncestorTable != null && lockedAncestorTable.getKey().equals(pTableKey)) {
-                pTable = lockedAncestorTable;
-            } else {
-                // if we are currently combining columns for a view index and are looking up its
-                // ancestors we do not add the indexes to the ancestor PTable (or else we end up in
-                // a circular loop)
-                // we also don't need to add parent columns of the ancestors as we combine columns
-                // from all ancestors
-                pTable =
-                        doGetTable(parentTableInfo.getTenantId(), parentTableInfo.getSchemaName(),
-                            parentTableInfo.getTableName(), timestamp, null, clientVersion, hasIndexId,
-                            true, null);
-            }
-            if (pTable == null) {
-                throw new ParentTableNotFoundException(parentTableInfo, fullTableName);
-            } else {
-                if (immediateParent == null) {
-                    immediateParent = pTable;
-                }
-                // only combine columns for view indexes (and not local indexes on regular tables
-                // which also have a viewIndexId)
-                if (i == 0 && hasIndexId && pTable.getType() != PTableType.VIEW) {
-                    return table;
-                }
-                if (TABLE.equals(pTable.getType())) {
-                    baseTable = pTable;
-                }
-                // set the final table timestamp as the max timestamp of the view/view index or its
-                // ancestors
-                maxTableTimestamp = Math.max(maxTableTimestamp, pTable.getTimeStamp());
-                if (hasIndexId) {
-                    // add all pk columns of parent tables to indexes
-                    // skip salted column as it will be added from the base table columns
-                    int startIndex = pTable.getBucketNum() != null ? 1 : 0;
-                    for (int index=startIndex; index<pTable.getPKColumns().size(); index++) {
-                        PColumn pkColumn = pTable.getPKColumns().get(index);
-                        // don't add the salt column of ancestor tables for view indexes
-                        if (pkColumn.equals(SaltingUtil.SALTING_COLUMN) || pkColumn.isExcluded()) {
-                            continue;
-                        }
-                        pkColumn = IndexUtil.getIndexPKColumn(++numPKCols, pkColumn);
-                        int existingColumnIndex = allColumns.indexOf(pkColumn);
-                        if (existingColumnIndex == -1) {
-                            allColumns.add(0, pkColumn);
-                        }
-                    }
-                    for (int j = 0; j < pTable.getColumns().size(); j++) {
-                        PColumn tableColumn = pTable.getColumns().get(j);
-                        if (tableColumn.isExcluded()) {
-                            continue;
-                        }
-                        String dataColumnName = tableColumn.getName().getString();
-                        // remove from list of columns since it has not been dropped
-                        for (Entry<PColumn, List<String>> entry : indexRequiredDroppedDataColMap
-                                .entrySet()) {
-                            entry.getValue().remove(dataColumnName);
-                        }
-                    }
-                } else {
-                    List<PColumn> currAncestorTableCols = PTableImpl.getColumnsToClone(pTable);
-                    if (currAncestorTableCols != null) {
-                        // add the ancestor columns in reverse order so that the final column list
-                        // contains ancestor columns and then the view columns in the right order
-                        for (int j = currAncestorTableCols.size() - 1; j >= 0; j--) {
-                            PColumn column = currAncestorTableCols.get(j);
-                            // for diverged views we always include pk columns of the base table. We
-                            // have to include these pk columns to be able to support adding pk
-                            // columns to the diverged view
-                            // we only include regular columns that were created before the view
-                            // diverged
-                            if (isDiverged && column.getFamilyName() != null
-                                    && column.getTimestamp() > table.getTimeStamp()) {
-                                continue;
-                            }
-                            // need to check if this column is in the list of excluded (dropped)
-                            // columns of the view
-                            int existingIndex = excludedColumns.indexOf(column);
-                            if (existingIndex != -1) {
-                                // if it is, only exclude the column if was created before the
-                                // column was dropped in the view in order to handle the case where
-                                // a base table column is dropped in a view, then dropped in the
-                                // base table and then added back to the base table
-                                if (column.getTimestamp() <= excludedColumns.get(existingIndex)
-                                        .getTimestamp()) {
-                                    continue;
-                                }
-                            }
-                            if (column.isExcluded()) {
-                                excludedColumns.add(column);
-                            } else {
-                                int existingColumnIndex = allColumns.indexOf(column);
-                                if (existingColumnIndex != -1) {
-                                    // for diverged views if the view was created before
-                                    // PHOENIX-3534 the parent table columns will be present in the
-                                    // view PTable (since the base column count is
-                                    // QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT we can't
-                                    // filter them out) so we always pick the parent column  
-                                    // for non diverged views if the same column exists in a parent
-                                    // and child, we keep the latest column
-                                    PColumn existingColumn = allColumns.get(existingColumnIndex);
-                                    if (isDiverged || column.getTimestamp() > existingColumn.getTimestamp()) {
-                                        allColumns.remove(existingColumnIndex);
-                                        allColumns.add(column);
-                                    }
-                                } else {
-                                    allColumns.add(column);
-                                }
-                            }
-                        }
-                    }
-                }
-            }
-        }
-        // at this point indexRequiredDroppedDataColMap only contain the columns required by a view
-        // index that have dropped
-        for (Entry<PColumn, List<String>> entry : indexRequiredDroppedDataColMap.entrySet()) {
-            if (!entry.getValue().isEmpty()) {
-                PColumn indexColumnToBeDropped = entry.getKey();
-                if (SchemaUtil.isPKColumn(indexColumnToBeDropped)) {
-                    // if an indexed column was dropped in an ancestor then we
-                    // cannot use this index an more
-                    // TODO figure out a way to actually drop this view index
-                    return null;
-                } else {
-                    allColumns.remove(indexColumnToBeDropped);
-                }
-            }
-        }
-        // remove the excluded columns if the timestamp is newer than the added column
-        for (PColumn excludedColumn : excludedColumns) {
-            int index = allColumns.indexOf(excludedColumn);
-            if (index != -1) {
-                if (allColumns.get(index).getTimestamp() <= excludedColumn.getTimestamp()) {
-                    allColumns.remove(excludedColumn);
-                }
-            }
-        }
-        List<PColumn> columnsToAdd = Lists.newArrayList();
-        int position = isSalted ? 1 : 0;
-        // allColumns contains the columns in the reverse order
-        for (int i = allColumns.size() - 1; i >= 0; i--) {
-            PColumn column = allColumns.get(i);
-            if (table.getColumns().contains(column)) {
-                // for views this column is not derived from an ancestor
-                columnsToAdd.add(new PColumnImpl(column, position++));
-            } else {
-                columnsToAdd.add(new PColumnImpl(column, true, position++));
-            }
-        }
-        // we need to include the salt column when setting the base table column count in order to
-        // maintain b/w compatibility
-        int baseTableColumnCount =
-                isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT
-                        : columnsToAdd.size() - myColumns.size() + (isSalted ? 1 : 0);
-
-        // Inherit view-modifiable properties from the parent table/view if the current view has
-        // not previously modified this property
-        Long updateCacheFreq = (table.getType() != PTableType.VIEW ||
-                table.hasViewModifiedUpdateCacheFrequency()) ?
-                table.getUpdateCacheFrequency() : immediateParent.getUpdateCacheFrequency();
-        Boolean useStatsForParallelization = (table.getType() != PTableType.VIEW ||
-                table.hasViewModifiedUseStatsForParallelization()) ?
-                table.useStatsForParallelization() : immediateParent.useStatsForParallelization();
-        // When creating a PTable for views or view indexes, use the baseTable PTable for attributes
-        // inherited from the physical base table.
-        // if a TableProperty is not valid on a view we set it to the base table value
-        // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
-        // if a TableProperty is valid on a view and is mutable on a view, we use the value set
-        // on the view if the view had previously modified the property, otherwise we propagate the
-        // value from the base table (see PHOENIX-4763)
-        PTableImpl pTable = PTableImpl.builderWithColumns(table, columnsToAdd)
-                .setImmutableRows(baseTable.isImmutableRows())
-                .setDisableWAL(baseTable.isWALDisabled())
-                .setMultiTenant(baseTable.isMultiTenant())
-                .setStoreNulls(baseTable.getStoreNulls())
-                .setTransactionProvider(baseTable.getTransactionProvider())
-                .setAutoPartitionSeqName(baseTable.getAutoPartitionSeqName())
-                .setAppendOnlySchema(baseTable.isAppendOnlySchema())
-                .setImmutableStorageScheme(baseTable.getImmutableStorageScheme() == null ?
-                        ImmutableStorageScheme.ONE_CELL_PER_COLUMN : baseTable.getImmutableStorageScheme())
-                .setQualifierEncodingScheme(baseTable.getEncodingScheme() == null ?
-                        QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : baseTable.getEncodingScheme())
-                .setBaseColumnCount(baseTableColumnCount)
-                .setTimeStamp(maxTableTimestamp)
-                .setExcludedColumns(excludedColumns == null ?
-                        ImmutableList.<PColumn>of() : ImmutableList.copyOf(excludedColumns))
-                .setUpdateCacheFrequency(updateCacheFreq)
-                .setUseStatsForParallelization(useStatsForParallelization)
-                .build();
-        return WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
     }
 
     private PhoenixMetaDataCoprocessorHost getCoprocessorHost() {
         return phoenixAccessCoprocessorHost;
     }
 
-    /**
-     * @param skipAddingIndexes if true the PTable will not include indexes for tables or views
-     * @param skipAddingParentColumns if true the PTable will not include parent columns for views
-     *            or indexes
-     * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to
-     *            resolve this table as its locked)
-     */
     private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
-            long clientTimeStamp, int clientVersion, boolean skipAddingIndexes,
-            boolean skipAddingParentColumns, PTable lockedAncestorTable)
+                              long clientTimeStamp, int clientVersion)
             throws IOException, SQLException {
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
-        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
         PTable newTable;
         try (RegionScanner scanner = region.getScanner(scan)) {
-            PTable oldTable = (PTable)metaDataCache.getIfPresent(cacheKey);
-            long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP-1 : oldTable.getTimeStamp();
-            newTable = getTable(scanner, clientTimeStamp, tableTimeStamp, clientVersion, skipAddingIndexes, skipAddingParentColumns);
+            PTable oldTable = (PTable) metaDataCache.getIfPresent(cacheKey);
+            long tableTimeStamp = oldTable == null ? MIN_TABLE_TIMESTAMP - 1 : oldTable.getTimeStamp();
+            newTable = getTable(scanner, clientTimeStamp, tableTimeStamp, clientVersion);
             if (newTable != null
                     && (oldTable == null || tableTimeStamp < newTable.getTimeStamp()
-                            || (blockWriteRebuildIndex && newTable.getIndexDisableTimestamp() > 0))
-                    // only cache the PTable if it has the required indexes,
-                    // the PTable added to the cache doesn't include parent columns as we always call 
-                    // combine columns after looking up the PTable from the cache
-                    && !skipAddingIndexes) {
+                    || (blockWriteRebuildIndex && newTable.getIndexDisableTimestamp() > 0))) {
                 if (LOGGER.isDebugEnabled()) {
                     LOGGER.debug("Caching table "
                             + Bytes.toStringBinary(cacheKey.get(), cacheKey.getOffset(),
-                                cacheKey.getLength())
+                            cacheKey.getLength())
                             + " at seqNum " + newTable.getSequenceNumber()
                             + " with newer timestamp " + newTable.getTimeStamp() + " versus "
                             + tableTimeStamp);
                 }
                 metaDataCache.put(cacheKey, newTable);
             }
-            if (newTable != null) {
-                newTable = combineColumns(newTable, clientTimeStamp, clientVersion, skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable).getFirst();
-            }
         }
         return newTable;
     }
 
     private List<PFunction> buildFunctions(List<byte[]> keys, Region region,
-            long clientTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace) throws IOException, SQLException {
+                                           long clientTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace) throws IOException, SQLException {
         List<KeyRange> keyRanges = Lists.newArrayListWithExpectedSize(keys.size());
         for (byte[] key : keys) {
             byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
@@ -1055,11 +678,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         ScanRanges scanRanges = ScanRanges.createPointLookup(keyRanges);
         scanRanges.initializeScan(scan);
         scan.setFilter(scanRanges.getSkipScanFilter());
-        Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
         List<PFunction> functions = new ArrayList<PFunction>();
         PFunction function = null;
         try (RegionScanner scanner = region.getScanner(scan)) {
-            for(int i = 0; i< keys.size(); i++) {
+            for (int i = 0; i < keys.size(); i++) {
                 function = null;
                 function =
                         getFunction(scanner, isReplace, clientTimeStamp, deleteMutationsForReplace);
@@ -1068,9 +691,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 byte[] functionKey =
                         SchemaUtil.getFunctionKey(
-                            function.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : function
-                                    .getTenantId().getBytes(), Bytes.toBytes(function
-                                    .getFunctionName()));
+                                function.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : function
+                                        .getTenantId().getBytes(), Bytes.toBytes(function
+                                        .getFunctionName()));
                 metaDataCache.put(new FunctionBytesPtr(functionKey), function);
                 functions.add(function);
             }
@@ -1079,7 +702,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private List<PSchema> buildSchemas(List<byte[]> keys, Region region, long clientTimeStamp,
-            ImmutableBytesPtr cacheKey) throws IOException, SQLException {
+                                       ImmutableBytesPtr cacheKey) throws IOException, SQLException {
         List<KeyRange> keyRanges = Lists.newArrayListWithExpectedSize(keys.size());
         for (byte[] key : keys) {
             byte[] stopKey = ByteUtil.concat(key, QueryConstants.SEPARATOR_BYTE_ARRAY);
@@ -1088,7 +711,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         Scan scan = new Scan();
         if (clientTimeStamp != HConstants.LATEST_TIMESTAMP
-            && clientTimeStamp != HConstants.OLDEST_TIMESTAMP) {
+                && clientTimeStamp != HConstants.OLDEST_TIMESTAMP) {
             scan.setTimeRange(MIN_TABLE_TIMESTAMP, clientTimeStamp + 1);
         } else {
             scan.setTimeRange(MIN_TABLE_TIMESTAMP, clientTimeStamp);
@@ -1103,7 +726,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             for (int i = 0; i < keys.size(); i++) {
                 schema = null;
                 schema = getSchema(scanner, clientTimeStamp);
-                if (schema == null) { return null; }
+                if (schema == null) {
+                    return null;
+                }
                 metaDataCache.put(cacheKey, schema);
                 schemas.add(schema);
             }
@@ -1112,11 +737,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private void addIndexToTable(PName tenantId, PName schemaName, PName indexName, PName tableName,
-            long clientTimeStamp, List<PTable> indexes, int clientVersion, boolean skipAddingParentColumns)
+                                 long clientTimeStamp, List<PTable> indexes, int clientVersion)
             throws IOException, SQLException {
         byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         PTable indexTable = doGetTable(tenantIdBytes, schemaName.getBytes(), indexName.getBytes(), clientTimeStamp,
-                null, clientVersion, false, skipAddingParentColumns, null);
+                null, clientVersion);
         if (indexTable == null) {
             ServerUtil.throwIOException("Index not found", new TableNotFoundException(schemaName.getString(), indexName.getString()));
             return;
@@ -1130,8 +755,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private void addColumnToTable(List<Cell> results, PName colName, PName famName,
-            Cell[] colKeyValues, List<PColumn> columns, boolean isSalted, int baseColumnCount,
-            boolean isRegularView) {
+                                  Cell[] colKeyValues, List<PColumn> columns, boolean isSalted, int baseColumnCount,
+                                  boolean isRegularView) {
         int i = 0;
         int j = 0;
         while (i < results.size() && j < COLUMN_KV_COLUMNS.size()) {
@@ -1139,8 +764,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             Cell searchKv = COLUMN_KV_COLUMNS.get(j);
             int cmp =
                     Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
-                        kv.getQualifierLength(), searchKv.getQualifierArray(),
-                        searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+                            kv.getQualifierLength(), searchKv.getQualifierArray(),
+                            searchKv.getQualifierOffset(), searchKv.getQualifierLength());
             if (cmp == 0) {
                 colKeyValues[j++] = kv;
                 i++;
@@ -1156,37 +781,29 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             throw new IllegalStateException("Didn't find all required key values in '"
                     + colName.getString() + "' column metadata row");
         }
-        
+
         Cell columnSizeKv = colKeyValues[COLUMN_SIZE_INDEX];
         Integer maxLength =
                 columnSizeKv == null ? null : PInteger.INSTANCE.getCodec().decodeInt(
-                    columnSizeKv.getValueArray(), columnSizeKv.getValueOffset(), SortOrder.getDefault());
+                        columnSizeKv.getValueArray(), columnSizeKv.getValueOffset(), SortOrder.getDefault());
         Cell decimalDigitKv = colKeyValues[DECIMAL_DIGITS_INDEX];
         Integer scale =
                 decimalDigitKv == null ? null : PInteger.INSTANCE.getCodec().decodeInt(
-                    decimalDigitKv.getValueArray(), decimalDigitKv.getValueOffset(), SortOrder.getDefault());
+                        decimalDigitKv.getValueArray(), decimalDigitKv.getValueOffset(), SortOrder.getDefault());
         Cell ordinalPositionKv = colKeyValues[ORDINAL_POSITION_INDEX];
         int position =
-            PInteger.INSTANCE.getCodec().decodeInt(ordinalPositionKv.getValueArray(),
-                    ordinalPositionKv.getValueOffset(), SortOrder.getDefault()) + (isSalted ? 1 : 0);;
+                PInteger.INSTANCE.getCodec().decodeInt(ordinalPositionKv.getValueArray(),
+                        ordinalPositionKv.getValueOffset(), SortOrder.getDefault()) + (isSalted ? 1 : 0);
+        ;
 
-        // Prior to PHOENIX-4766 we were sending the parent table column metadata while creating a
-        // child view, now that we combine columns by resolving the parent table hierarchy we
-        // don't need to include the parent table column while loading the PTable of the view
-        if (isRegularView && position <= baseColumnCount) {
-            return;
-        }
-        
-        // if this column was inherited from a parent and was dropped that we create an excluded
-        // column, this check is only needed to handle view metadata that was created before
-        // PHOENIX-4766 where we were sending the parent table column metadata when creating a
-        // childview
+        // if this column was inherited from a parent and was dropped then we create an excluded column
+        // which will be used to exclude the parent column while combining columns from ancestors
         Cell excludedColumnKv = colKeyValues[EXCLUDED_COLUMN_LINK_TYPE_KV_INDEX];
         if (excludedColumnKv != null && colKeyValues[DATA_TYPE_INDEX]
                 .getTimestamp() <= excludedColumnKv.getTimestamp()) {
             LinkType linkType =
                     LinkType.fromSerializedValue(
-                        excludedColumnKv.getValueArray()[excludedColumnKv.getValueOffset()]);
+                            excludedColumnKv.getValueArray()[excludedColumnKv.getValueOffset()]);
             if (linkType == LinkType.EXCLUDED_COLUMN) {
                 addExcludedColumnToTable(columns, colName, famName, excludedColumnKv.getTimestamp());
             } else {
@@ -1198,18 +815,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             return;
         }
-        
+
         Cell nullableKv = colKeyValues[NULLABLE_INDEX];
         boolean isNullable =
-            PInteger.INSTANCE.getCodec().decodeInt(nullableKv.getValueArray(),
-                    nullableKv.getValueOffset(), SortOrder.getDefault()) != ResultSetMetaData.columnNoNulls;
+                PInteger.INSTANCE.getCodec().decodeInt(nullableKv.getValueArray(),
+                        nullableKv.getValueOffset(), SortOrder.getDefault()) != ResultSetMetaData.columnNoNulls;
         Cell dataTypeKv = colKeyValues[DATA_TYPE_INDEX];
         PDataType dataType =
                 PDataType.fromTypeId(PInteger.INSTANCE.getCodec().decodeInt(
-                  dataTypeKv.getValueArray(), dataTypeKv.getValueOffset(), SortOrder.getDefault()));
+                        dataTypeKv.getValueArray(), dataTypeKv.getValueOffset(), SortOrder.getDefault()));
         if (maxLength == null && dataType == PBinary.INSTANCE) dataType = PVarbinary.INSTANCE;   // For
-                                                                                               // backward
-                                                                                               // compatibility.
+        // backward
+        // compatibility.
         Cell sortOrderKv = colKeyValues[SORT_ORDER_INDEX];
         SortOrder sortOrder =
                 sortOrderKv == null ? SortOrder.getDefault() : SortOrder.fromSystemValue(PInteger.INSTANCE
@@ -1218,14 +835,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         Cell arraySizeKv = colKeyValues[ARRAY_SIZE_INDEX];
         Integer arraySize = arraySizeKv == null ? null :
-            PInteger.INSTANCE.getCodec().decodeInt(arraySizeKv.getValueArray(), arraySizeKv.getValueOffset(), SortOrder.getDefault());
+                PInteger.INSTANCE.getCodec().decodeInt(arraySizeKv.getValueArray(), arraySizeKv.getValueOffset(), SortOrder.getDefault());
 
         Cell viewConstantKv = colKeyValues[VIEW_CONSTANT_INDEX];
         byte[] viewConstant = viewConstantKv == null ? null : viewConstantKv.getValue();
         Cell isViewReferencedKv = colKeyValues[IS_VIEW_REFERENCED_INDEX];
         boolean isViewReferenced = isViewReferencedKv != null && Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(isViewReferencedKv.getValueArray(), isViewReferencedKv.getValueOffset(), isViewReferencedKv.getValueLength()));
         Cell columnDefKv = colKeyValues[COLUMN_DEF_INDEX];
-        String expressionStr = columnDefKv==null ? null : (String)PVarchar.INSTANCE.toObject(columnDefKv.getValueArray(), columnDefKv.getValueOffset(), columnDefKv.getValueLength());
+        String expressionStr = columnDefKv == null ? null : (String) PVarchar.INSTANCE.toObject(columnDefKv.getValueArray(), columnDefKv.getValueOffset(), columnDefKv.getValueLength());
         Cell isRowTimestampKV = colKeyValues[IS_ROW_TIMESTAMP_INDEX];
         boolean isRowTimestamp =
                 isRowTimestampKV == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
@@ -1236,10 +853,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Cell columnQualifierKV = colKeyValues[COLUMN_QUALIFIER_INDEX];
         // Older tables won't have column qualifier metadata present. To make things simpler, just set the
         // column qualifier bytes by using the column name.
-        byte[] columnQualifierBytes = columnQualifierKV != null ? 
+        byte[] columnQualifierBytes = columnQualifierKV != null ?
                 Arrays.copyOfRange(columnQualifierKV.getValueArray(),
-                    columnQualifierKV.getValueOffset(), columnQualifierKV.getValueOffset()
-                            + columnQualifierKV.getValueLength()) : (isPkColumn ? null : colName.getBytes());
+                        columnQualifierKV.getValueOffset(), columnQualifierKV.getValueOffset()
+                                + columnQualifierKV.getValueLength()) : (isPkColumn ? null : colName.getBytes());
         PColumn column =
                 new PColumnImpl(colName, famName, dataType, maxLength, scale, isNullable,
                         position - 1, sortOrder, arraySize, viewConstant, isViewReferenced,
@@ -1249,7 +866,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private void addArgumentToFunction(List<Cell> results, PName functionName, PName type,
-        Cell[] functionKeyValues, List<FunctionArgument> arguments, short argPosition) throws SQLException {
+                                       Cell[] functionKeyValues, List<FunctionArgument> arguments, short argPosition) throws SQLException {
         int i = 0;
         int j = 0;
         while (i < results.size() && j < FUNCTION_ARG_KV_COLUMNS.size()) {
@@ -1257,8 +874,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             Cell searchKv = FUNCTION_ARG_KV_COLUMNS.get(j);
             int cmp =
                     Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
-                        kv.getQualifierLength(), searchKv.getQualifierArray(),
-                        searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+                            kv.getQualifierLength(), searchKv.getQualifierArray(),
+                            searchKv.getQualifierOffset(), searchKv.getQualifierLength());
             if (cmp == 0) {
                 functionKeyValues[j++] = kv;
                 i++;
@@ -1272,28 +889,28 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Cell isArrayKv = functionKeyValues[IS_ARRAY_INDEX];
         boolean isArrayType =
                 isArrayKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
-                    isArrayKv.getValueArray(), isArrayKv.getValueOffset(),
-                    isArrayKv.getValueLength()));
+                        isArrayKv.getValueArray(), isArrayKv.getValueOffset(),
+                        isArrayKv.getValueLength()));
         Cell isConstantKv = functionKeyValues[IS_CONSTANT_INDEX];
         boolean isConstant =
                 isConstantKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(
-                    isConstantKv.getValueArray(), isConstantKv.getValueOffset(),
-                    isConstantKv.getValueLength()));
+                        isConstantKv.getValueArray(), isConstantKv.getValueOffset(),
+                        isConstantKv.getValueLength()));
         Cell defaultValueKv = functionKeyValues[DEFAULT_VALUE_INDEX];
         String defaultValue =
                 defaultValueKv == null ? null : (String) PVarchar.INSTANCE.toObject(
-                    defaultValueKv.getValueArray(), defaultValueKv.getValueOffset(),
-                    defaultValueKv.getValueLength());
+                        defaultValueKv.getValueArray(), defaultValueKv.getValueOffset(),
+                        defaultValueKv.getValueLength());
         Cell minValueKv = functionKeyValues[MIN_VALUE_INDEX];
         String minValue =
                 minValueKv == null ? null : (String) PVarchar.INSTANCE.toObject(
-                    minValueKv.getValueArray(), minValueKv.getValueOffset(),
-                    minValueKv.getValueLength());
+                        minValueKv.getValueArray(), minValueKv.getValueOffset(),
+                        minValueKv.getValueLength());
         Cell maxValueKv = functionKeyValues[MAX_VALUE_INDEX];
         String maxValue =
                 maxValueKv == null ? null : (String) PVarchar.INSTANCE.toObject(
-                    maxValueKv.getValueArray(), maxValueKv.getValueOffset(),
-                    maxValueKv.getValueLength());
+                        maxValueKv.getValueArray(), maxValueKv.getValueOffset(),
+                        maxValueKv.getValueLength());
         FunctionArgument arg =
                 new FunctionArgument(type.getString(), isArrayType, isConstant,
                         defaultValue == null ? null : LiteralExpression.newConstant((new LiteralParseNode(defaultValue)).getValue()),
@@ -1303,14 +920,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         arguments.add(arg);
     }
 
-    /**
-     * @param skipAddingIndexes if true the returned PTable for a table or view won't include indexes
-     * @param skipAddingParentColumns if true the returned PTable won't include inherited columns
-     * @return PTable 
-     */
     private PTable getTable(RegionScanner scanner, long clientTimeStamp, long tableTimeStamp,
-            int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns)
-        throws IOException, SQLException {
+                            int clientVersion)
+            throws IOException, SQLException {
         List<Cell> results = Lists.newArrayList();
         scanner.next(results);
         if (results.isEmpty()) {
@@ -1329,12 +941,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         if (tenantIdLength == 0) {
             tenantId = null;
         }
-        PName schemaName = newPName(keyBuffer, keyOffset+tenantIdLength+1, keyLength);
+        PName schemaName = newPName(keyBuffer, keyOffset + tenantIdLength + 1, keyLength);
         int schemaNameLength = schemaName.getBytes().length;
         int tableNameLength = keyLength - schemaNameLength - 1 - tenantIdLength - 1;
         byte[] tableNameBytes = new byte[tableNameLength];
         System.arraycopy(keyBuffer, keyOffset + schemaNameLength + 1 + tenantIdLength + 1,
-            tableNameBytes, 0, tableNameLength);
+                tableNameBytes, 0, tableNameLength);
         PName tableName = PNameFactory.newName(tableNameBytes);
 
         int offset = tenantIdLength + schemaNameLength + tableNameLength + 3;
@@ -1358,11 +970,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             Cell searchKv = TABLE_KV_COLUMNS.get(j);
             int cmp =
                     Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
-                        kv.getQualifierLength(), searchKv.getQualifierArray(),
-                        searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+                            kv.getQualifierLength(), searchKv.getQualifierArray(),
+                            searchKv.getQualifierOffset(), searchKv.getQualifierLength());
             if (cmp == 0) {
                 timeStamp = Math.max(timeStamp, kv.getTimestamp()); // Find max timestamp of table
-                                                                    // header row
+                // header row
                 tableKeyValues[j++] = kv;
                 i++;
             } else if (cmp > 0) {
@@ -1375,8 +987,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // TABLE_TYPE, TABLE_SEQ_NUM and COLUMN_COUNT are required.
         if (tableKeyValues[TABLE_TYPE_INDEX] == null || tableKeyValues[TABLE_SEQ_NUM_INDEX] == null
                 || tableKeyValues[COLUMN_COUNT_INDEX] == null) {
-            throw new IllegalStateException(
-                    "Didn't find expected key values for table row in metadata row");
+            // since we allow SYSTEM.CATALOG to split in certain cases there might be child links or
+            // other metadata rows that are invalid and should be ignored
+            Cell cell = results.get(0);
+            LOGGER.error("Found invalid metadata rows for rowkey " +
+                    Bytes.toString(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength()));
+            return null;
         }
 
         Cell tableTypeKv = tableKeyValues[TABLE_TYPE_INDEX];
@@ -1385,27 +1001,27 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         .fromSerializedValue(tableTypeKv.getValueArray()[tableTypeKv.getValueOffset()]);
         Cell tableSeqNumKv = tableKeyValues[TABLE_SEQ_NUM_INDEX];
         long tableSeqNum =
-            PLong.INSTANCE.getCodec().decodeLong(tableSeqNumKv.getValueArray(),
-                    tableSeqNumKv.getValueOffset(), SortOrder.getDefault());
+                PLong.INSTANCE.getCodec().decodeLong(tableSeqNumKv.getValueArray(),
+                        tableSeqNumKv.getValueOffset(), SortOrder.getDefault());
         Cell columnCountKv = tableKeyValues[COLUMN_COUNT_INDEX];
         int columnCount =
-            PInteger.INSTANCE.getCodec().decodeInt(columnCountKv.getValueArray(),
-                    columnCountKv.getValueOffset(), SortOrder.getDefault());
+                PInteger.INSTANCE.getCodec().decodeInt(columnCountKv.getValueArray(),
+                        columnCountKv.getValueOffset(), SortOrder.getDefault());
         Cell pkNameKv = tableKeyValues[PK_NAME_INDEX];
         PName pkName =
                 pkNameKv != null ? newPName(pkNameKv.getValueArray(), pkNameKv.getValueOffset(),
-                    pkNameKv.getValueLength()) : null;
+                        pkNameKv.getValueLength()) : null;
         Cell saltBucketNumKv = tableKeyValues[SALT_BUCKETS_INDEX];
         Integer saltBucketNum =
                 saltBucketNumKv != null ? (Integer) PInteger.INSTANCE.getCodec().decodeInt(
-                    saltBucketNumKv.getValueArray(), saltBucketNumKv.getValueOffset(), SortOrder.getDefault()) : null;
+                        saltBucketNumKv.getValueArray(), saltBucketNumKv.getValueOffset(), SortOrder.getDefault()) : null;
         if (saltBucketNum != null && saltBucketNum.intValue() == 0) {
             saltBucketNum = null; // Zero salt buckets means not salted
         }
         Cell dataTableNameKv = tableKeyValues[DATA_TABLE_NAME_INDEX];
         PName dataTableName =
                 dataTableNameKv != null ? newPName(dataTableNameKv.getValueArray(),
-                    dataTableNameKv.getValueOffset(), dataTableNameKv.getValueLength()) : null;
+                        dataTableNameKv.getValueOffset(), dataTableNameKv.getValueLength()) : null;
         Cell indexStateKv = tableKeyValues[INDEX_STATE_INDEX];
         PIndexState indexState =
                 indexStateKv == null ? null : PIndexState.fromSerializedValue(indexStateKv
@@ -1414,8 +1030,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Cell immutableRowsKv = tableKeyValues[IMMUTABLE_ROWS_INDEX];
         boolean isImmutableRows =
                 immutableRowsKv == null ? false : (Boolean) PBoolean.INSTANCE.toObject(
-                    immutableRowsKv.getValueArray(), immutableRowsKv.getValueOffset(),
-                    immutableRowsKv.getValueLength());
+                        immutableRowsKv.getValueArray(), immutableRowsKv.getValueOffset(),
+                        immutableRowsKv.getValueLength());
         Cell defaultFamilyNameKv = tableKeyValues[DEFAULT_COLUMN_FAMILY_INDEX];
         PName defaultFamilyName = defaultFamilyNameKv != null ? newPName(defaultFamilyNameKv.getValueArray(), defaultFamilyNameKv.getValueOffset(), defaultFamilyNameKv.getValueLength()) : null;
         Cell viewStatementKv = tableKeyValues[VIEW_STATEMENT_INDEX];
@@ -1423,7 +1039,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 viewStatementKv.getValueLength()) : null;
         Cell disableWALKv = tableKeyValues[DISABLE_WAL_INDEX];
         boolean disableWAL = disableWALKv == null ? PTable.DEFAULT_DISABLE_WAL : Boolean.TRUE.equals(
-            PBoolean.INSTANCE.toObject(disableWALKv.getValueArray(), disableWALKv.getValueOffset(), disableWALKv.getValueLength()));
+                PBoolean.INSTANCE.toObject(disableWALKv.getValueArray(), disableWALKv.getValueOffset(), disableWALKv.getValueLength()));
         Cell multiTenantKv = tableKeyValues[MULTI_TENANT_INDEX];
         boolean multiTenant = multiTenantKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(multiTenantKv.getValueArray(), multiTenantKv.getValueOffset(), multiTenantKv.getValueLength()));
         Cell storeNullsKv = tableKeyValues[STORE_NULLS_INDEX];
@@ -1434,8 +1050,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         if (transactionProviderKv == null) {
             if (transactionalKv != null && Boolean.TRUE.equals(
                     PBoolean.INSTANCE.toObject(
-                            transactionalKv.getValueArray(), 
-                            transactionalKv.getValueOffset(), 
+                            transactionalKv.getValueArray(),
+                            transactionalKv.getValueOffset(),
                             transactionalKv.getValueLength()))) {
                 // For backward compat, prior to client setting TRANSACTION_PROVIDER
                 transactionProvider = TransactionFactory.Provider.TEPHRA;
@@ -1444,7 +1060,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             transactionProvider = TransactionFactory.Provider.fromCode(
                     PTinyint.INSTANCE.getCodec().decodeByte(
                             transactionProviderKv.getValueArray(),
-                            transactionProviderKv.getValueOffset(), 
+                            transactionProviderKv.getValueOffset(),
                             SortOrder.getDefault()));
         }
         Cell viewTypeKv = tableKeyValues[VIEW_TYPE_INDEX];
@@ -1455,13 +1071,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         IndexType indexType = indexTypeKv == null ? null : IndexType.fromSerializedValue(indexTypeKv.getValueArray()[indexTypeKv.getValueOffset()]);
         Cell baseColumnCountKv = tableKeyValues[BASE_COLUMN_COUNT_INDEX];
         int baseColumnCount = baseColumnCountKv == null ? 0 : PInteger.INSTANCE.getCodec().decodeInt(baseColumnCountKv.getValueArray(),
-            baseColumnCountKv.getValueOffset(), SortOrder.getDefault());
+                baseColumnCountKv.getValueOffset(), SortOrder.getDefault());
         Cell rowKeyOrderOptimizableKv = tableKeyValues[ROW_KEY_ORDER_OPTIMIZABLE_INDEX];
         boolean rowKeyOrderOptimizable = rowKeyOrderOptimizableKv == null ? false : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(rowKeyOrderOptimizableKv.getValueArray(), rowKeyOrderOptimizableKv.getValueOffset(), rowKeyOrderOptimizableKv.getValueLength()));
         Cell updateCacheFrequencyKv = tableKeyValues[UPDATE_CACHE_FREQUENCY_INDEX];
         long updateCacheFrequency = updateCacheFrequencyKv == null ? 0 :
-            PLong.INSTANCE.getCodec().decodeLong(updateCacheFrequencyKv.getValueArray(),
-                    updateCacheFrequencyKv.getValueOffset(), SortOrder.getDefault());
+                PLong.INSTANCE.getCodec().decodeLong(updateCacheFrequencyKv.getValueArray(),
+                        updateCacheFrequencyKv.getValueOffset(), SortOrder.getDefault());
 
         // Check the cell tag to see whether the view has modified this property
         final byte[] tagUpdateCacheFreq = (updateCacheFrequencyKv == null) ?
@@ -1474,23 +1090,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Cell isNamespaceMappedKv = tableKeyValues[IS_NAMESPACE_MAPPED_INDEX];
         boolean isNamespaceMapped = isNamespaceMappedKv == null ? false
                 : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(isNamespaceMappedKv.getValueArray(),
-                        isNamespaceMappedKv.getValueOffset(), isNamespaceMappedKv.getValueLength()));
+                isNamespaceMappedKv.getValueOffset(), isNamespaceMappedKv.getValueLength()));
         Cell autoPartitionSeqKv = tableKeyValues[AUTO_PARTITION_SEQ_INDEX];
         String autoPartitionSeq = autoPartitionSeqKv != null ? (String) PVarchar.INSTANCE.toObject(autoPartitionSeqKv.getValueArray(), autoPartitionSeqKv.getValueOffset(),
-            autoPartitionSeqKv.getValueLength()) : null;
+                autoPartitionSeqKv.getValueLength()) : null;
         Cell isAppendOnlySchemaKv = tableKeyValues[APPEND_ONLY_SCHEMA_INDEX];
         boolean isAppendOnlySchema = isAppendOnlySchemaKv == null ? false
                 : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(isAppendOnlySchemaKv.getValueArray(),
-                    isAppendOnlySchemaKv.getValueOffset(), isAppendOnlySchemaKv.getValueLength()));
+                isAppendOnlySchemaKv.getValueOffset(), isAppendOnlySchemaKv.getValueLength()));
         Cell storageSchemeKv = tableKeyValues[STORAGE_SCHEME_INDEX];
         //TODO: change this once we start having other values for storage schemes
         ImmutableStorageScheme storageScheme = storageSchemeKv == null ? ImmutableStorageScheme.ONE_CELL_PER_COLUMN : ImmutableStorageScheme
-                .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
+                .fromSerializedValue((byte) PTinyint.INSTANCE.toObject(storageSchemeKv.getValueArray(),
                         storageSchemeKv.getValueOffset(), storageSchemeKv.getValueLength()));
         Cell encodingSchemeKv = tableKeyValues[QUALIFIER_ENCODING_SCHEME_INDEX];
         QualifierEncodingScheme encodingScheme = encodingSchemeKv == null ? QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : QualifierEncodingScheme
-                .fromSerializedValue((byte)PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
-                    encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
+                .fromSerializedValue((byte) PTinyint.INSTANCE.toObject(encodingSchemeKv.getValueArray(),
+                        encodingSchemeKv.getValueOffset(), encodingSchemeKv.getValueLength()));
         Cell useStatsForParallelizationKv = tableKeyValues[USE_STATS_FOR_PARALLELIZATION_INDEX];
         Boolean useStatsForParallelization = useStatsForParallelizationKv == null ? null : Boolean.TRUE.equals(PBoolean.INSTANCE.toObject(useStatsForParallelizationKv.getValueArray(), useStatsForParallelizationKv.getValueOffset(), useStatsForParallelizationKv.getValueLength()));
 
@@ -1499,7 +1115,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 HConstants.EMPTY_BYTE_ARRAY : CellUtil.getTagArray(useStatsForParallelizationKv);
         boolean viewModifiedUseStatsForParallelization = (PTableType.VIEW.equals(tableType)) &&
                 Bytes.contains(tagUseStatsForParallelization, VIEW_MODIFIED_PROPERTY_BYTES);
-        
+
         List<PColumn> columns = Lists.newArrayListWithExpectedSize(columnCount);
         List<PTable> indexes = Lists.newArrayList();
         List<PName> physicalTables = Lists.newArrayList();
@@ -1508,37 +1124,37 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         EncodedCQCounter cqCounter =
                 (!EncodedColumnsUtil.usesEncodedColumnNames(encodingScheme) || tableType == PTableType.VIEW) ? PTable.EncodedCQCounter.NULL_COUNTER
                         : new EncodedCQCounter();
-        boolean isRegularView = (tableType == PTableType.VIEW && viewType!=ViewType.MAPPED);
+        boolean isRegularView = (tableType == PTableType.VIEW && viewType != ViewType.MAPPED);
         while (true) {
-          results.clear();
-          scanner.next(results);
-          if (results.isEmpty()) {
-              break;
-          }
-          Cell colKv = results.get(LINK_TYPE_INDEX);
-          int colKeyLength = colKv.getRowLength();
-          PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength-offset);
-          int colKeyOffset = offset + colName.getBytes().length + 1;
-          PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength-colKeyOffset);
-          if (isQualifierCounterKV(colKv)) {
-              Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
-              cqCounter.setValue(famName.getString(), value);
-          } else if (Bytes.compareTo(LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length, colKv.getQualifierArray(), colKv.getQualifierOffset(), colKv.getQualifierLength())==0) {
-              LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
-              if (linkType == LinkType.INDEX_TABLE && !skipAddingIndexes) {
-                  addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion, skipAddingParentColumns);
-              } else if (linkType == LinkType.PHYSICAL_TABLE) {
-                  physicalTables.add(famName);
-              } else if (linkType == LinkType.PARENT_TABLE) {
-                  parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
-                  parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
-              } else if (linkType == LinkType.EXCLUDED_COLUMN) {
-                  // add the excludedColumn
-                  addExcludedColumnToTable(columns, colName, famName, colKv.getTimestamp());
-              }
-          } else {
-              addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null, baseColumnCount, isRegularView);
-          }
+            results.clear();
+            scanner.next(results);
+            if (results.isEmpty()) {
+                break;
+            }
+            Cell colKv = results.get(LINK_TYPE_INDEX);
+            int colKeyLength = colKv.getRowLength();
+            PName colName = newPName(colKv.getRowArray(), colKv.getRowOffset() + offset, colKeyLength - offset);
+            int colKeyOffset = offset + colName.getBytes().length + 1;
+            PName famName = newPName(colKv.getRowArray(), colKv.getRowOffset() + colKeyOffset, colKeyLength - colKeyOffset);
+            if (isQualifierCounterKV(colKv)) {
+                Integer value = PInteger.INSTANCE.getCodec().decodeInt(colKv.getValueArray(), colKv.getValueOffset(), SortOrder.ASC);
+                cqCounter.setValue(famName.getString(), value);
+            } else if (Bytes.compareTo(LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length, colKv.getQualifierArray(), colKv.getQualifierOffset(), colKv.getQualifierLength()) == 0) {
+                LinkType linkType = LinkType.fromSerializedValue(colKv.getValueArray()[colKv.getValueOffset()]);
+                if (linkType == LinkType.INDEX_TABLE) {
+                    addIndexToTable(tenantId, schemaName, famName, tableName, clientTimeStamp, indexes, clientVersion);
+                } else if (linkType == LinkType.PHYSICAL_TABLE) {
+                    physicalTables.add(famName);
+                } else if (linkType == LinkType.PARENT_TABLE) {
+                    parentTableName = PNameFactory.newName(SchemaUtil.getTableNameFromFullName(famName.getBytes()));
+                    parentSchemaName = PNameFactory.newName(SchemaUtil.getSchemaNameFromFullName(famName.getBytes()));
+                } else if (linkType == LinkType.EXCLUDED_COLUMN) {
+                    // add the excludedColumn
+                    addExcludedColumnToTable(columns, colName, famName, colKv.getTimestamp());
+                }
+            } else {
+                addColumnToTable(results, colName, famName, colKeyValues, columns, saltBucketNum != null, baseColumnCount, isRegularView);
+            }
         }
         // Avoid querying the stats table because we're holding the rowLock here. Issuing an RPC to a remote
         // server while holding this lock is a bad idea and likely to cause contention.
@@ -1587,6 +1203,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 .setColumns(columns)
                 .build();
     }
+
     private Long getViewIndexId(Cell[] tableKeyValues, PDataType viewIndexIdType) {
         Cell viewIndexIdKv = tableKeyValues[VIEW_INDEX_ID_INDEX];
         return viewIndexIdKv == null ? null :
@@ -1644,15 +1261,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private boolean isQualifierCounterKV(Cell kv) {
         int cmp =
                 Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
-                    kv.getQualifierLength(), QUALIFIER_COUNTER_KV.getQualifierArray(),
-                    QUALIFIER_COUNTER_KV.getQualifierOffset(), QUALIFIER_COUNTER_KV.getQualifierLength());
+                        kv.getQualifierLength(), QUALIFIER_COUNTER_KV.getQualifierArray(),
+                        QUALIFIER_COUNTER_KV.getQualifierOffset(), QUALIFIER_COUNTER_KV.getQualifierLength());
         return cmp == 0;
     }
 
     private PSchema getSchema(RegionScanner scanner, long clientTimeStamp) throws IOException, SQLException {
         List<Cell> results = Lists.newArrayList();
         scanner.next(results);
-        if (results.isEmpty()) { return null; }
+        if (results.isEmpty()) {
+            return null;
+        }
 
         Cell keyValue = results.get(0);
         byte[] keyBuffer = keyValue.getRowArray();
@@ -1683,11 +1302,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         int keyLength = keyValue.getRowLength();
         int keyOffset = keyValue.getRowOffset();
         long currentTimeMillis = EnvironmentEdgeManager.currentTimeMillis();
-        if(isReplace) {
+        if (isReplace) {
             long deleteTimeStamp =
                     clientTimeStamp == HConstants.LATEST_TIMESTAMP ? currentTimeMillis - 1
                             : (keyValue.getTimestamp() < clientTimeStamp ? clientTimeStamp - 1
-                                    : keyValue.getTimestamp());
+                            : keyValue.getTimestamp());
             deleteMutationsForReplace.add(new Delete(keyBuffer, keyOffset, keyLength, deleteTimeStamp));
         }
         PName tenantId = newPName(keyBuffer, keyOffset, keyLength);
@@ -1697,7 +1316,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         PName functionName =
                 newPName(keyBuffer, keyOffset + tenantIdLength + 1, keyLength - tenantIdLength - 1);
-        int functionNameLength = functionName.getBytes().length+1;
+        int functionNameLength = functionName.getBytes().length + 1;
         int offset = tenantIdLength + functionNameLength + 1;
 
         long timeStamp = keyValue.getTimestamp();
@@ -1709,11 +1328,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             Cell searchKv = FUNCTION_KV_COLUMNS.get(j);
             int cmp =
                     Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(),
-                        kv.getQualifierLength(), searchKv.getQualifierArray(),
-                        searchKv.getQualifierOffset(), searchKv.getQualifierLength());
+                            kv.getQualifierLength(), searchKv.getQualifierArray(),
+                            searchKv.getQualifierOffset(), searchKv.getQualifierLength());
             if (cmp == 0) {
                 timeStamp = Math.max(timeStamp, kv.getTimestamp()); // Find max timestamp of table
-                                                                    // header row
+                // header row
                 functionKeyValues[j++] = kv;
                 i++;
             } else if (cmp > 0) {
@@ -1731,21 +1350,21 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
         Cell classNameKv = functionKeyValues[CLASS_NAME_INDEX];
         PName className = newPName(classNameKv.getValueArray(), classNameKv.getValueOffset(),
-            classNameKv.getValueLength());
+                classNameKv.getValueLength());
         Cell jarPathKv = functionKeyValues[JAR_PATH_INDEX];
         PName jarPath = null;
-        if(jarPathKv != null) {
+        if (jarPathKv != null) {
             jarPath = newPName(jarPathKv.getValueArray(), jarPathKv.getValueOffset(),
-                jarPathKv.getValueLength());
+                    jarPathKv.getValueLength());
         }
         Cell numArgsKv = functionKeyValues[NUM_ARGS_INDEX];
         int numArgs =
                 PInteger.INSTANCE.getCodec().decodeInt(numArgsKv.getValueArray(),
-                    numArgsKv.getValueOffset(), SortOrder.getDefault());
+                        numArgsKv.getValueOffset(), SortOrder.getDefault());
         Cell returnTypeKv = functionKeyValues[RETURN_TYPE_INDEX];
         PName returnType =
                 returnTypeKv == null ? null : newPName(returnTypeKv.getValueArray(),
-                    returnTypeKv.getValueOffset(), returnTypeKv.getValueLength());
+                        returnTypeKv.getValueOffset(), returnTypeKv.getValueLength());
 
         List<FunctionArgument> arguments = Lists.newArrayListWithExpectedSize(numArgs);
         for (int k = 0; k < numArgs; k++) {
@@ -1755,11 +1374,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 break;
             }
             Cell typeKv = results.get(0);
-            if(isReplace) {
+            if (isReplace) {
                 long deleteTimeStamp =
                         clientTimeStamp == HConstants.LATEST_TIMESTAMP ? currentTimeMillis - 1
                                 : (typeKv.getTimestamp() < clientTimeStamp ? clientTimeStamp - 1
-                                        : typeKv.getTimestamp());
+                                : typeKv.getTimestamp());
                 deleteMutationsForReplace.add(new Delete(typeKv.getRowArray(), typeKv
                         .getRowOffset(), typeKv.getRowLength(), deleteTimeStamp));
             }
@@ -1768,9 +1387,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     newPName(typeKv.getRowArray(), typeKv.getRowOffset() + offset, typeKeyLength
                             - offset - 3);
 
-            int argPositionOffset =  offset + typeName.getBytes().length + 1;
+            int argPositionOffset = offset + typeName.getBytes().length + 1;
             short argPosition = Bytes.toShort(typeKv.getRowArray(), typeKv.getRowOffset() + argPositionOffset, typeKeyLength
-                - argPositionOffset);
+                    - argPositionOffset);
             addArgumentToFunction(results, functionName, typeName, functionArgKeyValues, arguments, argPosition);
         }
         Collections.sort(arguments, new Comparator<FunctionArgument>() {
@@ -1784,7 +1403,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private PTable buildDeletedTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
-        long clientTimeStamp) throws IOException {
+                                     long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
             return null;
         }
@@ -1792,9 +1411,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Scan scan = MetaDataUtil.newTableRowsScan(key, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
         scan.setFilter(new FirstKeyOnlyFilter());
         scan.setRaw(true);
-        List<Cell> results = Lists.<Cell> newArrayList();
+        List<Cell> results = Lists.<Cell>newArrayList();
         try (RegionScanner scanner = region.getScanner(scan)) {
-          scanner.next(results);
+            scanner.next(results);
         }
         for (Cell kv : results) {
             KeyValue.Type type = Type.codeToType(kv.getTypeByte());
@@ -1811,7 +1430,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
 
     private PFunction buildDeletedFunction(byte[] key, ImmutableBytesPtr cacheKey, Region region,
-        long clientTimeStamp) throws IOException {
+                                           long clientTimeStamp) throws IOException {
         if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
             return null;
         }
@@ -1819,9 +1438,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         Scan scan = MetaDataUtil.newTableRowsScan(key, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
         scan.setFilter(new FirstKeyOnlyFilter());
         scan.setRaw(true);
-        List<Cell> results = Lists.<Cell> newArrayList();
+        List<Cell> results = Lists.<Cell>newArrayList();
         try (RegionScanner scanner = region.getScanner(scan);) {
-          scanner.next(results);
+            scanner.next(results);
         }
         // HBase ignores the time range on a raw scan (HBASE-7362)
         if (!results.isEmpty() && results.get(0).getTimestamp() > clientTimeStamp) {
@@ -1839,12 +1458,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private PSchema buildDeletedSchema(byte[] key, ImmutableBytesPtr cacheKey, Region region, long clientTimeStamp)
             throws IOException {
-        if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) { return null; }
+        if (clientTimeStamp == HConstants.LATEST_TIMESTAMP) {
+            return null;
+        }
 
         Scan scan = MetaDataUtil.newTableRowsScan(key, clientTimeStamp, HConstants.LATEST_TIMESTAMP);
         scan.setFilter(new FirstKeyOnlyFilter());
         scan.setRaw(true);
-        List<Cell> results = Lists.<Cell> newArrayList();
+        List<Cell> results = Lists.<Cell>newArrayList();
         try (RegionScanner scanner = region.getScanner(scan);) {
             scanner.next(results);
         }
@@ -1901,12 +1522,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private PTable loadTable(RegionCoprocessorEnvironment env, byte[] key,
-        ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
-        throws IOException, SQLException {
+                             ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, int clientVersion)
+            throws IOException, SQLException {
         Region region = env.getRegion();
-        PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion, false, false, null);
+        PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion);
         // We always cache the latest version - fault in if not in cache
-        if (table != null || (table = buildTable(key, cacheKey, region, asOfTimeStamp, clientVersion, false, false, null)) != null) {
+        if (table != null || (table = buildTable(key, cacheKey, region, asOfTimeStamp, clientVersion)) != null) {
             return table;
         }
         // if not found then check if newer table already exists and add delete marker for timestamp
@@ -1917,48 +1538,48 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         return null;
     }
-    
+
     /**
      * Returns a PTable if its found in the cache.
      */
-   private PTable getTableFromCache(ImmutableBytesPtr cacheKey, long clientTimeStamp, int clientVersion, boolean skipAddingIndexes, boolean skipAddingParentColumns, PTable lockedAncestorTable) throws SQLException, IOException {
-       Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-       PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
-       if (table!=null)
-           table = combineColumns(table, clientTimeStamp, clientVersion, skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable).getFirst();
-       return table;
-   }
+    private PTable getTableFromCache(ImmutableBytesPtr cacheKey, long clientTimeStamp, int clientVersion) {
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        PTable table = (PTable) metaDataCache.getIfPresent(cacheKey);
+        return table;
+    }
 
     private PFunction loadFunction(RegionCoprocessorEnvironment env, byte[] key,
-            ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace)
+                                   ImmutableBytesPtr cacheKey, long clientTimeStamp, long asOfTimeStamp, boolean isReplace, List<Mutation> deleteMutationsForReplace)
             throws IOException, SQLException {
-            Region region = env.getRegion();
-            Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-            PFunction function = (PFunction)metaDataCache.getIfPresent(cacheKey);
-            // We always cache the latest version - fault in if not in cache
-            if (function != null && !isReplace) {
-                return function;
-            }
-            ArrayList<byte[]> arrayList = new ArrayList<byte[]>(1);
-            arrayList.add(key);
-            List<PFunction> functions = buildFunctions(arrayList, region, asOfTimeStamp, isReplace, deleteMutationsForReplace);
-            if(functions != null) return functions.get(0);
-            // if not found then check if newer table already exists and add delete marker for timestamp
-            // found
-            if (function == null
-                    && (function = buildDeletedFunction(key, cacheKey, region, clientTimeStamp)) != null) {
-                return function;
-            }
-            return null;
+        Region region = env.getRegion();
+        Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+        PFunction function = (PFunction) metaDataCache.getIfPresent(cacheKey);
+        // We always cache the latest version - fault in if not in cache
+        if (function != null && !isReplace) {
+            return function;
+        }
+        ArrayList<byte[]> arrayList = new ArrayList<byte[]>(1);
+        arrayList.add(key);
+        List<PFunction> functions = buildFunctions(arrayList, region, asOfTimeStamp, isReplace, deleteMutationsForReplace);
+        if (functions != null) return functions.get(0);
+        // if not found then check if newer table already exists and add delete marker for timestamp
+        // found
+        if (function == null
+                && (function = buildDeletedFunction(key, cacheKey, region, clientTimeStamp)) != null) {
+            return function;
         }
+        return null;
+    }
 
     private PSchema loadSchema(RegionCoprocessorEnvironment env, byte[] key, ImmutableBytesPtr cacheKey,
-            long clientTimeStamp, long asOfTimeStamp) throws IOException, SQLException {
+                               long clientTimeStamp, long asOfTimeStamp) throws IOException, SQLException {
         Region region = env.getRegion();
         Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-        PSchema schema = (PSchema)metaDataCache.getIfPresent(cacheKey);
+        PSchema schema = (PSchema) metaDataCache.getIfPresent(cacheKey);
         // We always cache the latest version - fault in if not in cache
-        if (schema != null) { return schema; }
+        if (schema != null) {
+            return schema;
+        }
         ArrayList<byte[]> arrayList = new ArrayList<byte[]>(1);
         arrayList.add(key);
         List<PSchema> schemas = buildSchemas(arrayList, region, asOfTimeStamp, cacheKey);
@@ -1966,14 +1587,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         // if not found then check if newer schema already exists and add delete marker for timestamp
         // found
         if (schema == null
-                && (schema = buildDeletedSchema(key, cacheKey, region, clientTimeStamp)) != null) { return schema; }
+                && (schema = buildDeletedSchema(key, cacheKey, region, clientTimeStamp)) != null) {
+            return schema;
+        }
         return null;
     }
 
     /**
      * @return null if the physical table row information is not present.
      */
-    private static Mutation getPhysicalTableRowForView(List<Mutation> tableMetadata, byte[][] parentTenantSchemaTableNames, byte[][] physicalSchemaTableNames) {
+    private static void getParentAndPhysicalNames(List<Mutation> tableMetadata, byte[][] parentTenantSchemaTableNames, byte[][] physicalSchemaTableNames) {
         int size = tableMetadata.size();
         byte[][] rowKeyMetaData = new byte[3][];
         MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
@@ -1992,11 +1615,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         physicalTableLinkFound = true;
                     }
                     if (linkType == LinkType.PARENT_TABLE) {
-                        parentTableRow=m;
+                        parentTableRow = m;
                         parentTableLinkFound = true;
                     }
                 }
-                if(physicalTableLinkFound && parentTableLinkFound){
+                if (physicalTableLinkFound && parentTableLinkFound) {
                     break;
                 }
                 i--;
@@ -2006,7 +1629,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             parentTenantSchemaTableNames[0] = null;
             parentTenantSchemaTableNames[1] = null;
             parentTenantSchemaTableNames[2] = null;
-            
+
         }
         if (!physicalTableLinkFound) {
             physicalSchemaTableNames[0] = null;
@@ -2014,14 +1637,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             physicalSchemaTableNames[2] = null;
         }
         if (physicalTableLinkFound) {
-            getSchemaTableNames(physicalTableRow,physicalSchemaTableNames);
+            getSchemaTableNames(physicalTableRow, physicalSchemaTableNames);
         }
         if (parentTableLinkFound) {
-            getSchemaTableNames(parentTableRow,parentTenantSchemaTableNames);   
+            getSchemaTableNames(parentTableRow, parentTenantSchemaTableNames);
         }
-        return physicalTableRow;
     }
-    
+
     private static void getSchemaTableNames(Mutation row, byte[][] schemaTableNames) {
         byte[][] rowKeyMetaData = new byte[5][];
         getVarChars(row.getRow(), 5, rowKeyMetaData);
@@ -2031,7 +1653,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         if ((colBytes == null || colBytes.length == 0) && (famBytes != null && famBytes.length > 0)) {
             byte[] sName = SchemaUtil.getSchemaNameFromFullName(famBytes).getBytes();
             byte[] tName = SchemaUtil.getTableNameFromFullName(famBytes).getBytes();
-            schemaTableNames[0]= tenantId;
+            schemaTableNames[0] = tenantId;
             schemaTableNames[1] = sName;
             schemaTableNames[2] = tName;
         }
@@ -2039,7 +1661,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     @Override
     public void createTable(RpcController controller, CreateTableRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                            RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         byte[][] rowKeyMetaData = new byte[3][];
         byte[] schemaName = null;
@@ -2057,27 +1679,20 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     new ImmutableBytesWritable());
             final IndexType indexType = MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
                     new ImmutableBytesWritable());
-            byte[] parentTenantId = null;
             byte[] parentSchemaName = null;
             byte[] parentTableName = null;
+            PTable parentTable = request.hasParentTable() ? PTableImpl.createFromProto(request.getParentTable()) : null;
             PTableType tableType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
-            ViewType viewType = MetaDataUtil.getViewType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
 
             // Load table to see if it already exists
             byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
             ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(tableKey);
             long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
             PTable table = null;
-            try {
-                // Get as of latest timestamp so we can detect if we have a newer table that already
-                // exists without making an additional query
-                table = loadTable(env, tableKey, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP,
-                        clientVersion);
-            } catch (ParentTableNotFoundException e) {
-                if (clientVersion >= MIN_SPLITTABLE_SYSTEM_CATALOG) {
-                    dropChildViews(env, e.getParentTenantId(), e.getParentSchemaName(), e.getParentTableName());
-                }
-            }
+            // Get as of latest timestamp so we can detect if we have a newer table that already
+            // exists without making an additional query
+            table = loadTable(env, tableKey, cacheKey, clientTimeStamp, HConstants.LATEST_TIMESTAMP,
+                    clientVersion);
             if (table != null) {
                 if (table.getTimeStamp() < clientTimeStamp) {
                     // If the table is older than the client time stamp and it's deleted,
@@ -2097,65 +1712,27 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return;
                 }
             }
-            
+
             // check if the table was dropped, but had child views that were have not yet been cleaned up
-            // We don't need to do this for older clients
             if (!Bytes.toString(schemaName).equals(QueryConstants.SYSTEM_SCHEMA_NAME) &&
                     clientVersion >= MIN_SPLITTABLE_SYSTEM_CATALOG) {
-                dropChildViews(env, tenantIdBytes, schemaName, tableName);
+                ViewUtil.dropChildViews(env, tenantIdBytes, schemaName, tableName);
             }
-            
+
             byte[] parentTableKey = null;
-            Mutation viewPhysicalTableRow = null;
-            Set<TableName> indexes = new HashSet<TableName>();;
+            Set<TableName> indexes = new HashSet<TableName>();
+            ;
             byte[] cPhysicalName = SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, isNamespaceMapped)
                     .getBytes();
-            byte[] cParentPhysicalName=null;
+            byte[] cParentPhysicalName = null;
             if (tableType == PTableType.VIEW) {
                 byte[][] parentSchemaTableNames = new byte[3][];
                 byte[][] parentPhysicalSchemaTableNames = new byte[3][];
-                /*
-                 * For a mapped view, there is no link present to the physical table. So the
-                 * viewPhysicalTableRow is null in that case.
-                 */
-                
-                viewPhysicalTableRow = getPhysicalTableRowForView(tableMetadata, parentSchemaTableNames,parentPhysicalSchemaTableNames);
+                getParentAndPhysicalNames(tableMetadata, parentSchemaTableNames, parentPhysicalSchemaTableNames);
                 if (parentPhysicalSchemaTableNames[2] != null) {
-                    
                     parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
                             parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
-                    PTable parentTable =
-                            doGetTable(ByteUtil.EMPTY_BYTE_ARRAY, parentPhysicalSchemaTableNames[1],
-                                parentPhysicalSchemaTableNames[2], clientTimeStamp, clientVersion);
-                    if (parentTable == null) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        done.run(builder.build());
-                        return;
-                    }
                     cParentPhysicalName = parentTable.getPhysicalName().getBytes();
-                    if (parentSchemaTableNames[2] != null
-                            && Bytes.compareTo(parentSchemaTableNames[2], parentPhysicalSchemaTableNames[2]) != 0) {
-                        // if view is created on view
-                        byte[] tenantId =
-                                parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY
-                                        : parentSchemaTableNames[0];
-                        parentTable =
-                                doGetTable(tenantId, parentSchemaTableNames[1],
-                                    parentSchemaTableNames[2], clientTimeStamp, clientVersion);
-                        if (parentTable == null) {
-                            // it could be a global view
-                            parentTable =
-                                    doGetTable(ByteUtil.EMPTY_BYTE_ARRAY, parentSchemaTableNames[1],
-                                        parentSchemaTableNames[2], clientTimeStamp, clientVersion);
-                        }
-                    }
-                    if (parentTable == null) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        done.run(builder.build());
-                        return;
-                    }
                     for (PTable index : parentTable.getIndexes()) {
                         indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
                     }
@@ -2163,41 +1740,36 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     // Mapped View
                     cParentPhysicalName = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
                 }
-                parentTenantId = ByteUtil.EMPTY_BYTE_ARRAY;
                 parentSchemaName = parentPhysicalSchemaTableNames[1];
                 parentTableName = parentPhysicalSchemaTableNames[2];
-                    
+
             } else if (tableType == PTableType.INDEX) {
                 parentSchemaName = schemaName;
-                /* 
+                /*
                  * For an index we lock the parent table's row which could be a physical table or a view.
                  * If the parent table is a physical table, then the tenantIdBytes is empty because
                  * we allow creating an index with a tenant connection only if the parent table is a view.
                  */
-                parentTenantId = tenantIdBytes;
                 parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
                 parentTableKey = SchemaUtil.getTableKey(tenantIdBytes, parentSchemaName, parentTableName);
-                PTable parentTable =
-                        doGetTable(tenantIdBytes, parentSchemaName, parentTableName, clientTimeStamp, null,
-                            request.getClientVersion(), false, false, null);
                 if (IndexType.LOCAL == indexType) {
                     cPhysicalName = parentTable.getPhysicalName().getBytes();
-                    cParentPhysicalName=parentTable.getPhysicalName().getBytes();
+                    cParentPhysicalName = parentTable.getPhysicalName().getBytes();
                 } else if (parentTable.getType() == PTableType.VIEW) {
                     cPhysicalName = MetaDataUtil.getViewIndexPhysicalName(parentTable.getPhysicalName().getBytes());
                     cParentPhysicalName = parentTable.getPhysicalName().getBytes();
-                }else{
+                } else {
                     cParentPhysicalName = SchemaUtil
                             .getPhysicalHBaseTableName(parentSchemaName, parentTableName, isNamespaceMapped).getBytes();
                 }
             }
-            
+
             getCoprocessorHost().preCreateTable(Bytes.toString(tenantIdBytes),
                     fullTableName,
                     (tableType == PTableType.VIEW) ? null : TableName.valueOf(cPhysicalName),
                     cParentPhysicalName == null ? null : TableName.valueOf(cParentPhysicalName), tableType,
                     /* TODO: During inital create we may not need the family map */
-                    Collections.<byte[]> emptySet(), indexes);
+                    Collections.<byte[]>emptySet(), indexes);
 
             Region region = env.getRegion();
             List<RowLock> locks = Lists.newArrayList();
@@ -2212,18 +1784,16 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return;
                 }
 
-                ImmutableBytesPtr parentCacheKey = null;
-                PTable parentTable = null;
                 if (parentTableName != null) {
                     // From 4.15 onwards we only need to lock the parent table :
                     // 1) when creating an index on a table or a view
-                    // 2) if allowSystemCatalogRollback is true we try to lock the parent table to prevent it
+                    // 2) if allowSplittableSystemCatalogRollback is true we try to lock the parent table to prevent it
                     // from changing concurrently while a view is being created
-                    if (tableType == PTableType.INDEX || allowSystemCatalogRollback) {
+                    if (tableType == PTableType.INDEX || allowSplittableSystemCatalogRollback) {
                         result = checkTableKeyInRegion(parentTableKey, region);
                         if (result != null) {
-                            LOGGER.error("Unable to lock parentTableKey "+Bytes.toStringBinary(parentTableKey));
-                            // if allowSystemCatalogRollback is true and we can't lock the parentTableKey (because
+                            LOGGER.error("Unable to lock parentTableKey " + Bytes.toStringBinary(parentTableKey));
+                            // if allowSplittableSystemCatalogRollback is true and we can't lock the parentTableKey (because
                             // SYSTEM.CATALOG already split) return UNALLOWED_TABLE_MUTATION so that the client
                             // knows the create statement failed
                             MetaDataProtos.MutationCode code = tableType == PTableType.INDEX ?
@@ -2236,15 +1806,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         }
                         acquireLock(region, parentTableKey, locks);
                     }
-                    parentTable =  doGetTable(parentTenantId, parentSchemaName, parentTableName,
-                        clientTimeStamp, null, clientVersion, false, false, null);
-                    parentCacheKey = new ImmutableBytesPtr(parentTableKey);
-                    if (parentTable == null || isTableDeleted(parentTable)) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        done.run(builder.build());
-                        return;
-                    }
                     // make sure we haven't gone over our threshold for indexes on this table.
                     if (execeededIndexQuota(tableType, parentTable)) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.TOO_MANY_INDEXES);
@@ -2252,27 +1813,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         done.run(builder.build());
                         return;
                     }
-                    long parentTableSeqNumber;
-                    if (tableType == PTableType.VIEW && viewPhysicalTableRow != null && request.hasClientVersion()) {
-                        // Starting 4.5, the client passes the sequence number of the physical table in the table metadata.
-                        parentTableSeqNumber = MetaDataUtil.getSequenceNumber(viewPhysicalTableRow);
-                    } else if (tableType == PTableType.VIEW && !request.hasClientVersion()) {
-                        // Before 4.5, due to a bug, the parent table key wasn't available.
-                        // So don't do anything and prevent the exception from being thrown.
-                        parentTableSeqNumber = parentTable.getSequenceNumber();
-                    } else {
-                        parentTableSeqNumber = MetaDataUtil.getParentSequenceNumber(tableMetadata);
-                    }
-                    // If parent table isn't at the expected sequence number, then return
-                    if (parentTable.getSequenceNumber() != parentTableSeqNumber) {
-                        builder.setReturnCode(MetaDataProtos.MutationCode.CONCURRENT_TABLE_MUTATION);
-                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
-                        builder.setTable(PTableImpl.toProto(parentTable));
-                        done.run(builder.build());
-                        return;
-                    }
                 }
-                
+
                 // Add cell for ROW_KEY_ORDER_OPTIMIZABLE = true, as we know that new tables
                 // conform the correct row key. The exception is for a VIEW, which the client
                 // sends over depending on its base physical table.
@@ -2281,10 +1823,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 // If the parent table of the view has the auto partition sequence name attribute, modify the
                 // tableMetadata and set the view statement and partition column correctly
-                if (parentTable!=null && parentTable.getAutoPartitionSeqName()!=null) {
+                if (parentTable != null && parentTable.getAutoPartitionSeqName() != null) {
                     long autoPartitionNum = 1;
                     try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
-                        Statement stmt = connection.createStatement()) {
+                         Statement stmt = connection.createStatement()) {
                         String seqName = parentTable.getAutoPartitionSeqName();
                         // Not going through the standard route of using statement.execute() as that code path
                         // is blocked if the metadata hasn't been been upgraded to the new minor release.
@@ -2295,8 +1837,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         PhoenixResultSet rs = ps.newResultSet(resultIterator, plan.getProjector(), plan.getContext());
                         rs.next();
                         autoPartitionNum = rs.getLong(1);
-                    }
-                    catch (SequenceNotFoundException e) {
+                    } catch (SequenceNotFoundException e) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.AUTO_PARTITION_SEQUENCE_NOT_FOUND);
                         builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                         done.run(builder.build());
@@ -2326,12 +1867,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     // if we have an existing where clause add the auto partition where clause to it
                     if (!Bytes.equals(value, QueryConstants.EMPTY_COLUMN_VALUE_BYTES)) {
                         viewStatement = Bytes.add(value, Bytes.toBytes(" AND "), Bytes.toBytes(autoPartitionWhere));
-                    }
-                    else {
+                    } else {
                         viewStatement = Bytes.toBytes(QueryUtil.getViewStatement(parentTable.getSchemaName().getString(), parentTable.getTableName().getString(), autoPartitionWhere));
                     }
                     Cell viewStatementCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_STATEMENT_BYTES,
-                        cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), viewStatement);
+                            cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), viewStatement);
                     cells.add(viewStatementCell);
 
                     // set the IS_VIEW_REFERENCED column of the auto partition column row
@@ -2341,10 +1881,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     cell = cells.get(0);
                     PDataType dataType = autoPartitionCol.getDataType();
                     Object val = dataType.toObject(autoPartitionNum, PLong.INSTANCE);
-                    byte[] bytes = new byte [dataType.getByteSize() + 1];
+                    byte[] bytes = new byte[dataType.getByteSize() + 1];
                     dataType.toBytes(val, bytes, 0);
                     Cell viewConstantCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_CONSTANT_BYTES,
-                        cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
+                            cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
                     cells.add(viewConstantCell);
                 }
                 Long indexId = null;
@@ -2360,15 +1900,15 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         Cell cell = cells.get(0);
                         PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
                         Object val = dataType.toObject(seqValue, PLong.INSTANCE);
-                        byte[] bytes = new byte [dataType.getByteSize() + 1];
+                        byte[] bytes = new byte[dataType.getByteSize() + 1];
                         dataType.toBytes(val, bytes, 0);
                         Cell indexIdCell = new KeyValue(cell.getRow(), cell.getFamily(), VIEW_INDEX_ID_BYTES,
-                            cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
+                                cell.getTimestamp(), Type.codeToType(cell.getTypeByte()), bytes);
                         cells.add(indexIdCell);
                         indexId = seqValue;
                     }
                 }
-                
+
                 // The mutations to create a table are written in the following order:
                 // 1. Write the child link as if the next two steps fail we
                 // ignore missing children while processing a parent
@@ -2382,8 +1922,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 List<Mutation> childLinkMutations = MetaDataUtil.removeChildLinks(tableMetadata);
                 MetaDataResponse response =
                         processRemoteRegionMutations(
-                            PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                            childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
+                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                                childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
                 if (response != null) {
                     done.run(response);
                     return;
@@ -2392,7 +1932,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (tableType == PTableType.VIEW) {
                     // Pass in the parent's PTable so that we only tag cells corresponding to the
                     // view's property in case they are different from the parent
-                    addTagsToPutsForViewAlteredProperties(tableMetadata, parentTable);
+                    ViewUtil.addTagsToPutsForViewAlteredProperties(tableMetadata, parentTable);
                 }
 
                 // When we drop a view we first drop the view metadata and then drop the parent->child linking row
@@ -2409,20 +1949,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             .getEncodingScheme() != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) {
                         response =
                                 processRemoteRegionMutations(
-                                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
-                                    remoteMutations, MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
-                        clearParentTableFromCache(clientTimeStamp,
-                            parentTable.getSchemaName() != null
-                                    ? parentTable.getSchemaName().getBytes()
-                                    : ByteUtil.EMPTY_BYTE_ARRAY,
-                            parentTable.getName().getBytes());
+                                        PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
+                                        remoteMutations, MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                        clearRemoteTableFromCache(clientTimeStamp,
+                                parentTable.getSchemaName() != null
+                                        ? parentTable.getSchemaName().getBytes()
+                                        : ByteUtil.EMPTY_BYTE_ARRAY,
+                                parentTable.getTableName().getBytes());
                         if (response != null) {
                             done.run(response);
                             return;
                         }
-                    }
-                    else {
-                        String msg = "Found unexpected mutations while creating "+fullTableName;
+                    } else {
+                        String msg = "Found unexpected mutations while creating " + fullTableName;
                         LOGGER.error(msg);
                         for (Mutation m : remoteMutations) {
                             LOGGER.debug("Mutation rowkey : " + Bytes.toStringBinary(m.getRow()));
@@ -2431,34 +1970,34 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         throw new IllegalStateException(msg);
                     }
                 }
-                
+
                 // TODO: Switch this to HRegion#batchMutate when we want to support indexes on the
                 // system table. Basically, we get all the locks that we don't already hold for all the
                 // tableMetadata rows. This ensures we don't have deadlock situations (ensuring
                 // primary and then index table locks are held, in that order). For now, we just don't support
                 // indexing on the system table. This is an issue because of the way we manage batch mutation
                 // in the Indexer.
-                mutateRowsWithLocks(region, localMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, localMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getTable call will add it
                 // TODO: consider loading the table that was just created here, patching up the parent table, and updating the cache
-                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
-                if (parentCacheKey != null) {
-                    metaDataCache.invalidate(parentCacheKey);
+                Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                if (parentTableKey != null) {
+                    metaDataCache.invalidate(new ImmutableBytesPtr(parentTableKey));
                 }
                 metaDataCache.invalidate(cacheKey);
                 // Get timeStamp from mutations - the above method sets it if it's unset
                 long currentTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                 if (indexId != null) {
-                   builder.setViewIndexId(indexId);
-                   builder.setViewIndexIdType(PLong.INSTANCE.getSqlType());
+                    builder.setViewIndexId(indexId);
+                    builder.setViewIndexIdType(PLong.INSTANCE.getSqlType());
                 }
                 builder.setMutationTime(currentTimeStamp);
                 done.run(builder.build());
                 return;
             } finally {
-                releaseRowLocks(region,locks);
+                releaseRowLocks(region, locks);
             }
         } catch (Throwable t) {
             LOGGER.error("createTable failed", t);
@@ -2471,14 +2010,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         int nSequenceSaltBuckets = connection.getQueryServices().getSequenceSaltBuckets();
 
         SequenceKey key = MetaDataUtil.getViewIndexSequenceKey(tenantIdStr, physicalName,
-            nSequenceSaltBuckets, parentTable.isNamespaceMapped() );
+                nSequenceSaltBuckets, parentTable.isNamespaceMapped());
         // Earlier sequence was created at (SCN-1/LATEST_TIMESTAMP) and incremented at the client max(SCN,dataTable.getTimestamp), but it seems we should
         // use always LATEST_TIMESTAMP to avoid seeing wrong sequence values by different connection having SCN
         // or not.
         long sequenceTimestamp = HConstants.LATEST_TIMESTAMP;
         try {
             connection.getQueryServices().createSequence(key.getTenantId(), key.getSchemaName(), key.getSequenceName(),
-                Long.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
+                    Long.MIN_VALUE, 1, 1, Long.MIN_VALUE, Long.MAX_VALUE, false, sequenceTimestamp);
         } catch (SequenceAlreadyExistsException e) {
             //someone else got here first and created the sequence, or it was pre-existing. Not a problem.
         }
@@ -2487,106 +2026,57 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         long[] seqValues = new long[1];
         SQLException[] sqlExceptions = new SQLException[1];
         connection.getQueryServices().incrementSequences(Collections.singletonList(new SequenceAllocation(key, 1)),
-            HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
+                HConstants.LATEST_TIMESTAMP, seqValues, sqlExceptions);
         if (sqlExceptions[0] != null) {
             throw sqlExceptions[0];
         }
         return seqValues[0];
     }
 
-    public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes, byte[] schemaName, byte[] tableName)
-            throws IOException, SQLException, ClassNotFoundException {
-        Table hTable =
-                ServerUtil.getHTableForCoprocessorScan(env,
-                        SchemaUtil.getPhysicalTableName(
-                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                                env.getConfiguration()).getName());
-        TableViewFinderResult childViewsResult = ViewFinder.findRelatedViews(hTable, tenantIdBytes, schemaName, tableName,
-                PTable.LinkType.CHILD_TABLE, HConstants.LATEST_TIMESTAMP);
-
-        if (childViewsResult.hasLinks()) {
-
-            for (TableInfo viewInfo : childViewsResult.getLinks()) {
-                byte[] viewTenantId = viewInfo.getTenantId();
-                byte[] viewSchemaName = viewInfo.getSchemaName();
-                byte[] viewName = viewInfo.getTableName();
-                if (LOGGER.isDebugEnabled()) {
-                    LOGGER.debug("dropChildViews :" + Bytes.toString(schemaName) + "." + Bytes.toString(tableName) +
-                            " -> " + Bytes.toString(viewSchemaName) + "." + Bytes.toString(viewName) +
-                            "with tenant id :" + Bytes.toString(viewTenantId));
-                }
-                Properties props = new Properties();
-                if (viewTenantId != null && viewTenantId.length != 0)
-                    props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(viewTenantId));
-                try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(props, env.getConfiguration())
-                        .unwrap(PhoenixConnection.class)) {
-                    MetaDataClient client = new MetaDataClient(connection);
-                    org.apache.phoenix.parse.TableName viewTableName = org.apache.phoenix.parse.TableName
-                            .create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
-                    try {
-                        client.dropTable(
-                                new DropTableStatement(viewTableName, PTableType.VIEW, true, true, true));
-                    }
-                    catch (TableNotFoundException e) {
-                        LOGGER.info("Ignoring view "+viewTableName+" as it has already been dropped");
-                    }
-                }
-            }
-        }
-    }
     private boolean execeededIndexQuota(PTableType tableType, PTable parentTable) {
         return PTableType.INDEX == tableType && parentTable.getIndexes().size() >= maxIndexesPerTable;
     }
 
-    private void findAncestorViewsOfIndex(byte[] tenantId, byte[] schemaName, byte[] indexName,
-            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
+    private List<PTable> findAllChildViews(long clientTimeStamp, byte[] tenantId, byte[] schemaName, byte[] tableName) throws IOException, SQLException {
+        TableViewFinderResult result = new TableViewFinderResult();
         try (Table hTable =
-                env.getTable(SchemaUtil.getPhysicalTableName(
-                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
-            TableViewFinderResult currentResult =
-                    ViewFinder.findParentViewofIndex(hTable, tenantId, schemaName, indexName);
-            if (currentResult.getLinks().size() == 1) {
-                result.addResult(currentResult);
-                TableInfo tableInfo = currentResult.getLinks().get(0);
-                findAncestorViews(tableInfo.getTenantId(), tableInfo.getSchemaName(),
-                    tableInfo.getTableName(), result, isNamespaceMapped);
+                     env.getTable(SchemaUtil.getPhysicalTableName(
+                             PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                             env.getConfiguration()))) {
+            ViewUtil.findAllRelatives(hTable, tenantId, schemaName, tableName,
+                    LinkType.CHILD_TABLE, result);
+        }
+        List<PTable> childViews = Lists.newArrayListWithExpectedSize(result.getLinks().size());
+        for (TableInfo viewInfo : result.getLinks()) {
+            byte[] viewTenantId = viewInfo.getTenantId();
+            byte[] viewSchemaName = viewInfo.getSchemaName();
+            byte[] viewName = viewInfo.getTableName();
+            PTable view;
+            Properties props = new Properties();
+            if (viewTenantId != null) {
+                props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(viewTenantId));
             }
-            // else this is an index on a regular table and so we don't need to combine columns
-        }
-    }
-    
-    private void findAncestorViews(byte[] tenantId, byte[] schemaName, byte[] tableName,
-            TableViewFinderResult result, boolean isNamespaceMapped) throws IOException {
-        try (Table hTable =
-                env.getTable(SchemaUtil.getPhysicalTableName(
-                    PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, env.getConfiguration()))) {
-            ViewFinder.findAllRelatives(hTable, tenantId, schemaName, tableName,
-                LinkType.PARENT_TABLE, result);
-            if (!isNamespaceMapped || schemaName.length==0) {
-				// When namespace mapping is enabled and the schema name is not empty the
-				// physical table name is of the form S:T while the table name is of the form
-				// S.T
-				// When namespace mapping is disabled or the schema name is empty the
-				// child->parent link is overwritten by the child->physical table link for first
-				// level children of base table as both the parent table name and physical table
-				// name are the same (S.T or T) so we need to query for the PHYSICAL_TABLE link
-                result.addResult(ViewFinder.findBaseTable(hTable, tenantId, schemaName, tableName));
+            if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
+                props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
+            }
+            try (PhoenixConnection connection =
+                         QueryUtil.getConnectionOnServer(props, env.getConfiguration())
+                                 .unwrap(PhoenixConnection.class)) {
+                view = PhoenixRuntime.getTableNoCache(connection, SchemaUtil.getTableName(viewSchemaName, viewName));
+            } catch (ClassNotFoundException e) {
+                throw new IOException(e);
+            }
+            if (view == null) {
+                ServerUtil.throwIOException("View not found", new TableNotFoundException(Bytes.toString(viewSchemaName),
+                        Bytes.toString(viewName)));
             }
+            childViews.add(view);
         }
+        return childViews;
     }
 
-    private void findAllChildViews(byte[] tenantId, byte[] schemaName, byte[] tableName, TableViewFinderResult result) throws IOException {
-        try (Table hTable =
-                env.getTable(SchemaUtil.getPhysicalTableName(
-                    PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                    env.getConfiguration()))) {
-            ViewFinder.findAllRelatives(hTable, tenantId, schemaName, tableName,
-                LinkType.CHILD_TABLE, result);
-        }
-    }
-    
     private void separateLocalAndRemoteMutations(Region region, List<Mutation> mutations,
-            List<Mutation> localMutations, List<Mutation> remoteMutations) {
+                                                 List<Mutation> localMutations, List<Mutation> remoteMutations) {
         HRegionInfo regionInfo = region.getRegionInfo();
         for (Mutation mutation : mutations) {
             if (regionInfo.containsRow(mutation.getRow())) {
@@ -2599,7 +2089,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     @Override
     public void dropTable(RpcController controller, DropTableRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                          RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         boolean isCascade = request.getCascade();
         byte[][] rowKeyMetaData = new byte[3][];
@@ -2614,7 +2104,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-            PTableType pTableType=PTableType.fromSerializedValue(tableType);
+            PTableType pTableType = PTableType.fromSerializedValue(tableType);
             // Disallow deletion of a system table
             if (pTableType == PTableType.SYSTEM) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
@@ -2622,10 +2112,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             }
-            
+
             List<byte[]> tableNamesToDelete = Lists.newArrayList();
             List<SharedTableState> sharedTablesToDelete = Lists.newArrayList();
-            
+
             byte[] lockKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
             Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(lockKey, region);
@@ -2633,7 +2123,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             }
-            
+
             byte[] parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
             byte[] parentLockKey = null;
             // Only lock parent table for indexes
@@ -2645,14 +2135,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return;
                 }
             }
-            
+
             long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-            boolean skipAddingParentColumns = request.hasSkipAddingParentColumns()
-                    ? request.getSkipAddingParentColumns()
-                    : false;
-            PTable loadedTable =
-                    doGetTable(tenantIdBytes, schemaName, tableName, clientTimeStamp, null,
-                        request.getClientVersion(), false, skipAddingParentColumns, null);
+            PTable loadedTable = doGetTable(tenantIdBytes, schemaName, tableName,
+                    clientTimeStamp, null, request.getClientVersion());
             if (loadedTable == null) {
                 builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                 builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
@@ -2673,8 +2159,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 result =
                         doDropTable(lockKey, tenantIdBytes, schemaName, tableName, parentTableName,
-                            PTableType.fromSerializedValue(tableType), tableMetadata, childLinkMutations,
-                            invalidateList, tableNamesToDelete, sharedTablesToDelete, isCascade, request.getClientVersion());
+                                PTableType.fromSerializedValue(tableType), tableMetadata, childLinkMutations,
+                                invalidateList, tableNamesToDelete, sharedTablesToDelete, isCascade, request.getClientVersion());
                 if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
@@ -2688,7 +2174,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 separateLocalAndRemoteMutations(region, tableMetadata, localMutations, remoteMutations);
                 if (!remoteMutations.isEmpty()) {
                     // while dropping a table all the mutations should be local
-                    String msg = "Found unexpected mutations while dropping table "+SchemaUtil.getTableName(schemaName, tableName);
+                    String msg = "Found unexpected mutations while dropping table " + SchemaUtil.getTableName(schemaName, tableName);
                     LOGGER.error(msg);
                     for (Mutation m : remoteMutations) {
                         LOGGER.debug("Mutation rowkey : " + Bytes.toStringBinary(m.getRow()));
@@ -2698,7 +2184,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
 
                 // drop rows from catalog on this region
-                mutateRowsWithLocks(region, localMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, localMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
 
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
@@ -2713,9 +2199,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // after the view metadata is dropped drop parent->child link
                 MetaDataResponse response =
                         processRemoteRegionMutations(
-                            PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                            childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
-                if (response!=null) {
+                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                                childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
+                if (response != null) {
                     done.run(response);
                     return;
                 }
@@ -2728,10 +2214,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } catch (Throwable t) {
             LOGGER.error("dropTable failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
+                    ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
-    
+
     protected void releaseRowLocks(Region region, List<RowLock> locks) {
         if (locks != null) {
             region.releaseRowLocks(locks);
@@ -2750,13 +2236,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private MetaDataResponse processRemoteRegionMutations(byte[] systemTableName,
-            List<Mutation> remoteMutations, MetaDataProtos.MutationCode mutationCode) throws IOException {
+                                                          List<Mutation> remoteMutations, MetaDataProtos.MutationCode mutationCode) throws IOException {
         if (remoteMutations.isEmpty())
             return null;
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         try (Table hTable =
-                env.getTable(
-                    SchemaUtil.getPhysicalTableName(systemTableName, env.getConfiguration()))) {
+                     env.getTable(
+                             SchemaUtil.getPhysicalTableName(systemTableName, env.getConfiguration()))) {
             hTable.batch(remoteMutations);
         } catch (Throwable t) {
             LOGGER.error("Unable to write mutations to " + Bytes.toString(systemTableName), t);
@@ -2766,22 +2252,22 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
         return null;
     }
-    
+
     private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName, byte[] tableName,
-            byte[] parentTableName, PTableType tableType, List<Mutation> catalogMutations,
-            List<Mutation> childLinkMutations, List<ImmutableBytesPtr> invalidateList, List<byte[]> tableNamesToDelete,
-            List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion)
+                                               byte[] parentTableName, PTableType tableType, List<Mutation> catalogMutations,
+                                               List<Mutation> childLinkMutations, List<ImmutableBytesPtr> invalidateList, List<byte[]> tableNamesToDelete,
+                                               List<SharedTableState> sharedTablesToDelete, boolean isCascade, int clientVersion)
             throws IOException, SQLException {
 
         Region region = env.getRegion();
         long clientTimeStamp = MetaDataUtil.getClientTimeStamp(catalogMutations);
         ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
 
-        PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion, true, true, null);
+        PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion);
 
         // We always cache the latest version - fault in if not in cache
         if (table != null
-                || (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion, true, true, null)) != null) {
+                || (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion)) != null) {
             if (table.getTimeStamp() < clientTimeStamp) {
                 if (isTableDeleted(table) || tableType != table.getType()) {
                     return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
@@ -2800,7 +2286,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
         }
         // Make sure we're not deleting the "wrong" child
-        if (parentTableName!=null && table.getParentTableName() != null && !Arrays.equals(parentTableName, table.getParentTableName().getBytes())) {
+        if (parentTableName != null && table.getParentTableName() != null && !Arrays.equals(parentTableName, table.getParentTableName().getBytes())) {
             return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
         }
         // Since we don't allow back in time DDL, we know if we have a table it's the one
@@ -2819,12 +2305,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if (tableType == PTableType.TABLE || tableType == PTableType.VIEW || tableType == PTableType.SYSTEM) {
                 // check to see if the table has any child views
                 try (Table hTable =
-                        env.getTable(SchemaUtil.getPhysicalTableName(
-                            PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                            env.getConfiguration()))) {
+                             env.getTable(SchemaUtil.getPhysicalTableName(
+                                     PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                                     env.getConfiguration()))) {
                     boolean hasChildViews =
-                            ViewFinder.hasChildViews(hTable, tenantId, schemaName, tableName,
-                                clientTimeStamp);
+                            ViewUtil.hasChildViews(hTable, tenantId, schemaName, tableName,
+                                    clientTimeStamp);
                     if (hasChildViews) {
                         if (!isCascade) {
                             // DROP without CASCADE on tables with child views is not permitted
@@ -2834,7 +2320,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         try {
                             PhoenixConnection conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
                             Task.addTask(conn, PTable.TaskType.DROP_CHILD_VIEWS, Bytes.toString(tenantId),
-                                Bytes.toString(schemaName), Bytes.toString(tableName), this.accessCheckEnabled);
+                                    Bytes.toString(schemaName), Bytes.toString(tableName), this.accessCheckEnabled);
                         } catch (Throwable t) {
                             LOGGER.error("Adding a task to drop child views failed!", t);
                         }
@@ -2843,10 +2329,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
 
             // Add to list of HTables to delete, unless it's a view or its a shared index 
-            if (tableType == INDEX && table.getViewIndexId()!=null) {
+            if (tableType == INDEX && table.getViewIndexId() != null) {
                 sharedTablesToDelete.add(new SharedTableState(table));
-            }
-            else if (tableType != PTableType.VIEW) { 
+            } else if (tableType != PTableType.VIEW) {
                 tableNamesToDelete.add(table.getPhysicalName().getBytes());
             }
             invalidateList.add(cacheKey);
@@ -2857,18 +2342,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (nColumns == 5
                         && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0
                         && Bytes.compareTo(kv.getQualifierArray(), kv.getQualifierOffset(), kv.getQualifierLength(),
-                                LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0) {
-                        LinkType linkType = LinkType.fromSerializedValue(kv.getValueArray()[kv.getValueOffset()]);
-                        if (rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length == 0 && linkType == LinkType.INDEX_TABLE) {
-                            indexNames.add(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
-                        } else if (tableType == PTableType.VIEW && (linkType == LinkType.PARENT_TABLE || linkType == LinkType.PHYSICAL_TABLE)) {
-                            // delete parent->child link for views
-                            Cell parentTenantIdCell = MetaDataUtil.getCell(results, PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES);
-                            PName parentTenantId = parentTenantIdCell!=null ? PNameFactory.newName(parentTenantIdCell.getValueArray(), parentTenantIdCell.getValueOffset(), parentTenantIdCell.getValueLength()) : null;
-                            byte[] linkKey = MetaDataUtil.getChildLinkKey(parentTenantId, table.getParentSchemaName(), table.getParentTableName(), table.getTenantId(), table.getName());
-                            Delete linkDelete = new Delete(linkKey, clientTimeStamp);
-                            childLinkMutations.add(linkDelete);
-                        }
+                        LINK_TYPE_BYTES, 0, LINK_TYPE_BYTES.length) == 0) {
+                    LinkType linkType = LinkType.fromSerializedValue(kv.getValueArray()[kv.getValueOffset()]);
+                    if (rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length == 0 && linkType == LinkType.INDEX_TABLE) {
+                        indexNames.add(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
+                    } else if (tableType == PTableType.VIEW && (linkType == LinkType.PARENT_TABLE || linkType == LinkType.PHYSICAL_TABLE)) {
+                        // delete parent->child link for views
+                        Cell parentTenantIdCell = MetaDataUtil.getCell(results, PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES);
+                        PName parentTenantId = parentTenantIdCell != null ? PNameFactory.newName(parentTenantIdCell.getValueArray(), parentTenantIdCell.getValueOffset(), parentTenantIdCell.getValueLength()) : null;
+                        byte[] linkKey = MetaDataUtil.getChildLinkKey(parentTenantId, table.getParentSchemaName(), table.getParentTableName(), table.getTenantId(), table.getName());
+                        Delete linkDelete = new Delete(linkKey, clientTimeStamp);
+                        childLinkMutations.add(linkDelete);
+                    }
                 }
                 Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
                 catalogMutations.add(delete);
@@ -2888,7 +2373,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             catalogMutations.add(delete);
             MetaDataMutationResult result =
                     doDropTable(indexKey, tenantId, schemaName, indexName, tableName, PTableType.INDEX,
-                        catalogMutations, childLinkMutations, invalidateList, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
+                            catalogMutations, childLinkMutations, invalidateList, tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
             if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                 return result;
             }
@@ -2898,14 +2383,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 EnvironmentEdgeManager.currentTimeMillis(), table, tableNamesToDelete, sharedTablesToDelete);
     }
 
-    private static interface ColumnMutator {
-        MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
-                List<Mutation> tableMetadata, Region region, List<ImmutableBytesPtr> invalidateList,
-                List<RowLock> locks, long clientTimeStamp) throws IOException, SQLException;
-    }
-
     private MetaDataMutationResult
-    mutateColumn(MutatateColumnType mutateColumnType, List<Mutation> tableMetadata, ColumnMutator mutator, int clientVersion) throws IOException {
+    mutateColumn(List<Mutation> tableMetadata, ColumnMutator mutator, int clientVersion, PTable parentTable)
+            throws IOException {
         byte[][] rowKeyMetaData = new byte[5][];
         MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
         byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
@@ -2913,20 +2393,55 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         byte[] tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
         byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
         String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
+        // server-side, except for indexing, we always expect the keyvalues to be standard KeyValues
+        PTableType expectedType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
+                new ImmutableBytesWritable());
+        List<byte[]> tableNamesToDelete = Lists.newArrayList();
+        List<SharedTableState> sharedTablesToDelete = Lists.newArrayList();
+        List<PTable> childViews = Lists.newArrayList();
+        long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
         try {
             Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
                 return result;
             }
+
             List<RowLock> locks = Lists.newArrayList();
             try {
+                if (expectedType == PTableType.TABLE) {
+                    childViews = findAllChildViews(clientTimeStamp, tenantId, schemaName, tableName);
+
+                    if (!childViews.isEmpty()) {
+                        // From 4.15 onwards we allow SYSTEM.CATALOG to split and no longer propagate parent
+                        // metadata changes to child views.
+                        // If the client is on a version older than 4.15 we have to block adding a column to a
+                        // parent able as we no longer lock the parent table on the server side while creating a
+                        // child view to prevent conflicting changes. This is handled on the client side from
+                        // 4.15 onwards.
+                        // Also if QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK is true, we block adding
+                        // a column to a parent table so that we can rollback the upgrade if required.
+                        if (clientVersion < MIN_SPLITTABLE_SYSTEM_CATALOG) {
+                            LOGGER.error(
+                                    "Unable to add or drop a column as the client is older than "
+                                            + MIN_SPLITTABLE_SYSTEM_CATALOG);
+                            return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                    EnvironmentEdgeManager.currentTimeMillis(), null);
+                        } else if (allowSplittableSystemCatalogRollback) {
+                            LOGGER.error("Unable to add or drop a column as the "
+                                    + QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK
+                                    + " config is set to true");
+                            return new MetaDataProtocol.MetaDataMutationResult(MetaDataProtocol.MutationCode.UNALLOWED_TABLE_MUTATION,
+                                    EnvironmentEdgeManager.currentTimeMillis(), null);
+                        }
+                    }
+                }
+
                 acquireLock(region, key, locks);
                 ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
                 List<ImmutableBytesPtr> invalidateList = new ArrayList<ImmutableBytesPtr>();
                 invalidateList.add(cacheKey);
-                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
-                PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion, false, false, null);
+                PTable table = getTableFromCache(cacheKey, clientTimeStamp, clientVersion);
                 if (LOGGER.isDebugEnabled()) {
                     if (table == null) {
                         LOGGER.debug("Table " + Bytes.toStringBinary(key)
@@ -2939,7 +2454,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 // Get client timeStamp from mutations
                 if (table == null
-                        && (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion, false, false, null)) == null) {
+                        && (table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion)) == null) {
                     // if not found then call newerTableExists and add delete marker for timestamp
                     // found
                     table = buildDeletedTable(key, cacheKey, region, clientTimeStamp);
@@ -2951,15 +2466,36 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
                             EnvironmentEdgeManager.currentTimeMillis(), null);
                 }
+
+                // if this is a view or view index then we need to include columns and
+                // indexes derived from its ancestors
+                if (parentTable != null) {
+                    Properties props = new Properties();
+                    if (tenantId != null) {
+                        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(tenantId));
+                    }
+                    if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
+                        props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
+                    }
+                    try (PhoenixConnection connection =
+                                 QueryUtil.getConnectionOnServer(props, env.getConfiguration())
+                                         .unwrap(PhoenixConnection.class)) {
+                        table = ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, table, parentTable);
+                    } catch (ClassNotFoundException e) {
+                    }
+                }
+
                 if (table.getTimeStamp() >= clientTimeStamp) {
                     LOGGER.info("Found newer table as of " + table.getTimeStamp() + " versus client timestamp of "
                             + clientTimeStamp);
                     return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND,
                             EnvironmentEdgeManager.currentTimeMillis(), table);
-                } else if (isTableDeleted(table)) { return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
-                        EnvironmentEdgeManager.currentTimeMillis(), null); }
+                } else if (isTableDeleted(table)) {
+                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
+                            EnvironmentEdgeManager.currentTimeMillis(), null);
+                }
                 long expectedSeqNum = MetaDataUtil.getSequenceNumber(tableMetadata) - 1; // lookup TABLE_SEQ_NUM in
-                                                                                         // tableMetaData
+                // tableMetaData
 
                 if (LOGGER.isDebugEnabled()) {
                     LOGGER.debug("For table " + Bytes.toStringBinary(key) + " expecting seqNum "
@@ -2982,15 +2518,28 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
                             EnvironmentEdgeManager.currentTimeMillis(), null);
                 } else {
-                    // server-side, except for indexing, we always expect the keyvalues to be standard KeyValues
-                    PTableType expectedType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
-                            new ImmutableBytesWritable());
                     // We said to drop a table, but found a view or visa versa
-                    if (type != expectedType) { return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
-                            EnvironmentEdgeManager.currentTimeMillis(), null); }
+                    if (type != expectedType) {
+                        return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
+                                EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
                 }
-                result = mutator.updateMutation(table, rowKeyMetaData, tableMetadata, region,
-                            invalidateList, locks, clientTimeStamp);
+
+                if (!childViews.isEmpty()) {
+                    // validate the add or drop column mutations
+                    result = mutator.validateWithChildViews(table, childViews, tableMetadata, schemaName, tableName);
+                    if (result != null) {
+                        return result;
+                    }
+                }
+
+                getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
+                        SchemaUtil.getTableName(schemaName, tableName),
+                        TableName.valueOf(table.getPhysicalName().getBytes()),
+                        getParentPhysicalTableName(table), table.getType());
+
+                result = mutator.validateAndAddMetadata(table, rowKeyMetaData, tableMetadata, region,
+                        invalidateList, locks, clientTimeStamp);
                 // if the update mutation caused tables to be deleted, the mutation code returned
                 // will be MutationCode.TABLE_ALREADY_EXISTS
                 if (result != null
@@ -2998,6 +2547,25 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return result;
                 }
 
+                // drop any indexes on the base table that need the column that is going to be dropped
+                List<Pair<PTable, PColumn>> tableAndDroppedColumnPairs = mutator.getTableAndDroppedColumnPairs();
+                Iterator<Pair<PTable, PColumn>> iterator = tableAndDroppedColumnPairs.iterator();
+                while (iterator.hasNext()) {
+                    Pair<PTable, PColumn> pair = iterator.next();
+                    // remove the current table and column being dropped from the list
+                    // and drop any indexes that require the column being dropped while holding the row lock
+                    if (table.equals(pair.getFirst())) {
+                        iterator.remove();
+                        result = dropIndexes(env, pair.getFirst(), invalidateList, locks, clientTimeStamp,
+                                tableMetadata, pair.getSecond(), tableNamesToDelete, sharedTablesToDelete,
+                                clientVersion);
+                        if (result != null
+                                && result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                            return result;
+                        }
+                    }
+                }
+
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                         GlobalCache.getInstance(this.env).getMetaDataCache();
 
@@ -3011,23 +2579,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         Lists.newArrayListWithExpectedSize(tableMetadata.size());
                 List<Mutation> remoteMutations = Lists.newArrayList();
                 separateLocalAndRemoteMutations(region, tableMetadata, localMutations,
-                    remoteMutations);
+                        remoteMutations);
                 if (!remoteMutations.isEmpty()) {
                     // there should only be remote mutations if we are adding a column to a view that uses encoded column qualifiers
                     // (the remote mutations are to update the encoded column qualifier counter on the parent table)
-                    if (mutateColumnType == MutatateColumnType.ADD_COLUMN && type == PTableType.VIEW && table
-                            .getEncodingScheme() != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) {
+                    if (mutator.getMutateColumnType() == ColumnMutator.MutateColumnType.ADD_COLUMN
+                            && type == PTableType.VIEW
+                            && table.getEncodingScheme() != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) {
                         processRemoteRegionMutations(
-                            PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, remoteMutations,
-                            MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
-                        clearParentTableFromCache(clientTimeStamp,
-                            table.getParentSchemaName() != null
-                                    ? table.getParentSchemaName().getBytes()
-                                    : ByteUtil.EMPTY_BYTE_ARRAY,
-                            table.getParentTableName().getBytes());
-                    }
-                    else {
-                        String msg = "Found unexpected mutations while adding or dropping column to "+fullTableName;
+                                PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, remoteMutations,
+                                MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
+                        clearRemoteTableFromCache(clientTimeStamp,
+                                table.getParentSchemaName() != null
+                                        ? table.getParentSchemaName().getBytes()
+                                        : ByteUtil.EMPTY_BYTE_ARRAY,
+                                table.getParentTableName().getBytes());
+                    } else {
+                        String msg = "Found unexpected mutations while adding or dropping column to " + fullTableName;
                         LOGGER.error(msg);
                         for (Mutation m : remoteMutations) {
                             LOGGER.debug("Mutation rowkey : " + Bytes.toStringBinary(m.getRow()));
@@ -3036,7 +2604,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         throw new IllegalStateException(msg);
                     }
                 }
-                mutateRowsWithLocks(region, localMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, localMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 // Invalidate from cache
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
@@ -3045,14 +2613,26 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // mutateRowsWithLocks call
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 // if the update mutation caused tables to be deleted just return the result which will contain the table to be deleted
-                if (result !=null) {
+                if (result != null
+                        && result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
                     return result;
                 } else {
-                    table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion, false, false, null);
-                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table);
+                    table = buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion);
+                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table,
+                            tableNamesToDelete, sharedTablesToDelete);
                 }
             } finally {
-                releaseRowLocks(region,locks);
+                releaseRowLocks(region, locks);
+                // drop indexes on views that require the column being dropped
+                // these could be on a different region server so don't hold row locks while dropping them
+                for (Pair<PTable, PColumn> pair : mutator.getTableAndDroppedColumnPairs()) {
+                    result = dropRemoteIndexes(env, pair.getFirst(), clientTimeStamp, pair.getSecond(),
+                            tableNamesToDelete, sharedTablesToDelete);
+                    if (result != null
+                            && result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                        return result;
+                    }
+                }
             }
         } catch (Throwable t) {
             ServerUtil.throwIOException(fullTableName, t);
@@ -3063,42 +2643,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     /**
      * Removes the table from the server side cache
      */
-    private void clearParentTableFromCache(long clientTimeStamp, byte[] schemaName, byte[] tableName) throws SQLException {
+    private void clearRemoteTableFromCache(long clientTimeStamp, byte[] schemaName, byte[] tableName) throws SQLException {
         // remove the parent table from the metadata cache as we just mutated the table
         Properties props = new Properties();
         if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
             props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
         }
         try (PhoenixConnection connection =
-                QueryUtil.getConnectionOnServer(props, env.getConfiguration())
-                        .unwrap(PhoenixConnection.class)) {
+                     QueryUtil.getConnectionOnServer(props, env.getConfiguration())
+                             .unwrap(PhoenixConnection.class)) {
             ConnectionQueryServices queryServices = connection.getQueryServices();
             queryServices.clearTableFromCache(ByteUtil.EMPTY_BYTE_ARRAY, schemaName, tableName,
-                clientTimeStamp);
+                    clientTimeStamp);
         } catch (ClassNotFoundException e) {
         }
     }
-    
-    private static boolean isDivergedView(PTable view) {
-        return view.getBaseColumnCount() == QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
-    }
 
-    private boolean switchAttribute(PTable table, boolean currAttribute, List<Mutation> tableMetaData, byte[] attrQualifier) {
-        for (Mutation m : tableMetaData) {
-            if (m instanceof Put) {
-                Put p = (Put)m;
-                List<Cell> cells = p.get(TABLE_FAMILY_BYTES, attrQualifier);
-                if (cells != null && cells.size() > 0) {
-                    Cell cell = cells.get(0);
-                    boolean newAttribute = (boolean)PBoolean.INSTANCE.toObject(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
-                    return currAttribute != newAttribute;
-                }
-            }
-        }
-        return false;
-    }
-    
-    private class ColumnFinder extends StatelessTraverseAllExpressionVisitor<Void> {
+    public static class ColumnFinder extends StatelessTraverseAllExpressionVisitor<Void> {
         private boolean columnFound;
         private final Expression columnExpression;
 
@@ -3134,566 +2695,44 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
-    private MetaDataMutationResult validateColumnForAddToBaseTable(PTable basePhysicalTable,
-            List<Mutation> tableMetadata, byte[][] rowKeyMetaData,
-            TableViewFinderResult childViewsResult, long clientTimeStamp, int clientVersion)
-            throws IOException, SQLException {
-        byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
-        byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
-        List<Put> columnPutsForBaseTable =
-                Lists.newArrayListWithExpectedSize(tableMetadata.size());
-        boolean salted = basePhysicalTable.getBucketNum()!=null;
-        // Isolate the puts relevant to adding columns 
-        for (Mutation m : tableMetadata) {
-            if (m instanceof Put) {
-                byte[][] rkmd = new byte[5][];
-                int pkCount = getVarChars(m.getRow(), rkmd);
-                // check if this put is for adding a column
-                if (pkCount > COLUMN_NAME_INDEX && rkmd[COLUMN_NAME_INDEX] != null
-                        && rkmd[COLUMN_NAME_INDEX].length > 0
-                        && Bytes.compareTo(schemaName, rkmd[SCHEMA_NAME_INDEX]) == 0
-                        && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) == 0) {
-                    columnPutsForBaseTable.add((Put)m);
-                }
-            }
-        }
-        for (TableInfo viewInfo : childViewsResult.getLinks()) {
-            byte[] tenantId = viewInfo.getTenantId();
-            byte[] schema = viewInfo.getSchemaName();
-            byte[] table = viewInfo.getTableName();
-            PTable view =
-                    doGetTable(tenantId, schema, table, clientTimeStamp, null, clientVersion, true,
-                        true, null); 
-            // we don't need to include parent columns as we are only interested in the columns
-            // added by the view itself (also including parent columns will lead to a circular call
-            // which will deadlock)
-            // we also don't need to include indexes
-            if (view == null) {
-                LOGGER.warn("Found invalid tenant view row in SYSTEM.CATALOG with tenantId:"
-                        + Bytes.toString(tenantId) + ", schema:" + Bytes.toString(schema)
-                        + ", table:" + Bytes.toString(table));
-                continue;
-            }
-            /*
-             * Disallow adding columns to a base table with APPEND_ONLY_SCHEMA since this
-             * creates a gap in the column positions for every view (PHOENIX-4737).
-             */
-            if (!columnPutsForBaseTable.isEmpty() && view.isAppendOnlySchema()) {
-                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-            }
-            
-            // add the new columns to the child view
-            List<PColumn> viewPkCols = new ArrayList<>(view.getPKColumns());
-            // remove salted column
-            if (salted) {
-                viewPkCols.remove(0);
-            }
-            // remove pk columns that are present in the parent
-            viewPkCols.removeAll(basePhysicalTable.getPKColumns());
-            boolean addedPkColumn = false;
-            for (Put columnToBeAdded : columnPutsForBaseTable) {
-                PColumn existingViewColumn = null;
-                byte[][] rkmd = new byte[5][];
-                getVarChars(columnToBeAdded.getRow(), rkmd);
-                String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
-                String columnFamily =
-                        rkmd[FAMILY_NAME_INDEX] == null ? null
-                                : Bytes.toString(rkmd[FAMILY_NAME_INDEX]);
-                try {
-                    existingViewColumn =
-                            columnFamily == null ? view.getColumnForColumnName(columnName)
-                                    : view.getColumnFamily(columnFamily)
-                                            .getPColumnForColumnName(columnName);
-                } catch (ColumnFamilyNotFoundException e) {
-                    // ignore since it means that the column family is not present for the column to
-                    // be added.
-                } catch (ColumnNotFoundException e) {
-                    // ignore since it means the column is not present in the view
-                }
-
-                boolean isCurrColumnToBeAddPkCol = columnFamily == null;
-                addedPkColumn |= isCurrColumnToBeAddPkCol;
-                if (existingViewColumn != null) {
-                    if (EncodedColumnsUtil.usesEncodedColumnNames(basePhysicalTable)
-                            && !SchemaUtil.isPKColumn(existingViewColumn)) {
-                        /*
-                         * If the column already exists in a view, then we cannot add the column to
-                         * the base table. The reason is subtle and is as follows: consider the case
-                         * where a table has two views where both the views have the same key value
-                         * column KV. Now, we dole out encoded column qualifiers for key value
-                         * columns in views by using the counters stored in the base physical table.
-                         * So the KV column can have different column qualifiers for the two views.
-                         * For example, 11 for VIEW1 and 12 for VIEW2. This naturally extends to
-                         * rows being inserted using the two views having different column
-                         * qualifiers for the column named KV. Now, when an attempt is made to add
-                         * column KV to the base table, we cannot decide which column qualifier
-                         * should that column be assigned. It cannot be a number different than 11
-                         * or 12 since a query like SELECT KV FROM BASETABLE would return null for
-                         * KV which is incorrect since column KV is present in rows inserted from
-                         * the two views. We cannot use 11 or 12 either because we will then
-                         * incorrectly return value of KV column inserted using only one view.
-                         */
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-                    }
-                    // Validate data type is same
-                    int baseColumnDataType =
-                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, DATA_TYPE_BYTES);
-                    if (baseColumnDataType != existingViewColumn.getDataType().getSqlType()) {
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-                    }
-
-                    // Validate max length is same
-                    int maxLength =
-                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, COLUMN_SIZE_BYTES);
-                    int existingMaxLength =
-                            existingViewColumn.getMaxLength() == null ? 0
-                                    : existingViewColumn.getMaxLength();
-                    if (maxLength != existingMaxLength) {
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-                    }
-
-                    // Validate scale is same
-                    int scale =
-                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, DECIMAL_DIGITS_BYTES);
-                    int existingScale =
-                            existingViewColumn.getScale() == null ? 0
-                                    : existingViewColumn.getScale();
-                    if (scale != existingScale) {
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-                    }
-
-                    // Validate sort order is same
-                    int sortOrder =
-                            getInteger(columnToBeAdded, TABLE_FAMILY_BYTES, SORT_ORDER_BYTES);
-                    if (sortOrder != existingViewColumn.getSortOrder().getSystemValue()) {
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-                    }
-
-                    // if the column to be added to the base table is a pk column, then we need to
-                    // validate that the key slot position is the same
-                    if (isCurrColumnToBeAddPkCol) {
-                        List<Cell> keySeqCells =
-                                columnToBeAdded.get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-                                    PhoenixDatabaseMetaData.KEY_SEQ_BYTES);
-                        if (keySeqCells != null && keySeqCells.size() > 0) {
-                            Cell cell = keySeqCells.get(0);
-                            int keySeq =
-                                    PSmallint.INSTANCE.getCodec().decodeInt(cell.getValueArray(),
-                                        cell.getValueOffset(), SortOrder.getDefault());
-                            // we need to take into account the columns inherited from the base table
-                            // if the table is salted we don't include the salted column (which is
-                            // present in getPKColumns())
-                            int pkPosition =
-                                    basePhysicalTable.getPKColumns().size()
-                                            + SchemaUtil.getPKPosition(view, existingViewColumn) + 1
-                                            - (salted ? 2 : 0); 
-                            if (pkPosition != keySeq) {
-                                return new MetaDataMutationResult(
-                                        MutationCode.UNALLOWED_TABLE_MUTATION,
-                                        EnvironmentEdgeManager.currentTimeMillis(),
-                                        basePhysicalTable);
-                            }
-                        }
-                    }
-                }
-                if (existingViewColumn!=null && isCurrColumnToBeAddPkCol) {
-                    viewPkCols.remove(existingViewColumn);
-                }
-            }
-            /*
-             * Allow adding a pk columns to base table : 1. if all the view pk columns are exactly
-             * the same as the base table pk columns 2. if we are adding all the existing view pk
-             * columns to the base table
-             */
-            if (addedPkColumn && !viewPkCols.isEmpty()) {
-                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                        EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-            }
-        }
-        return null;
-    }
-    
-    private MetaDataMutationResult dropViewIndexes(Region region, PTable basePhysicalTable,
-            List<RowLock> locks, List<Mutation> tableMetadata,
-            List<Mutation> mutationsForAddingColumnsToViews, byte[] schemaName, byte[] tableName,
-            List<ImmutableBytesPtr> invalidateList, long clientTimeStamp,
-            TableViewFinderResult childViewsResult, List<byte[]> tableNamesToDelete,
-            List<SharedTableState> sharedTablesToDelete, int clientVersion)
-            throws IOException, SQLException {
-        List<Delete> columnDeletesForBaseTable = new ArrayList<>(tableMetadata.size());
-        // Isolate the deletes relevant to dropping columns. Also figure out what kind of columns
-        // are being added.
-        for (Mutation m : tableMetadata) {
-            if (m instanceof Delete) {
-                byte[][] rkmd = new byte[5][];
-                int pkCount = getVarChars(m.getRow(), rkmd);
-                if (pkCount > COLUMN_NAME_INDEX
-                        && Bytes.compareTo(schemaName, rkmd[SCHEMA_NAME_INDEX]) == 0
-                        && Bytes.compareTo(tableName, rkmd[TABLE_NAME_INDEX]) == 0) {
-                    columnDeletesForBaseTable.add((Delete) m);
-                }
-            }
-        }
-        for (TableInfo viewInfo : childViewsResult.getLinks()) {
-            byte[] viewTenantId = viewInfo.getTenantId();
-            byte[] viewSchemaName = viewInfo.getSchemaName();
-            byte[] viewName = viewInfo.getTableName();
-            PTable view =
-                    doGetTable(viewTenantId, viewSchemaName,
-                        viewName, clientTimeStamp, null, clientVersion, false,
-                        false, basePhysicalTable);
-            for (Delete columnDeleteForBaseTable : columnDeletesForBaseTable) {
-                PColumn existingViewColumn = null;
-                byte[][] rkmd = new byte[5][];
-                getVarChars(columnDeleteForBaseTable.getRow(), rkmd);
-                String columnName = Bytes.toString(rkmd[COLUMN_NAME_INDEX]);
-                String columnFamily =
-                        rkmd[FAMILY_NAME_INDEX] == null ? null : Bytes
-                                .toString(rkmd[FAMILY_NAME_INDEX]);
-                try {   
-                    existingViewColumn =
-                            columnFamily == null ? view.getColumnForColumnName(columnName) : view
-                                    .getColumnFamily(columnFamily).getPColumnForColumnName(columnName);
-                } catch (ColumnFamilyNotFoundException e) {
-                    // ignore since it means that the column family is not present for the column to
-                    // be added.
-                } catch (ColumnNotFoundException e) {
-                    // ignore since it means the column is not present in the view
-                }
-
-                // check if the view where expression contains the column being dropped and prevent
-                // it
-                if (existingViewColumn != null && view.getViewStatement() != null) {
-                    ParseNode viewWhere =
-                            new SQLParser(view.getViewStatement()).parseQuery().getWhere();
-                    PhoenixConnection conn=null;
-                    try {
-                        conn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(
-                            PhoenixConnection.class);
-                    } catch (ClassNotFoundException e) {
-                    }
-                    PhoenixStatement statement = new PhoenixStatement(conn);
-                    TableRef baseTableRef = new TableRef(basePhysicalTable);
-                    ColumnResolver columnResolver = FromCompiler.getResolver(baseTableRef);
-                    StatementContext context = new StatementContext(statement, columnResolver);
-                    Expression whereExpression = WhereCompiler.compile(context, viewWhere);
-                    Expression colExpression =
-                            new ColumnRef(baseTableRef, existingViewColumn.getPosition())
-                                    .newColumnExpression();
-                    ColumnFinder columnFinder = new ColumnFinder(colExpression);
-                    whereExpression.accept(columnFinder);
-                    if (columnFinder.getColumnFound()) {
-                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                EnvironmentEdgeManager.currentTimeMillis(), basePhysicalTable);
-                    }
-                }
-
-                if (existingViewColumn != null) {
-                    // drop any view indexes that need this column
-                    MetaDataMutationResult result =
-                            dropIndexes(view, region, invalidateList, locks, clientTimeStamp,
-                                view.getSchemaName().getBytes(), view.getTableName().getBytes(),
-                                mutationsForAddingColumnsToViews, existingViewColumn,
-                                tableNamesToDelete, sharedTablesToDelete, clientVersion, basePhysicalTable);
-                    if (result != null) {
-                        return result;
-                    }
-                }
-            }
-
-        }
-        return null;
-    }
-
-    private int getInteger(Put p, byte[] family, byte[] qualifier) {
-        List<Cell> cells = p.get(family, qualifier);
-        if (cells != null && cells.size() > 0) {
-            Cell cell = cells.get(0);
-            return (Integer)PInteger.INSTANCE.toObject(cell.getValueArray(), cell.getValueOffset(), cell.getValueLength());
-        }
-        return 0;
-    }
-
-    private enum MutatateColumnType {
-        ADD_COLUMN, DROP_COLUMN
-    }
-
     @Override
     public void addColumn(RpcController controller, final AddColumnRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                          RpcCallback<MetaDataResponse> done) {
         try {
             List<Mutation> tableMetaData = ProtobufUtil.getMutations(request);
-
-            MetaDataMutationResult result = mutateColumn(MutatateColumnType.ADD_COLUMN, tableMetaData, new ColumnMutator() {
-                @Override
-                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
-                        List<Mutation> tableMetaData, Region region, List<ImmutableBytesPtr> invalidateList,
-                        List<RowLock> locks, long clientTimeStamp) throws IOException, SQLException {
-                    byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
-                    byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
-                    byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
-                    PTableType type = table.getType();
-                    byte[] tableHeaderRowKey = SchemaUtil.getTableKey(tenantId,
-                            schemaName, tableName);
-                    byte[] cPhysicalTableName=table.getPhysicalName().getBytes();
-                    getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
-                            SchemaUtil.getTableName(schemaName, tableName), TableName.valueOf(cPhysicalTableName),
-                            getParentPhysicalTableName(table),type);
-
-                    List<Mutation> additionalTableMetadataMutations = Lists.newArrayListWithExpectedSize(2);
-                    if (type == PTableType.TABLE) {
-                        TableViewFinderResult childViewsResult = new TableViewFinderResult();
-                        findAllChildViews(tenantId, table.getSchemaName().getBytes(), table.getTableName().getBytes(), childViewsResult);
-                        if (childViewsResult.hasLinks()) {
-                            // Dis-allow if:
-                            //
-                            // 1) The base column count is 0 which means that the metadata hasn't been upgraded yet or
-                            // the upgrade is currently in progress.
-                            //
-                            // 2) If the request is from a client that is older than 4.5 version of phoenix.
-                            // Starting from 4.5, metadata requests have the client version included in them.
-                            // We don't want to allow clients before 4.5 to add a column to the base table if it
-                            // has views.
-                            //
-                            // 3) Trying to switch tenancy of a table that has views
-                            //
-                            // 4) From 4.15 onwards we allow SYSTEM.CATALOG to split and no longer propagate parent
-                            // metadata changes to child views.
-                            // If the client is on a version older than 4.15 we have to block adding a column to a
-                            // parent able as we no longer lock the parent table on the server side while creating a
-                            // child view to prevent conflicting changes. This is handled on the client side from
-                            // 4.15 onwards.
-                            // Also if QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK is true, we block adding
-                            // a column to a parent table so that we can rollback the upgrade if required.
-                            if (table.getBaseColumnCount() == 0
-                                    || !request.hasClientVersion()
-                                    || switchAttribute(table, table.isMultiTenant(), tableMetaData, MULTI_TENANT_BYTES)) {
-                                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                        EnvironmentEdgeManager.currentTimeMillis(), null);
-                            }
-                            else if (request.getClientVersion()< MIN_SPLITTABLE_SYSTEM_CATALOG ) {
-                                LOGGER.error(
-                                    "Unable to add a column as the client is older than "
-                                            + MIN_SPLITTABLE_SYSTEM_CATALOG);
-                                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                        EnvironmentEdgeManager.currentTimeMillis(), null);
-                            }
-                            else if (allowSystemCatalogRollback) {
-                                LOGGER.error("Unable to add a column as the "
-                                        + QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK
-                                        + " config is set to true");
-                                return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION,
-                                        EnvironmentEdgeManager.currentTimeMillis(), null);
-                            }
-                            else {
-                                MetaDataMutationResult mutationResult =
-                                        validateColumnForAddToBaseTable(table,
-                                            tableMetaData, rowKeyMetaData, childViewsResult,
-                                            clientTimeStamp, request.getClientVersion());
-                                // return if validation was not successful
-                                if (mutationResult!=null)
-                                    return mutationResult;
-                            } 
-                        }
-                    } 
-                    boolean addingCol = false;
-                    for (Mutation m : tableMetaData) {
-                        byte[] key = m.getRow();
-                        boolean addingPKColumn = false;
-                        int pkCount = getVarChars(key, rowKeyMetaData);
-                        // this means we have are adding a column 
-                        if (pkCount > COLUMN_NAME_INDEX
-                                && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
-                                && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
-                            try {
-                                addingCol = true;
-                                byte[] familyName = null;
-                                byte[] colName = null;
-                                if (pkCount > FAMILY_NAME_INDEX) {
-                                    familyName = rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX];
-                                }
-                                if (pkCount > COLUMN_NAME_INDEX) {
-                                    colName = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
-                                }
-                                if (table.getExcludedColumns().contains(
-                                    PColumnImpl.createExcludedColumn(newPName(familyName), newPName(colName), 0l))) {
-                                    // if this column was previously dropped in a view do not allow adding the column back
-                                    return new MetaDataMutationResult(
-                                            MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), null);
-                                }
-                                if (familyName!=null && familyName.length > 0) {
-                                    PColumnFamily family =
-                                            table.getColumnFamily(familyName);
-                                            family.getPColumnForColumnNameBytes(colName);
-                                } else if (colName!=null && colName.length > 0) {
-                                    addingPKColumn = true;
-                                    table.getPKColumn(new String(colName));
-                                } else {
-                                    continue;
-                                }
-                                return new MetaDataMutationResult(
-                                        MutationCode.COLUMN_ALREADY_EXISTS, EnvironmentEdgeManager
-                                        .currentTimeMillis(), table);
-                            } catch (ColumnFamilyNotFoundException e) {
-                                continue;
-                            } catch (ColumnNotFoundException e) {
-                                if (addingPKColumn) {
-                                    // We may be adding a DESC column, so if table is already
-                                    // able to be rowKeyOptimized, it should continue to be so.
-                                    if (table.rowKeyOrderOptimizable()) {
-                                        UpgradeUtil.addRowKeyOrderOptimizableCell(additionalTableMetadataMutations, tableHeaderRowKey, clientTimeStamp);
-                                    } else if (table.getType() == PTableType.VIEW){
-                                        // Don't allow view PK to diverge from table PK as our upgrade code
-                                        // does not handle this.
-                                        return new MetaDataMutationResult(
-                                                MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager
-                                                .currentTimeMillis(), null);
-                                    }
-                                    // Add all indexes to invalidate list, as they will all be
-                                    // adding the same PK column. No need to lock them, as we
-                                    // have the parent table lock at this point.
-                                    for (PTable index : table.getIndexes()) {
-                                        invalidateList.add(new ImmutableBytesPtr(SchemaUtil
-                                                .getTableKey(tenantId, index.getSchemaName()
-                                                        .getBytes(), index.getTableName()
-                                                        .getBytes())));
-                                        // We may be adding a DESC column, so if index is already
-                                        // able to be rowKeyOptimized, it should continue to be so.
-                                        if (index.rowKeyOrderOptimizable()) {
-                                            byte[] indexHeaderRowKey = SchemaUtil.getTableKey(index.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : index.getTenantId().getBytes(),
-                                                    index.getSchemaName().getBytes(), index.getTableName().getBytes());
-                                            UpgradeUtil.addRowKeyOrderOptimizableCell(additionalTableMetadataMutations, indexHeaderRowKey, clientTimeStamp);
-                                        }
-                                    }
-                                }
-                                continue;
-                            }
-                        } else if (pkCount == COLUMN_NAME_INDEX &&
-                                   ! (Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0 &&
-                                      Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0 ) ) {
-                            // Invalidate any table with mutations
-                            // TODO: this likely means we don't need the above logic that
-                            // loops through the indexes if adding a PK column, since we'd
-                            // always have header rows for those.
-                            invalidateList.add(new ImmutableBytesPtr(SchemaUtil
-                                    .getTableKey(tenantId,
-                                            rowKeyMetaData[SCHEMA_NAME_INDEX],
-                                            rowKeyMetaData[TABLE_NAME_INDEX])));
-                        }
-                    }
-                    tableMetaData.addAll(additionalTableMetadataMutations);
-                    if (type == PTableType.VIEW) {
-                        if (EncodedColumnsUtil.usesEncodedColumnNames(table) && addingCol
-                                && !table.isAppendOnlySchema()) {
-                            // When adding a column to a view that uses encoded column name
-                            // scheme, we need to modify the CQ counters stored in the view's
-                            // physical table. So to make sure clients get the latest PTable, we
-                            // need to invalidate the cache entry.
-                            // If the table uses APPEND_ONLY_SCHEMA we use the position of the
-                            // column as the encoded column qualifier and so we don't need to
-                            // update the CQ counter in the view physical table (see
-                            // PHOENIX-4737)
-                            invalidateList.add(new ImmutableBytesPtr(
-                                    MetaDataUtil.getPhysicalTableRowForView(table)));
-                        }
-                        // Pass in null as the parent PTable, since we always want to tag the cells
-                        // in this case, irrespective of the property values of the parent
-                        addTagsToPutsForViewAlteredProperties(tableMetaData, null);
-                    }
-                    return null;
-                }
-            }, request.getClientVersion());
+            PTable parentTable = request.hasParentTable() ? PTableImpl.createFromProto(request.getParentTable()) : null;
+            MetaDataMutationResult result = mutateColumn(tableMetaData, new AddColumnMutator(),
+                    request.getClientVersion(), parentTable);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
             }
         } catch (Throwable e) {
             LOGGER.error("Add column failed: ", e);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException("Error when adding column: ", e));
-        }
-    }
-
-    /**
-     * See PHOENIX-4763. If we are modifying any table-level properties that are mutable on a view,
-     * we mark these cells in SYSTEM.CATALOG with tags to indicate that this view property should
-     * not be kept in-sync with the base table and so we shouldn't propagate the base table's
-     * property value when resolving the view
-     * @param tableMetaData list of mutations on the view
-     * @param parent PTable of the parent or null
-     */
-    private void addTagsToPutsForViewAlteredProperties(List<Mutation> tableMetaData,
-            PTable parent) {
-        byte[] parentUpdateCacheFreqBytes = null;
-        byte[] parentUseStatsForParallelizationBytes = null;
-        if (parent != null) {
-            parentUpdateCacheFreqBytes = new byte[PLong.INSTANCE.getByteSize()];
-            PLong.INSTANCE.getCodec().encodeLong(parent.getUpdateCacheFrequency(),
-                    parentUpdateCacheFreqBytes, 0);
-            if (parent.useStatsForParallelization() != null) {
-                parentUseStatsForParallelizationBytes =
-                        PBoolean.INSTANCE.toBytes(parent.useStatsForParallelization());
-            }
-        }
-        for (Mutation m: tableMetaData) {
-            if (m instanceof Put) {
-                MetaDataUtil.conditionallyAddTagsToPutCells((Put)m,
-                        PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-                        PhoenixDatabaseMetaData.UPDATE_CACHE_FREQUENCY_BYTES,
-                        parentUpdateCacheFreqBytes,
-                        VIEW_MODIFIED_PROPERTY_BYTES);
-                MetaDataUtil.conditionallyAddTagsToPutCells((Put)m,
-                        PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-                        PhoenixDatabaseMetaData.USE_STATS_FOR_PARALLELIZATION_BYTES,
-                        parentUseStatsForParallelizationBytes,
-                        VIEW_MODIFIED_PROPERTY_BYTES);
-            }
-
+                    ServerUtil.createIOException("Error when adding column: ", e));
         }
     }
 
     private PTable doGetTable(byte[] tenantId, byte[] schemaName, byte[] tableName,
-            long clientTimeStamp, int clientVersion) throws IOException, SQLException {
-        return doGetTable(tenantId, schemaName, tableName, clientTimeStamp, null, clientVersion,
-            false, false, null);
+                              long clientTimeStamp, int clientVersion) throws IOException, SQLException {
+        return doGetTable(tenantId, schemaName, tableName, clientTimeStamp, null, clientVersion);
     }
 
     /**
      * Looks up the table locally if its present on this region, or else makes an rpc call
-     * to look up the region using PhoenixRuntime.getTable 
-     * @param lockedAncestorTable TODO
+     * to look up the region using PhoenixRuntime.getTable
      */
     private PTable doGetTable(byte[] tenantId, byte[] schemaName, byte[] tableName,
-            long clientTimeStamp, RowLock rowLock, int clientVersion, boolean skipAddingIndexes,
-            boolean skipAddingParentColumns, PTable lockedAncestorTable) throws IOException, SQLException {
+                              long clientTimeStamp, RowLock rowLock, int clientVersion) throws IOException, SQLException {
         Region region = env.getRegion();
         final byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
         // if this region doesn't contain the metadata rows look up the table by using PhoenixRuntime.getTable
         if (!region.getRegionInfo().containsRow(key)) {
-            Properties props = new Properties();
-            if (tenantId != null) {
-                props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(tenantId));
-            }
-            if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
-                props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
-            }
-            try (PhoenixConnection connection =
-                    QueryUtil.getConnectionOnServer(props, env.getConfiguration())
-                            .unwrap(PhoenixConnection.class)) {
-                ConnectionQueryServices queryServices = connection.getQueryServices();
-                MetaDataMutationResult result =
-                        queryServices.getTable(PNameFactory.newName(tenantId), schemaName,
-                            tableName, HConstants.LATEST_TIMESTAMP, clientTimeStamp,
-                            skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable);
-                return result.getTable();
-            } catch (ClassNotFoundException e) {
-            }
+            throw new SQLExceptionInfo.Builder(SQLExceptionCode.GET_TABLE_ERROR)
+                    .setSchemaName(Bytes.toString(schemaName))
+                    .setTableName(Bytes.toString(tableName)).build().buildException();
         }
-        
+
         ImmutableBytesPtr cacheKey = new ImmutableBytesPtr(key);
         // Ask Lars about the expense of this call - if we don't take the lock, we still won't get
         // partial results
@@ -3710,8 +2749,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 rowLock = acquireLock(region, key, null);
             }
             PTable table =
-                    getTableFromCache(cacheKey, clientTimeStamp, clientVersion, skipAddingIndexes,
-                        skipAddingParentColumns, lockedAncestorTable);
+                    getTableFromCache(cacheKey, clientTimeStamp, clientVersion);
             table = modifyIndexStateForOldClient(clientVersion, table);
             // We only cache the latest, so we'll end up building the table with every call if the
             // client connection has specified an SCN.
@@ -3727,19 +2765,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             // Query for the latest table first, since it's not cached
             table =
-                    buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion,
-                        skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable);
-            if ((table != null && table.getTimeStamp() < clientTimeStamp) || 
+                    buildTable(key, cacheKey, region, HConstants.LATEST_TIMESTAMP, clientVersion);
+            if ((table != null && table.getTimeStamp() < clientTimeStamp) ||
                     (blockWriteRebuildIndex && table.getIndexDisableTimestamp() > 0)) {
                 return table;
             }
             // Otherwise, query for an older version of the table - it won't be cached
             table =
-                    buildTable(key, cacheKey, region, clientTimeStamp, clientVersion,
-                        skipAddingIndexes, skipAddingParentColumns, lockedAncestorTable);
+                    buildTable(key, cacheKey, region, clientTimeStamp, clientVersion);
             return table;
         } finally {
-            if (!wasLocked && rowLock!=null) rowLock.release();
+            if (!wasLocked && rowLock != null) rowLock.release();
         }
     }
 
@@ -3757,7 +2793,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
          * Lock directly on key, though it may be an index table. This will just prevent a table
          * from getting rebuilt too often.
          */
-        List<RowLock> rowLocks = new ArrayList<RowLock>(keys.size());;
+        List<RowLock> rowLocks = new ArrayList<RowLock>(keys.size());
+        ;
         try {
             for (int i = 0; i < keys.size(); i++) {
                 acquireLock(region, keys.get(i), rowLocks);
@@ -3766,9 +2803,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             List<PFunction> functionsAvailable = new ArrayList<PFunction>(keys.size());
             int numFunctions = keys.size();
             Iterator<byte[]> iterator = keys.iterator();
-            while(iterator.hasNext()) {
+            while (iterator.hasNext()) {
                 byte[] key = iterator.next();
-                PFunction function = (PFunction)metaDataCache.getIfPresent(new FunctionBytesPtr(key));
+                PFunction function = (PFunction) metaDataCache.getIfPresent(new FunctionBytesPtr(key));
                 if (function != null && function.getTimeStamp() < clientTimeStamp) {
                     if (isFunctionDeleted(function)) {
                         return null;
@@ -3777,195 +2814,74 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     iterator.remove();
                 }
             }
-            if(functionsAvailable.size() == numFunctions) return functionsAvailable;
+            if (functionsAvailable.size() == numFunctions) return functionsAvailable;
 
             // Query for the latest table first, since it's not cached
             List<PFunction> buildFunctions =
                     buildFunctions(keys, region, clientTimeStamp, false,
-                        Collections.<Mutation> emptyList());
-            if(buildFunctions == null || buildFunctions.isEmpty()) {
+                            Collections.<Mutation>emptyList());
+            if (buildFunctions == null || buildFunctions.isEmpty()) {
                 return null;
             }
             functionsAvailable.addAll(buildFunctions);
-            if(functionsAvailable.size() == numFunctions) return functionsAvailable;
+            if (functionsAvailable.size() == numFunctions) return functionsAvailable;
             return null;
         } finally {
-            releaseRowLocks(region,rowLocks);
-        }
-    }
-    
-    private PColumn getColumn(int pkCount, byte[][] rowKeyMetaData, PTable table) throws ColumnFamilyNotFoundException, ColumnNotFoundException {
-        PColumn col = null;
-        if (pkCount > FAMILY_NAME_INDEX
-            && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
-            PColumnFamily family =
-                table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
-            col =
-                family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
-        } else if (pkCount > COLUMN_NAME_INDEX
-            && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
-            col = table.getPKColumn(new String(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
+            releaseRowLocks(region, rowLocks);
         }
-        return col;
     }
 
     @Override
     public void dropColumn(RpcController controller, final DropColumnRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                           RpcCallback<MetaDataResponse> done) {
         List<Mutation> tableMetaData = null;
         final List<byte[]> tableNamesToDelete = Lists.newArrayList();
         final List<SharedTableState> sharedTablesToDelete = Lists.newArrayList();
         try {
             tableMetaData = ProtobufUtil.getMutations(request);
-            MetaDataMutationResult result = mutateColumn(MutatateColumnType.DROP_COLUMN, tableMetaData, new ColumnMutator() {
-                @Override
-                public MetaDataMutationResult updateMutation(PTable table, byte[][] rowKeyMetaData,
-                        List<Mutation> tableMetaData, Region region,
-                        List<ImmutableBytesPtr> invalidateList, List<RowLock> locks, long clientTimeStamp)
-                        throws IOException, SQLException {
-
-                    byte[] tenantId = rowKeyMetaData[TENANT_ID_INDEX];
-                    byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
-                    byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
-                    boolean isView = table.getType() == PTableType.VIEW;
-                    boolean deletePKColumn = false;
-                    getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
-                            SchemaUtil.getTableName(schemaName, tableName),
-                            TableName.valueOf(table.getPhysicalName().getBytes()),
-                            getParentPhysicalTableName(table),table.getType());
-
-                    List<Mutation> additionalTableMetaData = Lists.newArrayList();
-                    PTableType type = table.getType();
-                    if (type == PTableType.TABLE) {
-                        TableViewFinderResult childViewsResult = new TableViewFinderResult();
-                        findAllChildViews(tenantId, table.getSchemaName().getBytes(), table.getTableName().getBytes(), childViewsResult);
-                        if (childViewsResult.hasLinks()) {
-                            MetaDataMutationResult mutationResult =
-                                    dropViewIndexes(region, table,
-                                        locks, tableMetaData, additionalTableMetaData,
-                                        schemaName, tableName, invalidateList,
-                                        clientTimeStamp, childViewsResult, tableNamesToDelete, sharedTablesToDelete, request.getClientVersion());
-                            // return if we were not able to drop view indexes that need this column successfully
-                            if (mutationResult != null) return mutationResult;
-                        }
-                    }
-                    ListIterator<Mutation> iterator = tableMetaData.listIterator();
-                    while (iterator.hasNext()) {
-                        Mutation mutation = iterator.next();
-                        byte[] key = mutation.getRow();
-                        int pkCount = getVarChars(key, rowKeyMetaData);
-                        if (isView && mutation instanceof Put) {
-                            PColumn column = getColumn(pkCount, rowKeyMetaData, table);
-                            if (column == null)
-                                continue;
-                            // ignore any puts that modify the ordinal positions of columns
-                            iterator.remove();
-                        } 
-                        else if (mutation instanceof Delete) {
-                            if (pkCount > COLUMN_NAME_INDEX
-                                && Bytes.compareTo(schemaName, rowKeyMetaData[SCHEMA_NAME_INDEX]) == 0
-                                && Bytes.compareTo(tableName, rowKeyMetaData[TABLE_NAME_INDEX]) == 0) {
-                                PColumn columnToDelete = null;
-                                try {
-                                    columnToDelete = getColumn(pkCount, rowKeyMetaData, table);
-                                    if (columnToDelete == null)
-                                        continue;
-                                    deletePKColumn = columnToDelete.getFamilyName() == null;
-                                    if (isView) {
-                                        // if we are dropping a derived column add it to the excluded column list
-                                        if (columnToDelete.isDerived()) {
-                                            mutation = MetaDataUtil
-                                                .cloneDeleteToPutAndAddColumn((Delete) mutation, TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, LinkType.EXCLUDED_COLUMN.getSerializedValueAsByteArray());
-                                            iterator.set(mutation);
-                                        }
-
-                                        if (table.getBaseColumnCount() != DIVERGED_VIEW_BASE_COLUMN_COUNT
-                                            && columnToDelete.isDerived()) {
-                                            /*
-                                             * If the column being dropped is inherited from the base table, then the
-                                             * view is about to diverge itself from the base table. The consequence of
-                                             * this divergence is that that any further meta-data changes made to the
-                                             * base table will not be propagated to the hierarchy of views where this
-                                             * view is the root.
-                                             */
-                                            byte[] viewKey = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
-                                            Put updateBaseColumnCountPut = new Put(viewKey);
-                                            byte[] baseColumnCountPtr = new byte[PInteger.INSTANCE.getByteSize()];
-                                            PInteger.INSTANCE.getCodec().encodeInt(DIVERGED_VIEW_BASE_COLUMN_COUNT,
-                                                baseColumnCountPtr, 0);
-                                            updateBaseColumnCountPut.addColumn(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES,
-                                                PhoenixDatabaseMetaData.BASE_COLUMN_COUNT_BYTES, clientTimeStamp,
-                                                baseColumnCountPtr);
-                                            additionalTableMetaData.add(updateBaseColumnCountPut);
-                                        }
-                                    }
-                                    if (columnToDelete.isViewReferenced()) { // Disallow deletion of column referenced in WHERE clause of view
-                                        return new MetaDataMutationResult(MutationCode.UNALLOWED_TABLE_MUTATION, EnvironmentEdgeManager.currentTimeMillis(), table, columnToDelete);
-                                    }
-                                    // drop any indexes that need the column that is going to be dropped
-                                    MetaDataMutationResult result = dropIndexes(table, region, invalidateList, locks,
-                                        clientTimeStamp, schemaName, tableName,
-                                        additionalTableMetaData, columnToDelete,
-                                        tableNamesToDelete, sharedTablesToDelete, request.getClientVersion(), null);
-                                    if (result != null) {
-                                        return result;
-                                    }
-                                } catch (ColumnFamilyNotFoundException e) {
-                                    return new MetaDataMutationResult(
-                                        MutationCode.COLUMN_NOT_FOUND, EnvironmentEdgeManager
-                                        .currentTimeMillis(), table, columnToDelete);
-                                } catch (ColumnNotFoundException e) {
-                                    return new MetaDataMutationResult(
-                                        MutationCode.COLUMN_NOT_FOUND, EnvironmentEdgeManager
-                                        .currentTimeMillis(), table, columnToDelete);
-                                }
-                            }
-                        }
-
-                    }
-                    if (deletePKColumn) {
-                        if (table.getPKColumns().size() == 1) {
-                            return new MetaDataMutationResult(MutationCode.NO_PK_COLUMNS,
-                                    EnvironmentEdgeManager.currentTimeMillis(), null);
-                        }
-                    }
-                    tableMetaData.addAll(additionalTableMetaData);
-                    long currentTime = MetaDataUtil.getClientTimeStamp(tableMetaData);
-                    return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, null, tableNamesToDelete, sharedTablesToDelete);
-                }
-            }, request.getClientVersion());
+            PTable parentTable = request.hasParentTable() ? PTableImpl.createFromProto(request.getParentTable()) : null;
+            MetaDataMutationResult result = mutateColumn(tableMetaData, new DropColumnMutator(env.getConfiguration()),
+                    request.getClientVersion(), parentTable);
             if (result != null) {
                 done.run(MetaDataMutationResult.toProto(result));
             }
         } catch (Throwable e) {
             LOGGER.error("Drop column failed: ", e);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException("Error when dropping column: ", e));
+                    ServerUtil.createIOException("Error when dropping column: ", e));
         }
     }
 
-    private MetaDataMutationResult dropIndexes(PTable table, Region region, List<ImmutableBytesPtr> invalidateList,
-            List<RowLock> locks, long clientTimeStamp, byte[] schemaName,
-            byte[] tableName, List<Mutation> additionalTableMetaData, PColumn columnToDelete, 
-            List<byte[]> tableNamesToDelete, List<SharedTableState> sharedTablesToDelete, int clientVersion, PTable basePhysicalTable)
+    private MetaDataMutationResult dropIndexes(RegionCoprocessorEnvironment env, PTable table,
+                                               List<ImmutableBytesPtr> invalidateList, List<RowLock> locks,
+                                               long clientTimeStamp, List<Mutation> tableMetaData,
+                                               PColumn columnToDelete, List<byte[]> tableNamesToDelete,
+                                               List<SharedTableState> sharedTablesToDelete, int clientVersion)
             throws IOException, SQLException {
         // Look for columnToDelete in any indexes. If found as PK column, get lock and drop the
         // index and then invalidate it
         // Covered columns are deleted from the index by the client
+        Region region = env.getRegion();
         PhoenixConnection connection = null;
         try {
             connection = table.getIndexes().isEmpty() ? null : QueryUtil.getConnectionOnServer(
-                env.getConfiguration()).unwrap(PhoenixConnection.class);
+                    env.getConfiguration()).unwrap(PhoenixConnection.class);
         } catch (ClassNotFoundException e) {
         }
         for (PTable index : table.getIndexes()) {
+            // ignore any indexes derived from ancestors
+            if (index.getName().getString().contains(QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR)) {
+                continue;
+            }
             byte[] tenantId = index.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : index.getTenantId().getBytes();
             IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
             byte[] indexKey =
                     SchemaUtil.getTableKey(tenantId, index.getSchemaName().getBytes(), index
                             .getTableName().getBytes());
-            Pair<String, String> columnToDeleteInfo = new Pair<>(columnToDelete.getFamilyName().getString(), columnToDelete.getName().getString());
-            ColumnReference colDropRef = new ColumnReference(columnToDelete.getFamilyName().getBytes(), columnToDelete.getColumnQualifierBytes());
+            Pair<String, String> columnToDeleteInfo = new Pair<>(columnToDelete.getFamilyName().getString(),
+                    columnToDelete.getName().getString());
+            ColumnReference colDropRef = new ColumnReference(columnToDelete.getFamilyName().getBytes(),
+                    columnToDelete.getColumnQualifierBytes());
             boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(columnToDeleteInfo);
             boolean isCoveredColumn = indexMaintainer.getCoveredColumns().contains(colDropRef);
             // If index requires this column for its pk, then drop it
@@ -3975,62 +2891,115 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // index table
                 Delete delete = new Delete(indexKey, clientTimeStamp);
                 byte[] linkKey =
-                        MetaDataUtil.getParentLinkKey(tenantId, schemaName, tableName, index
-                                .getTableName().getBytes());
+                        MetaDataUtil.getParentLinkKey(tenantId, table.getSchemaName().getBytes(),
+                                table.getTableName().getBytes(), index.getTableName().getBytes());
                 // Drop the link between the parent table and the
                 // index table
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
-                List<Mutation> tableMetaData = Lists.newArrayListWithExpectedSize(2);
                 Delete tableDelete = delete;
                 tableMetaData.add(tableDelete);
                 tableMetaData.add(linkDelete);
-                // if the index is not present on the current region make an rpc to drop it
-                if (!region.getRegionInfo().containsRow(indexKey)) {
-                    Properties props = new Properties();
-                    if (tenantId != null) {
-                        props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(tenantId));
-                    }
-                    if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
-                        props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
-                    }
-                    ConnectionQueryServices queryServices = connection.getQueryServices();
-                    MetaDataMutationResult result =
-                            queryServices.dropTable(tableMetaData, PTableType.INDEX, false, true);
-                    if (result.getTableNamesToDelete()!=null && !result.getTableNamesToDelete().isEmpty())
-                        tableNamesToDelete.addAll(result.getTableNamesToDelete());
-                    if (result.getSharedTablesToDelete()!=null && !result.getSharedTablesToDelete().isEmpty())
-                        sharedTablesToDelete.addAll(result.getSharedTablesToDelete());
-                    if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
-                        return result;
-                    }
+                // Since we're dropping the index, lock it to ensure
+                // that a change in index state doesn't
+                // occur while we're dropping it.
+                acquireLock(region, indexKey, locks);
+                List<Mutation> childLinksMutations = Lists.newArrayList();
+                MetaDataMutationResult result = doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(),
+                        index.getTableName().getBytes(), table.getName().getBytes(), index.getType(),
+                        tableMetaData, childLinksMutations, invalidateList, tableNamesToDelete, sharedTablesToDelete,
+                        false, clientVersion);
+                if (result != null && result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                    return result;
                 }
-                else {
-                    // Since we're dropping the index, lock it to ensure
-                    // that a change in index state doesn't
-                    // occur while we're dropping it.
-                    acquireLock(region, indexKey, locks);
-                    additionalTableMetaData.addAll(tableMetaData);
-                    List<Mutation> childLinksMutations = Lists.newArrayList();
-                    doDropTable(indexKey, tenantId, index.getSchemaName().getBytes(),
-                        index.getTableName().getBytes(), tableName, index.getType(),
-                        additionalTableMetaData, childLinksMutations, invalidateList,
-                        tableNamesToDelete, sharedTablesToDelete, false, clientVersion);
-                    // there should be no child links to delete since we are just dropping an index
-                    assert(childLinksMutations.isEmpty());
-                    invalidateList.add(new ImmutableBytesPtr(indexKey));
+                // there should be no child links to delete since we are just dropping an index
+                if (!childLinksMutations.isEmpty()) {
+                    LOGGER.error("Found unexpected child link mutations while dropping an index "
+                            + childLinksMutations);
                 }
+                invalidateList.add(new ImmutableBytesPtr(indexKey));
             }
             // If the dropped column is a covered index column, invalidate the index
-            else if (isCoveredColumn){
+            else if (isCoveredColumn) {
                 invalidateList.add(new ImmutableBytesPtr(indexKey));
             }
         }
         return null;
     }
 
+    private MetaDataMutationResult dropRemoteIndexes(RegionCoprocessorEnvironment env, PTable table,
+                                                     long clientTimeStamp, PColumn columnToDelete,
+                                                     List<byte[]> tableNamesToDelete,
+                                                     List<SharedTableState> sharedTablesToDelete)
+            throws SQLException {
+        // Look for columnToDelete in any indexes. If found as PK column, get lock and drop the
+        // index and then invalidate it
+        // Covered columns are deleted from the index by the client
+        PhoenixConnection connection = null;
+        try {
+            connection = table.getIndexes().isEmpty() ? null : QueryUtil.getConnectionOnServer(
+                    env.getConfiguration()).unwrap(PhoenixConnection.class);
+        } catch (ClassNotFoundException e) {
+        }
+        for (PTable index : table.getIndexes()) {
+            byte[] tenantId = index.getTenantId() == null ? ByteUtil.EMPTY_BYTE_ARRAY : index.getTenantId().getBytes();
+            IndexMaintainer indexMaintainer = index.getIndexMaintainer(table, connection);
+            byte[] indexKey =
+                    SchemaUtil.getTableKey(tenantId, index.getSchemaName().getBytes(), index
+                            .getTableName().getBytes());
+            Pair<String, String> columnToDeleteInfo =
+                    new Pair<>(columnToDelete.getFamilyName().getString(), columnToDelete.getName().getString());
+            ColumnReference colDropRef =
+                    new ColumnReference(columnToDelete.getFamilyName().getBytes(),
+                            columnToDelete.getColumnQualifierBytes());
+            boolean isColumnIndexed = indexMaintainer.getIndexedColumnInfo().contains(columnToDeleteInfo);
+            boolean isCoveredColumn = indexMaintainer.getCoveredColumns().contains(colDropRef);
+            // If index requires this column for its pk, then drop it
+            if (isColumnIndexed) {
+                // Drop the index table. The doDropTable will expand
+                // this to all of the table rows and invalidate the
+                // index table
+                Delete delete = new Delete(indexKey, clientTimeStamp);
+                byte[] linkKey =
+                        MetaDataUtil.getParentLinkKey(tenantId, table.getSchemaName().getBytes(),
+                                table.getTableName().getBytes(), index.getTableName().getBytes());
+                // Drop the link between the parent table and the
+                // index table
+                Delete linkDelete = new Delete(linkKey, clientTimeStamp);
+                List<Mutation> remoteDropMetadata = Lists.newArrayListWithExpectedSize(2);
+                Delete tableDelete = delete;
+                remoteDropMetadata.add(tableDelete);
+                remoteDropMetadata.add(linkDelete);
+                // if the index is not present on the current region make an rpc to drop it
+                Properties props = new Properties();
+                if (tenantId != null) {
+                    props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(tenantId));
+                }
+                if (clientTimeStamp != HConstants.LATEST_TIMESTAMP) {
+                    props.setProperty("CurrentSCN", Long.toString(clientTimeStamp));
+                }
+                ConnectionQueryServices queryServices = connection.getQueryServices();
+                MetaDataMutationResult result =
+                        queryServices.dropTable(remoteDropMetadata, PTableType.INDEX, false);
+                if (result.getTableNamesToDelete() != null && !result.getTableNamesToDelete().isEmpty())
+                    tableNamesToDelete.addAll(result.getTableNamesToDelete());
+                if (result.getSharedTablesToDelete() != null && !result.getSharedTablesToDelete().isEmpty())
+                    sharedTablesToDelete.addAll(result.getSharedTablesToDelete());
+                if (result.getMutationCode() != MutationCode.TABLE_ALREADY_EXISTS) {
+                    return result;
+                }
+            }
+            // If the dropped column is a covered index column, invalidate the index
+            else if (isCoveredColumn) {
+                clearRemoteTableFromCache(clientTimeStamp, index.getSchemaName() != null ?
+                        index.getSchemaName().getBytes() : ByteUtil.EMPTY_BYTE_ARRAY, index.getTableName().getBytes());
+            }
+        }
+        return null;
+    }
+
     @Override
     public void clearCache(RpcController controller, ClearCacheRequest request,
-            RpcCallback<ClearCacheResponse> done) {
+                           RpcCallback<ClearCacheResponse> done) {
         GlobalCache cache = GlobalCache.getInstance(this.env);
         Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                 GlobalCache.getInstance(this.env).getMetaDataCache();
@@ -4053,8 +3022,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     ServerUtil.createIOException(
                             SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
                                     isTablesMappingEnabled).toString(),
-                    new DoNotRetryIOException(
-                            "Old client is not compatible when" + " system tables are upgraded to map to namespace")));
+                            new DoNotRetryIOException(
+                                    "Old client is not compatible when" + " system tables are upgraded to map to namespace")));
         }
         long version = MetaDataUtil.encodeVersion(env.getHBaseVersion(), config);
 
@@ -4063,13 +3032,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             systemCatalog =
                     doGetTable(ByteUtil.EMPTY_BYTE_ARRAY, PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA_BYTES,
                             PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE_BYTES, HConstants.LATEST_TIMESTAMP, null,
-                            request.getClientVersion(), false, false, null);
+                            request.getClientVersion());
         } catch (Throwable t) {
             LOGGER.error("loading system catalog table inside getVersion failed", t);
             ProtobufUtil.setControllerException(controller,
-              ServerUtil.createIOException(
-                SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
-                  isTablesMappingEnabled).toString(), t));
+                    ServerUtil.createIOException(
+                            SchemaUtil.getPhysicalTableName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
+                                    isTablesMappingEnabled).toString(), t));
         }
         // In case this is the first connection, system catalog does not exist, and so we don't
         // set the optional system catalog timestamp.
@@ -4082,7 +3051,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     @Override
     public void updateIndexState(RpcController controller, UpdateIndexStateRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                                 RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         byte[] schemaName = null;
         byte[] tableName = null;
@@ -4107,15 +3076,15 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             int disableTimeStampKVIndex = -1;
             int indexStateKVIndex = 0;
             int index = 0;
-            for(Cell cell : newKVs) {
-                if(Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
-                      INDEX_STATE_BYTES, 0, INDEX_STATE_BYTES.length) == 0){
-                  newKV = cell;
-                  indexStateKVIndex = index;
-                  timeStamp = cell.getTimestamp();
+            for (Cell cell : newKVs) {
+                if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
+                        INDEX_STATE_BYTES, 0, INDEX_STATE_BYTES.length) == 0) {
+                    newKV = cell;
+                    indexStateKVIndex = index;
+                    timeStamp = cell.getTimestamp();
                 } else if (Bytes.compareTo(cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
-                  INDEX_DISABLE_TIMESTAMP_BYTES, 0, INDEX_DISABLE_TIMESTAMP_BYTES.length) == 0) {
-                  disableTimeStampKVIndex = index;
+                        INDEX_DISABLE_TIMESTAMP_BYTES, 0, INDEX_DISABLE_TIMESTAMP_BYTES.length) == 0) {
+                    disableTimeStampKVIndex = index;
                 }
                 index++;
             }
@@ -4147,7 +3116,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 PTable loadedTable =
                         doGetTable(tenantId, schemaName, tableName, clientTimeStamp, null,
-                            request.getClientVersion(), false, false, null);
+                                request.getClientVersion());
                 if (loadedTable == null) {
                     builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
                     builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
@@ -4191,7 +3160,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         // when a new data table write occurs.
                         // We do legitimately move the INDEX_DISABLE_TIMESTAMP to be newer when we're rebuilding the
                         // index in which case the state will be INACTIVE or PENDING_ACTIVE.
-                        if (curTimeStampVal != 0 
+                        if (curTimeStampVal != 0
                                 && (newState == PIndexState.DISABLE || newState == PIndexState.PENDING_ACTIVE || newState == PIndexState.PENDING_DISABLE)
                                 && Math.abs(curTimeStampVal) < Math.abs(newDisableTimeStamp)) {
                             // do not reset disable timestamp as we want to keep the min
@@ -4219,7 +3188,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
                     // Can't transition back to INACTIVE if INDEX_DISABLE_TIMESTAMP is 0
                     if (newState != PIndexState.BUILDING && newState != PIndexState.DISABLE &&
-                        (newState != PIndexState.INACTIVE || curTimeStampVal == 0)) {
+                            (newState != PIndexState.INACTIVE || curTimeStampVal == 0)) {
                         builder.setReturnCode(MetaDataProtos.MutationCode.UNALLOWED_TABLE_MUTATION);
                         builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                         done.run(builder.build());
@@ -4230,7 +3199,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         newState = PIndexState.DISABLE;
                     }
                 }
-                if (newState == PIndexState.PENDING_DISABLE && currentState != PIndexState.PENDING_DISABLE && currentState != PIndexState.INACTIVE) {
+                if (newState == PIndexState.PENDING_DISABLE && currentState != PIndexState.PENDING_DISABLE
+                        && currentState != PIndexState.INACTIVE) {
                     // reset count for first PENDING_DISABLE
                     newKVs.add(KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
                             PhoenixDatabaseMetaData.PENDING_DISABLE_COUNT_BYTES, timeStamp, Bytes.toBytes(0L)));
@@ -4249,7 +3219,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                                     INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
                         } else if (disableTimeStampKVIndex == -1) { // clear disableTimestamp if client didn't pass it in
                             newKVs.add(KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
-                                PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES, timeStamp, PLong.INSTANCE.toBytes(0)));
+                                    PhoenixDatabaseMetaData.INDEX_DISABLE_TIMESTAMP_BYTES, timeStamp, PLong.INSTANCE.toBytes(0)));
                             disableTimeStampKVIndex = newKVs.size() - 1;
                         }
                     } else if (newState == PIndexState.DISABLE) {
@@ -4259,19 +3229,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
 
                 }
-                
-                if(newState == PIndexState.ACTIVE||newState == PIndexState.PENDING_ACTIVE||newState == PIndexState.DISABLE){
+
+                if (newState == PIndexState.ACTIVE || newState == PIndexState.PENDING_ACTIVE || newState == PIndexState.DISABLE) {
                     newKVs.add(KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
-                            PhoenixDatabaseMetaData.PENDING_DISABLE_COUNT_BYTES, timeStamp, Bytes.toBytes(0L)));   
+                            PhoenixDatabaseMetaData.PENDING_DISABLE_COUNT_BYTES, timeStamp, Bytes.toBytes(0L)));
                 }
-                
+
                 if (currentState == PIndexState.BUILDING && newState != PIndexState.ACTIVE) {
                     timeStamp = currentStateKV.getTimestamp();
                 }
                 if ((currentState == PIndexState.ACTIVE || currentState == PIndexState.PENDING_ACTIVE) && newState == PIndexState.UNUSABLE) {
                     newState = PIndexState.INACTIVE;
                     newKVs.set(indexStateKVIndex, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
-                        INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
+                            INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
                 } else if ((currentState == PIndexState.INACTIVE || currentState == PIndexState.PENDING_ACTIVE) && newState == PIndexState.USABLE) {
                     // Don't allow manual state change to USABLE (i.e. ACTIVE) if non zero INDEX_DISABLE_TIMESTAMP
                     if (curTimeStampVal != 0) {
@@ -4280,7 +3250,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         newState = PIndexState.ACTIVE;
                     }
                     newKVs.set(indexStateKVIndex, KeyValueUtil.newKeyValue(key, TABLE_FAMILY_BYTES,
-                        INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
+                            INDEX_STATE_BYTES, timeStamp, Bytes.toBytes(newState.getSerializedValue())));
                 }
 
                 PTable returnTable = null;
@@ -4290,9 +3260,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     // Always include the empty column value at latest timestamp so
                     // that clients pull over update.
                     Put emptyValue = new Put(key);
-                    emptyValue.addColumn(TABLE_FAMILY_BYTES, 
-                            QueryConstants.EMPTY_COLUMN_BYTES, 
-                            HConstants.LATEST_TIMESTAMP, 
+                    emptyValue.addColumn(TABLE_FAMILY_BYTES,
+                            QueryConstants.EMPTY_COLUMN_BYTES,
+                            HConstants.LATEST_TIMESTAMP,
                             QueryConstants.EMPTY_COLUMN_VALUE_BYTES);
                     tableMetadata.add(emptyValue);
                     byte[] dataTableKey = null;
@@ -4312,17 +3282,19 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (setRowKeyOrderOptimizableCell) {
                         UpgradeUtil.addRowKeyOrderOptimizableCell(tableMetadata, key, timeStamp);
                     }
-                    mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
-                        HConstants.NO_NONCE);
+                    mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
+                            HConstants.NO_NONCE);
                     // Invalidate from cache
-                    Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                    Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
+                            GlobalCache.getInstance(this.env).getMetaDataCache();
                     metaDataCache.invalidate(cacheKey);
-                    if(dataTableKey != null) {
+                    if (dataTableKey != null) {
                         metaDataCache.invalidate(new ImmutableBytesPtr(dataTableKey));
                     }
                     if (setRowKeyOrderOptimizableCell || disableTimeStampKVIndex != -1
                             || currentState == PIndexState.DISABLE || newState == PIndexState.BUILDING) {
-                        returnTable = doGetTable(tenantId, schemaName, tableName, HConstants.LATEST_TIMESTAMP, rowLock, request.getClientVersion(), false, false, null);
+                        returnTable = doGetTable(tenantId, schemaName, tableName,
+                                HConstants.LATEST_TIMESTAMP, rowLock, request.getClientVersion());
                     }
                 }
                 // Get client timeStamp from mutations, since it may get updated by the
@@ -4341,20 +3313,20 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } catch (Throwable t) {
             LOGGER.error("updateIndexState failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
+                    ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
 
     private static MetaDataMutationResult checkKeyInRegion(byte[] key, Region region, MutationCode code) {
-        return ServerUtil.isKeyInRegion(key, region) ? null : 
-            new MetaDataMutationResult(code, EnvironmentEdgeManager.currentTimeMillis(), null);
+        return ServerUtil.isKeyInRegion(key, region) ? null :
+                new MetaDataMutationResult(code, EnvironmentEdgeManager.currentTimeMillis(), null);
     }
 
     private static MetaDataMutationResult checkTableKeyInRegion(byte[] key, Region region) {
         MetaDataMutationResult result = checkKeyInRegion(key, region, MutationCode.TABLE_NOT_IN_REGION);
-        if (result!=null) {
+        if (result != null) {
             LOGGER.error("Table rowkey " + Bytes.toStringBinary(key)
-            + " is not in the current region " + region.getRegionInfo());
+                    + " is not in the current region " + region.getRegionInfo());
         }
         return result;
 
@@ -4368,12 +3340,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         return checkKeyInRegion(key, region, MutationCode.SCHEMA_NOT_IN_REGION);
 
     }
-    
+
     private static class ViewInfo {
         private byte[] tenantId;
         private byte[] schemaName;
         private byte[] viewName;
-        
+
         public ViewInfo(byte[] tenantId, byte[] schemaName, byte[] viewName) {
             super();
             this.tenantId = tenantId;
@@ -4396,7 +3368,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     @Override
     public void clearTableFromCache(RpcController controller, ClearTableFromCacheRequest request,
-            RpcCallback<ClearTableFromCacheResponse> done) {
+                                    RpcCallback<ClearTableFromCacheResponse> done) {
         byte[] schemaName = request.getSchemaName().toByteArray();
         byte[] tableName = request.getTableName().toByteArray();
         try {
@@ -4409,7 +3381,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } catch (Throwable t) {
             LOGGER.error("clearTableFromCache failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
+                    ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
 
@@ -4458,13 +3430,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             done.run(builder.build());
             return;
         } finally {
-            releaseRowLocks(region,locks);
+            releaseRowLocks(region, locks);
         }
     }
 
     @Override
     public void getFunctions(RpcController controller, GetFunctionsRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                             RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         byte[] tenantId = request.getTenantId().toByteArray();
         List<String> functionNames = new ArrayList<>(request.getFunctionNamesCount());
@@ -4474,7 +3446,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             List<Long> functionTimestampsList = request.getFunctionTimestampsList();
             List<byte[]> keys = new ArrayList<byte[]>(request.getFunctionNamesCount());
             List<Pair<byte[], Long>> functions = new ArrayList<Pair<byte[], Long>>(request.getFunctionNamesCount());
-            for(int i = 0; i< functionNamesList.size();i++) {
+            for (int i = 0; i < functionNamesList.size(); i++) {
                 byte[] functionName = functionNamesList.get(i).toByteArray();
                 functionNames.add(Bytes.toString(functionName));
                 byte[] key = SchemaUtil.getFunctionKey(tenantId, functionName);
@@ -4483,7 +3455,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                functions.add(new Pair<byte[], Long>(functionName,functionTimestampsList.get(i)));
+                functions.add(new Pair<byte[], Long>(functionName, functionTimestampsList.get(i)));
                 keys.add(key);
             }
 
@@ -4506,13 +3478,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } catch (Throwable t) {
             LOGGER.error("getFunctions failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(functionNames.toString(), t));
+                    ServerUtil.createIOException(functionNames.toString(), t));
         }
     }
 
     @Override
     public void createFunction(RpcController controller, CreateFunctionRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                               RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         byte[][] rowKeyMetaData = new byte[2][];
         byte[] functionName = null;
@@ -4547,7 +3519,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
                             builder.addFunction(PFunction.toProto(function));
                             done.run(builder.build());
-                            if(!request.getReplace()) {
+                            if (!request.getReplace()) {
                                 return;
                             }
                         }
@@ -4560,13 +3532,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     }
                 }
                 // Don't store function info for temporary functions.
-                if(!temporaryFunction) {
-                    mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                if (!temporaryFunction) {
+                    mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 }
 
                 // Invalidate the cache - the next getFunction call will add it
                 // TODO: consider loading the function that was just created here, patching up the parent function, and updating the cache
-                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 metaDataCache.invalidate(cacheKey);
                 // Get timeStamp from mutations - the above method sets it if it's unset
                 long currentTimeStamp = MetaDataUtil.getClientTimeStamp(functionMetaData);
@@ -4575,18 +3547,18 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                releaseRowLocks(region,locks);
+                releaseRowLocks(region, locks);
             }
         } catch (Throwable t) {
             LOGGER.error("createFunction failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(Bytes.toString(functionName), t));
+                    ServerUtil.createIOException(Bytes.toString(functionName), t));
         }
     }
 
     @Override
     public void dropFunction(RpcController controller, DropFunctionRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                             RpcCallback<MetaDataResponse> done) {
         byte[][] rowKeyMetaData = new byte[2][];
         byte[] functionName = null;
         try {
@@ -4614,11 +3586,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
-                Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
+                Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 long currentTime = MetaDataUtil.getClientTimeStamp(functionMetaData);
-                for(ImmutableBytesPtr ptr: invalidateList) {
+                for (ImmutableBytesPtr ptr : invalidateList) {
                     metaDataCache.invalidate(ptr);
                     metaDataCache.put(ptr, newDeletedFunctionMarker(currentTime));
 
@@ -4627,12 +3599,12 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                releaseRowLocks(region,locks);
+                releaseRowLocks(region, locks);
             }
         } catch (Throwable t) {
             LOGGER.error("dropFunction failed", t);
             ProtobufUtil.setControllerException(controller,
-                ServerUtil.createIOException(Bytes.toString(functionName), t));
+                    ServerUtil.createIOException(Bytes.toString(functionName), t));
         }
     }
 
@@ -4662,17 +3634,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 Scan scan = MetaDataUtil.newTableRowsScan(keys.get(0), MIN_TABLE_TIMESTAMP, clientTimeStamp);
                 List<Cell> results = Lists.newArrayList();
                 try (RegionScanner scanner = region.getScanner(scan);) {
-                  scanner.next(results);
-                  if (results.isEmpty()) { // Should not be possible
-                    return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
-                  }
-                  do {
-                    Cell kv = results.get(0);
-                    Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
-                    functionMetaData.add(delete);
-                    results.clear();
                     scanner.next(results);
-                  } while (!results.isEmpty());
+                    if (results.isEmpty()) { // Should not be possible
+                        return new MetaDataMutationResult(MutationCode.FUNCTION_NOT_FOUND, EnvironmentEdgeManager.currentTimeMillis(), null);
+                    }
+                    do {
+                        Cell kv = results.get(0);
+                        Delete delete = new Delete(kv.getRowArray(), kv.getRowOffset(), kv.getRowLength(), clientTimeStamp);
+                        functionMetaData.add(delete);
+                        results.clear();
+                        scanner.next(results);
+                    } while (!results.isEmpty());
                 }
                 return new MetaDataMutationResult(MutationCode.FUNCTION_ALREADY_EXISTS,
                         EnvironmentEdgeManager.currentTimeMillis(), functions, true);
@@ -4684,7 +3656,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     @Override
     public void createSchema(RpcController controller, CreateSchemaRequest request,
-            RpcCallback<MetaDataResponse> done) {
+                             RpcCallback<MetaDataResponse> done) {
         MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
         String schemaName = null;
         try {
@@ -4724,7 +3696,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                 }
-                mutateRowsWithLocks(region, schemaMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, schemaMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getSchema call will add it
@@ -4742,7 +3714,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                releaseRowLocks(region,locks);
+                releaseRowLocks(region, locks);
             }
         } catch (Throwable t) {
             LOGGER.error("Creating the schema" + schemaName + "failed", t);
@@ -4774,7 +3746,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                mutateRowsWithLocks(region, schemaMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, schemaMetaData, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env)
                         .getMetaDataCache();
@@ -4786,7 +3758,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                releaseRowLocks(region,locks);
+                releaseRowLocks(region, locks);
             }
         } catch (Throwable t) {
             LOGGER.error("drop schema failed:", t);
@@ -4795,11 +3767,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     }
 
     private MetaDataMutationResult doDropSchema(long clientTimeStamp, String schemaName, byte[] key,
-            List<Mutation> schemaMutations, List<ImmutableBytesPtr> invalidateList) throws IOException, SQLException {
+                                                List<Mutation> schemaMutations, List<ImmutableBytesPtr> invalidateList) throws IOException, SQLException {
         PSchema schema = loadSchema(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp);
         boolean areTablesExists = false;
-        if (schema == null) { return new MetaDataMutationResult(MutationCode.SCHEMA_NOT_FOUND,
-                EnvironmentEdgeManager.currentTimeMillis(), null); }
+        if (schema == null) {
+            return new MetaDataMutationResult(MutationCode.SCHEMA_NOT_FOUND,
+                    EnvironmentEdgeManager.currentTimeMillis(), null);
+        }
         if (schema.getTimeStamp() < clientTimeStamp) {
             Region region = env.getRegion();
             Scan scan = MetaDataUtil.newTableRowsScan(SchemaUtil.getKeyForSchema(null, schemaName), MIN_TABLE_TIMESTAMP,
@@ -4822,8 +3796,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     scanner.next(results);
                 } while (!results.isEmpty());
             }
-            if (areTablesExists) { return new MetaDataMutationResult(MutationCode.TABLES_EXIST_ON_SCHEMA, schema,
-                    EnvironmentEdgeManager.currentTimeMillis()); }
+            if (areTablesExists) {
+                return new MetaDataMutationResult(MutationCode.TABLES_EXIST_ON_SCHEMA, schema,
+                        EnvironmentEdgeManager.currentTimeMillis());
+            }
             invalidateList.add(new ImmutableBytesPtr(key));
             return new MetaDataMutationResult(MutationCode.SCHEMA_ALREADY_EXISTS, schema,
                     EnvironmentEdgeManager.currentTimeMillis());
@@ -4832,9 +3808,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 null);
 
     }
-    
+
     private void mutateRowsWithLocks(final Region region, final List<Mutation> mutations, final Set<byte[]> rowsToLock,
-            final long nonceGroup, final long nonce) throws IOException {
+                                     final long nonceGroup, final long nonce) throws IOException {
         // we need to mutate SYSTEM.CATALOG with HBase/login user if access is enabled.
         if (this.accessCheckEnabled) {
             User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
@@ -4858,22 +3834,22 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             region.mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce);
         }
     }
-    
+
     private TableName getParentPhysicalTableName(PTable table) {
         return table
                 .getType() == PTableType.VIEW
-                        ? TableName.valueOf(table.getPhysicalName().getBytes())
-                        : table.getType() == PTableType.INDEX
-                                ? TableName
-                                        .valueOf(SchemaUtil
-                                                .getPhysicalHBaseTableName(table.getParentSchemaName(),
-                                                        table.getParentTableName(), table.isNamespaceMapped())
-                                                .getBytes())
-                                : TableName
-                                        .valueOf(
-                                                SchemaUtil
-                                                        .getPhysicalHBaseTableName(table.getSchemaName(),
-                                                                table.getTableName(), table.isNamespaceMapped())
-                                                        .getBytes());
+                ? TableName.valueOf(table.getPhysicalName().getBytes())
+                : table.getType() == PTableType.INDEX
+                ? TableName
+                .valueOf(SchemaUtil
+                        .getPhysicalHBaseTableName(table.getParentSchemaName(),
+                                table.getParentTableName(), table.isNamespaceMapped())
+                        .getBytes())
+                : TableName
+                .valueOf(
+                        SchemaUtil
+                                .getPhysicalHBaseTableName(table.getSchemaName(),
+                                        table.getTableName(), table.isNamespaceMapped())
+                                .getBytes());
     }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
deleted file mode 100644
index 5060d73..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ViewFinder.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- * http://www.apache.org/licenses/LICENSE-2.0
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.coprocessor;
-
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
-import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
-import static org.apache.phoenix.util.SchemaUtil.getVarChars;
-
-import java.io.IOException;
-import java.util.List;
-
-import org.apache.hadoop.hbase.HConstants;
-import org.apache.hadoop.hbase.client.Result;
-import org.apache.hadoop.hbase.client.ResultScanner;
-import org.apache.hadoop.hbase.client.Scan;
-import org.apache.hadoop.hbase.client.Table;
-import org.apache.hadoop.hbase.filter.CompareFilter;
-import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
-import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
-import org.apache.phoenix.schema.PTable;
-import org.apache.phoenix.schema.PTable.LinkType;
-import org.apache.phoenix.util.MetaDataUtil;
-import org.apache.phoenix.util.SchemaUtil;
-
-import com.google.common.collect.Lists;
-
-public class ViewFinder {
-
-	// The PHYSICAL_TABLE link from view to the base table overwrites the PARENT_TABLE link (when namespace mapping is disabled)
-    static TableViewFinderResult findBaseTable(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
-        throws IOException {
-        return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.PHYSICAL_TABLE,
-            HConstants.LATEST_TIMESTAMP);
-    }
-    
-    static TableViewFinderResult findParentViewofIndex(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table)
-            throws IOException {
-            return findRelatedViews(systemCatalog, tenantId, schema, table, PTable.LinkType.VIEW_INDEX_PARENT_TABLE,
-                HConstants.LATEST_TIMESTAMP);
-        }
-
-    public static void findAllRelatives(Table systemTable, byte[] tenantId, byte[] schema, byte[] table,
-        PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
-        findAllRelatives(systemTable, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
-    }
-
-    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
-        PTable.LinkType linkType, long timestamp, TableViewFinderResult result) throws IOException {
-        TableViewFinderResult currentResult =
-            findRelatedViews(systemCatalog, tenantId, schema, table, linkType, timestamp);
-        result.addResult(currentResult);
-        for (TableInfo viewInfo : currentResult.getLinks()) {
-            findAllRelatives(systemCatalog, viewInfo.getTenantId(), viewInfo.getSchemaName(), viewInfo.getTableName(), linkType, timestamp, result);
-        }
-    }
-
-    /**
-     * Runs a scan on SYSTEM.CATALOG or SYSTEM.CHILD_LINK to get the related tables/views
-     */
-    static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
-        PTable.LinkType linkType, long timestamp) throws IOException {
-        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
-            throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
-        }
-        byte[] key = SchemaUtil.getTableKey(tenantId, schema, table);
-		Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
-        SingleColumnValueFilter linkFilter =
-            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
-                linkType.getSerializedValueAsByteArray());
-        linkFilter.setFilterIfMissing(true);
-        scan.setFilter(linkFilter);
-        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
-        if (linkType==PTable.LinkType.PARENT_TABLE)
-            scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
-        if (linkType==PTable.LinkType.PHYSICAL_TABLE)
-            scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
-        List<TableInfo> tableInfoList = Lists.newArrayList();
-        try (ResultScanner scanner = systemCatalog.getScanner(scan))  {
-            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
-                byte[][] rowKeyMetaData = new byte[5][];
-                byte[] viewTenantId = null;
-                getVarChars(result.getRow(), 5, rowKeyMetaData);
-                if (linkType==PTable.LinkType.PARENT_TABLE) {
-                    viewTenantId = result.getValue(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
-                } else if (linkType==PTable.LinkType.CHILD_TABLE) {
-                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
-                } else if (linkType==PTable.LinkType.VIEW_INDEX_PARENT_TABLE) {
-                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
-                } 
-                else if (linkType==PTable.LinkType.PHYSICAL_TABLE && result.getValue(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES)!=null) {
-                    // do not links from indexes to their physical table
-                    continue;
-                }
-                byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
-                byte[] viewName = SchemaUtil.getTableNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
-                tableInfoList.add(new TableInfo(viewTenantId, viewSchemaName, viewName));
-            }
-            return new TableViewFinderResult(tableInfoList);
-        } 
-    }
-    
-    /**
-     * @return true if the given table has at least one child view
-     * @throws IOException 
-     */
-    public static boolean hasChildViews(Table systemCatalog, byte[] tenantId, byte[] schemaName, byte[] tableName, long timestamp) throws IOException {
-        byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
-        Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
-        SingleColumnValueFilter linkFilter =
-                new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES,
-                        CompareFilter.CompareOp.EQUAL,
-                        LinkType.CHILD_TABLE.getSerializedValueAsByteArray()) {
-                    // if we found a row with the CHILD_TABLE link type we are done and can
-                    // terminate the scan
-                    @Override
-                    public boolean filterAllRemaining() throws IOException {
-                        return matchedColumn;
-                    }
-                };
-        linkFilter.setFilterIfMissing(true);
-        scan.setFilter(linkFilter);
-        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
-        try (ResultScanner scanner = systemCatalog.getScanner(scan)) {
-            Result result = scanner.next();
-            return result!=null; 
-        }
-    }
-
-}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
index 9fc030c..c12ad18 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/WhereConstantParser.java
@@ -49,7 +49,7 @@ import com.google.common.collect.Lists;
 
 public class WhereConstantParser {
 
-    static PTable addViewInfoToPColumnsIfNeeded(PTable view) throws SQLException {
+    public static PTable addViewInfoToPColumnsIfNeeded(PTable view) throws SQLException {
     	boolean[] viewColumnConstantsMatched = new boolean[view.getColumns().size()];
         byte[][] viewColumnConstantsToBe = new byte[view.getColumns().size()][];
         if (view.getViewStatement() == null) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
index d8152a9..ca8b20c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/MetaDataProtos.java
@@ -4146,40 +4146,6 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 6;</code>
      */
     int getClientVersion();
-
-    // optional bool skipAddingParentColumns = 7;
-    /**
-     * <code>optional bool skipAddingParentColumns = 7;</code>
-     */
-    boolean hasSkipAddingParentColumns();
-    /**
-     * <code>optional bool skipAddingParentColumns = 7;</code>
-     */
-    boolean getSkipAddingParentColumns();
-
-    // optional bool skipAddingIndexes = 8;
-    /**
-     * <code>optional bool skipAddingIndexes = 8;</code>
-     */
-    boolean hasSkipAddingIndexes();
-    /**
-     * <code>optional bool skipAddingIndexes = 8;</code>
-     */
-    boolean getSkipAddingIndexes();
-
-    // optional .PTable lockedAncestorTable = 9;
-    /**
-     * <code>optional .PTable lockedAncestorTable = 9;</code>
-     */
-    boolean hasLockedAncestorTable();
-    /**
-     * <code>optional .PTable lockedAncestorTable = 9;</code>
-     */
-    org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable();
-    /**
-     * <code>optional .PTable lockedAncestorTable = 9;</code>
-     */
-    org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder();
   }
   /**
    * Protobuf type {@code GetTableRequest}
@@ -4262,29 +4228,6 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
-            case 56: {
-              bitField0_ |= 0x00000040;
-              skipAddingParentColumns_ = input.readBool();
-              break;
-            }
-            case 64: {
-              bitField0_ |= 0x00000080;
-              skipAddingIndexes_ = input.readBool();
-              break;
-            }
-            case 74: {
-              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder subBuilder = null;
-              if (((bitField0_ & 0x00000100) == 0x00000100)) {
-                subBuilder = lockedAncestorTable_.toBuilder();
-              }
-              lockedAncestorTable_ = input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.PARSER, extensionRegistry);
-              if (subBuilder != null) {
-                subBuilder.mergeFrom(lockedAncestorTable_);
-                lockedAncestorTable_ = subBuilder.buildPartial();
-              }
-              bitField0_ |= 0x00000100;
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4421,60 +4364,6 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
-    // optional bool skipAddingParentColumns = 7;
-    public static final int SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER = 7;
-    private boolean skipAddingParentColumns_;
-    /**
-     * <code>optional bool skipAddingParentColumns = 7;</code>
-     */
-    public boolean hasSkipAddingParentColumns() {
-      return ((bitField0_ & 0x00000040) == 0x00000040);
-    }
-    /**
-     * <code>optional bool skipAddingParentColumns = 7;</code>
-     */
-    public boolean getSkipAddingParentColumns() {
-      return skipAddingParentColumns_;
-    }
-
-    // optional bool skipAddingIndexes = 8;
-    public static final int SKIPADDINGINDEXES_FIELD_NUMBER = 8;
-    private boolean skipAddingIndexes_;
-    /**
-     * <code>optional bool skipAddingIndexes = 8;</code>
-     */
-    public boolean hasSkipAddingIndexes() {
-      return ((bitField0_ & 0x00000080) == 0x00000080);
-    }
-    /**
-     * <code>optional bool skipAddingIndexes = 8;</code>
-     */
-    public boolean getSkipAddingIndexes() {
-      return skipAddingIndexes_;
-    }
-
-    // optional .PTable lockedAncestorTable = 9;
-    public static final int LOCKEDANCESTORTABLE_FIELD_NUMBER = 9;
-    private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable lockedAncestorTable_;
-    /**
-     * <code>optional .PTable lockedAncestorTable = 9;</code>
-     */
-    public boolean hasLockedAncestorTable() {
-      return ((bitField0_ & 0x00000100) == 0x00000100);
-    }
-    /**
-     * <code>optional .PTable lockedAncestorTable = 9;</code>
-     */
-    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable() {
-      return lockedAncestorTable_;
-    }
-    /**
-     * <code>optional .PTable lockedAncestorTable = 9;</code>
-     */
-    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder() {
-      return lockedAncestorTable_;
-    }
-
     private void initFields() {
       tenantId_ = com.google.protobuf.ByteString.EMPTY;
       schemaName_ = com.google.protobuf.ByteString.EMPTY;
@@ -4482,9 +4371,6 @@ public final class MetaDataProtos {
       tableTimestamp_ = 0L;
       clientTimestamp_ = 0L;
       clientVersion_ = 0;
-      skipAddingParentColumns_ = false;
-      skipAddingIndexes_ = false;
-      lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4511,12 +4397,6 @@ public final class MetaDataProtos {
         memoizedIsInitialized = 0;
         return false;
       }
-      if (hasLockedAncestorTable()) {
-        if (!getLockedAncestorTable().isInitialized()) {
-          memoizedIsInitialized = 0;
-          return false;
-        }
-      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -4542,15 +4422,6 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000020) == 0x00000020)) {
         output.writeInt32(6, clientVersion_);
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        output.writeBool(7, skipAddingParentColumns_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        output.writeBool(8, skipAddingIndexes_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        output.writeMessage(9, lockedAncestorTable_);
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -4584,18 +4455,6 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(6, clientVersion_);
       }
-      if (((bitField0_ & 0x00000040) == 0x00000040)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(7, skipAddingParentColumns_);
-      }
-      if (((bitField0_ & 0x00000080) == 0x00000080)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(8, skipAddingIndexes_);
-      }
-      if (((bitField0_ & 0x00000100) == 0x00000100)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeMessageSize(9, lockedAncestorTable_);
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -4649,21 +4508,6 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
-      result = result && (hasSkipAddingParentColumns() == other.hasSkipAddingParentColumns());
-      if (hasSkipAddingParentColumns()) {
-        result = result && (getSkipAddingParentColumns()
-            == other.getSkipAddingParentColumns());
-      }
-      result = result && (hasSkipAddingIndexes() == other.hasSkipAddingIndexes());
-      if (hasSkipAddingIndexes()) {
-        result = result && (getSkipAddingIndexes()
-            == other.getSkipAddingIndexes());
-      }
-      result = result && (hasLockedAncestorTable() == other.hasLockedAncestorTable());
-      if (hasLockedAncestorTable()) {
-        result = result && getLockedAncestorTable()
-            .equals(other.getLockedAncestorTable());
-      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -4701,18 +4545,6 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
-      if (hasSkipAddingParentColumns()) {
-        hash = (37 * hash) + SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getSkipAddingParentColumns());
-      }
-      if (hasSkipAddingIndexes()) {
-        hash = (37 * hash) + SKIPADDINGINDEXES_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getSkipAddingIndexes());
-      }
-      if (hasLockedAncestorTable()) {
-        hash = (37 * hash) + LOCKEDANCESTORTABLE_FIELD_NUMBER;
-        hash = (53 * hash) + getLockedAncestorTable().hashCode();
-      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -4814,7 +4646,6 @@ public final class MetaDataProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
-          getLockedAncestorTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -4835,16 +4666,6 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000010);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000020);
-        skipAddingParentColumns_ = false;
-        bitField0_ = (bitField0_ & ~0x00000040);
-        skipAddingIndexes_ = false;
-        bitField0_ = (bitField0_ & ~0x00000080);
-        if (lockedAncestorTableBuilder_ == null) {
-          lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
-        } else {
-          lockedAncestorTableBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000100);
         return this;
       }
 
@@ -4897,22 +4718,6 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000020;
         }
         result.clientVersion_ = clientVersion_;
-        if (((from_bitField0_ & 0x00000040) == 0x00000040)) {
-          to_bitField0_ |= 0x00000040;
-        }
-        result.skipAddingParentColumns_ = skipAddingParentColumns_;
-        if (((from_bitField0_ & 0x00000080) == 0x00000080)) {
-          to_bitField0_ |= 0x00000080;
-        }
-        result.skipAddingIndexes_ = skipAddingIndexes_;
-        if (((from_bitField0_ & 0x00000100) == 0x00000100)) {
-          to_bitField0_ |= 0x00000100;
-        }
-        if (lockedAncestorTableBuilder_ == null) {
-          result.lockedAncestorTable_ = lockedAncestorTable_;
-        } else {
-          result.lockedAncestorTable_ = lockedAncestorTableBuilder_.build();
-        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -4947,15 +4752,6 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
-        if (other.hasSkipAddingParentColumns()) {
-          setSkipAddingParentColumns(other.getSkipAddingParentColumns());
-        }
-        if (other.hasSkipAddingIndexes()) {
-          setSkipAddingIndexes(other.getSkipAddingIndexes());
-        }
-        if (other.hasLockedAncestorTable()) {
-          mergeLockedAncestorTable(other.getLockedAncestorTable());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4981,12 +4777,6 @@ public final class MetaDataProtos {
           
           return false;
         }
-        if (hasLockedAncestorTable()) {
-          if (!getLockedAncestorTable().isInitialized()) {
-            
-            return false;
-          }
-        }
         return true;
       }
 
@@ -5216,189 +5006,6 @@ public final class MetaDataProtos {
         return this;
       }
 
-      // optional bool skipAddingParentColumns = 7;
-      private boolean skipAddingParentColumns_ ;
-      /**
-       * <code>optional bool skipAddingParentColumns = 7;</code>
-       */
-      public boolean hasSkipAddingParentColumns() {
-        return ((bitField0_ & 0x00000040) == 0x00000040);
-      }
-      /**
-       * <code>optional bool skipAddingParentColumns = 7;</code>
-       */
-      public boolean getSkipAddingParentColumns() {
-        return skipAddingParentColumns_;
-      }
-      /**
-       * <code>optional bool skipAddingParentColumns = 7;</code>
-       */
-      public Builder setSkipAddingParentColumns(boolean value) {
-        bitField0_ |= 0x00000040;
-        skipAddingParentColumns_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool skipAddingParentColumns = 7;</code>
-       */
-      public Builder clearSkipAddingParentColumns() {
-        bitField0_ = (bitField0_ & ~0x00000040);
-        skipAddingParentColumns_ = false;
-        onChanged();
-        return this;
-      }
-
-      // optional bool skipAddingIndexes = 8;
-      private boolean skipAddingIndexes_ ;
-      /**
-       * <code>optional bool skipAddingIndexes = 8;</code>
-       */
-      public boolean hasSkipAddingIndexes() {
-        return ((bitField0_ & 0x00000080) == 0x00000080);
-      }
-      /**
-       * <code>optional bool skipAddingIndexes = 8;</code>
-       */
-      public boolean getSkipAddingIndexes() {
-        return skipAddingIndexes_;
-      }
-      /**
-       * <code>optional bool skipAddingIndexes = 8;</code>
-       */
-      public Builder setSkipAddingIndexes(boolean value) {
-        bitField0_ |= 0x00000080;
-        skipAddingIndexes_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional bool skipAddingIndexes = 8;</code>
-       */
-      public Builder clearSkipAddingIndexes() {
-        bitField0_ = (bitField0_ & ~0x00000080);
-        skipAddingIndexes_ = false;
-        onChanged();
-        return this;
-      }
-
-      // optional .PTable lockedAncestorTable = 9;
-      private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> lockedAncestorTableBuilder_;
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public boolean hasLockedAncestorTable() {
-        return ((bitField0_ & 0x00000100) == 0x00000100);
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getLockedAncestorTable() {
-        if (lockedAncestorTableBuilder_ == null) {
-          return lockedAncestorTable_;
-        } else {
-          return lockedAncestorTableBuilder_.getMessage();
-        }
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public Builder setLockedAncestorTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
-        if (lockedAncestorTableBuilder_ == null) {
-          if (value == null) {
-            throw new NullPointerException();
-          }
-          lockedAncestorTable_ = value;
-          onChanged();
-        } else {
-          lockedAncestorTableBuilder_.setMessage(value);
-        }
-        bitField0_ |= 0x00000100;
-        return this;
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public Builder setLockedAncestorTable(
-          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder builderForValue) {
-        if (lockedAncestorTableBuilder_ == null) {
-          lockedAncestorTable_ = builderForValue.build();
-          onChanged();
-        } else {
-          lockedAncestorTableBuilder_.setMessage(builderForValue.build());
-        }
-        bitField0_ |= 0x00000100;
-        return this;
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public Builder mergeLockedAncestorTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
-        if (lockedAncestorTableBuilder_ == null) {
-          if (((bitField0_ & 0x00000100) == 0x00000100) &&
-              lockedAncestorTable_ != org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance()) {
-            lockedAncestorTable_ =
-              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.newBuilder(lockedAncestorTable_).mergeFrom(value).buildPartial();
-          } else {
-            lockedAncestorTable_ = value;
-          }
-          onChanged();
-        } else {
-          lockedAncestorTableBuilder_.mergeFrom(value);
-        }
-        bitField0_ |= 0x00000100;
-        return this;
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public Builder clearLockedAncestorTable() {
-        if (lockedAncestorTableBuilder_ == null) {
-          lockedAncestorTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
-          onChanged();
-        } else {
-          lockedAncestorTableBuilder_.clear();
-        }
-        bitField0_ = (bitField0_ & ~0x00000100);
-        return this;
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder getLockedAncestorTableBuilder() {
-        bitField0_ |= 0x00000100;
-        onChanged();
-        return getLockedAncestorTableFieldBuilder().getBuilder();
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getLockedAncestorTableOrBuilder() {
-        if (lockedAncestorTableBuilder_ != null) {
-          return lockedAncestorTableBuilder_.getMessageOrBuilder();
-        } else {
-          return lockedAncestorTable_;
-        }
-      }
-      /**
-       * <code>optional .PTable lockedAncestorTable = 9;</code>
-       */
-      private com.google.protobuf.SingleFieldBuilder<
-          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> 
-          getLockedAncestorTableFieldBuilder() {
-        if (lockedAncestorTableBuilder_ == null) {
-          lockedAncestorTableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
-              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder>(
-                  lockedAncestorTable_,
-                  getParentForChildren(),
-                  isClean());
-          lockedAncestorTable_ = null;
-        }
-        return lockedAncestorTableBuilder_;
-      }
-
       // @@protoc_insertion_point(builder_scope:GetTableRequest)
     }
 
@@ -7111,6 +6718,20 @@ public final class MetaDataProtos {
      * <code>optional bool allocateIndexId = 3;</code>
      */
     boolean getAllocateIndexId();
+
+    // optional .PTable parentTable = 4;
+    /**
+     * <code>optional .PTable parentTable = 4;</code>
+     */
+    boolean hasParentTable();
+    /**
+     * <code>optional .PTable parentTable = 4;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable();
+    /**
+     * <code>optional .PTable parentTable = 4;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder();
   }
   /**
    * Protobuf type {@code CreateTableRequest}
@@ -7185,6 +6806,19 @@ public final class MetaDataProtos {
               allocateIndexId_ = input.readBool();
               break;
             }
+            case 34: {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000004) == 0x00000004)) {
+                subBuilder = parentTable_.toBuilder();
+              }
+              parentTable_ = input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(parentTable_);
+                parentTable_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000004;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -7283,16 +6917,45 @@ public final class MetaDataProtos {
       return allocateIndexId_;
     }
 
+    // optional .PTable parentTable = 4;
+    public static final int PARENTTABLE_FIELD_NUMBER = 4;
+    private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable parentTable_;
+    /**
+     * <code>optional .PTable parentTable = 4;</code>
+     */
+    public boolean hasParentTable() {
+      return ((bitField0_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional .PTable parentTable = 4;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable() {
+      return parentTable_;
+    }
+    /**
+     * <code>optional .PTable parentTable = 4;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder() {
+      return parentTable_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       clientVersion_ = 0;
       allocateIndexId_ = false;
+      parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
+      if (hasParentTable()) {
+        if (!getParentTable().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -7309,6 +6972,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000002) == 0x00000002)) {
         output.writeBool(3, allocateIndexId_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        output.writeMessage(4, parentTable_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -7335,6 +7001,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeBoolSize(3, allocateIndexId_);
       }
+      if (((bitField0_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, parentTable_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -7370,6 +7040,11 @@ public final class MetaDataProtos {
         result = result && (getAllocateIndexId()
             == other.getAllocateIndexId());
       }
+      result = result && (hasParentTable() == other.hasParentTable());
+      if (hasParentTable()) {
+        result = result && getParentTable()
+            .equals(other.getParentTable());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -7395,6 +7070,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + ALLOCATEINDEXID_FIELD_NUMBER;
         hash = (53 * hash) + hashBoolean(getAllocateIndexId());
       }
+      if (hasParentTable()) {
+        hash = (37 * hash) + PARENTTABLE_FIELD_NUMBER;
+        hash = (53 * hash) + getParentTable().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -7500,6 +7179,7 @@ public final class MetaDataProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getParentTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -7514,6 +7194,12 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         allocateIndexId_ = false;
         bitField0_ = (bitField0_ & ~0x00000004);
+        if (parentTableBuilder_ == null) {
+          parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+        } else {
+          parentTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -7555,6 +7241,14 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000002;
         }
         result.allocateIndexId_ = allocateIndexId_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000004;
+        }
+        if (parentTableBuilder_ == null) {
+          result.parentTable_ = parentTable_;
+        } else {
+          result.parentTable_ = parentTableBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -7587,11 +7281,20 @@ public final class MetaDataProtos {
         if (other.hasAllocateIndexId()) {
           setAllocateIndexId(other.getAllocateIndexId());
         }
+        if (other.hasParentTable()) {
+          mergeParentTable(other.getParentTable());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
+        if (hasParentTable()) {
+          if (!getParentTable().isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -7752,6 +7455,123 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional .PTable parentTable = 4;
+      private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> parentTableBuilder_;
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public boolean hasParentTable() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable() {
+        if (parentTableBuilder_ == null) {
+          return parentTable_;
+        } else {
+          return parentTableBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public Builder setParentTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (parentTableBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          parentTable_ = value;
+          onChanged();
+        } else {
+          parentTableBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public Builder setParentTable(
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder builderForValue) {
+        if (parentTableBuilder_ == null) {
+          parentTable_ = builderForValue.build();
+          onChanged();
+        } else {
+          parentTableBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public Builder mergeParentTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (parentTableBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              parentTable_ != org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance()) {
+            parentTable_ =
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.newBuilder(parentTable_).mergeFrom(value).buildPartial();
+          } else {
+            parentTable_ = value;
+          }
+          onChanged();
+        } else {
+          parentTableBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public Builder clearParentTable() {
+        if (parentTableBuilder_ == null) {
+          parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+          onChanged();
+        } else {
+          parentTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder getParentTableBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getParentTableFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder() {
+        if (parentTableBuilder_ != null) {
+          return parentTableBuilder_.getMessageOrBuilder();
+        } else {
+          return parentTable_;
+        }
+      }
+      /**
+       * <code>optional .PTable parentTable = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> 
+          getParentTableFieldBuilder() {
+        if (parentTableBuilder_ == null) {
+          parentTableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder>(
+                  parentTable_,
+                  getParentForChildren(),
+                  isClean());
+          parentTable_ = null;
+        }
+        return parentTableBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:CreateTableRequest)
     }
 
@@ -9371,16 +9191,6 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 4;</code>
      */
     int getClientVersion();
-
-    // optional bool skipAddingParentColumns = 5;
-    /**
-     * <code>optional bool skipAddingParentColumns = 5;</code>
-     */
-    boolean hasSkipAddingParentColumns();
-    /**
-     * <code>optional bool skipAddingParentColumns = 5;</code>
-     */
-    boolean getSkipAddingParentColumns();
   }
   /**
    * Protobuf type {@code DropTableRequest}
@@ -9456,11 +9266,6 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
-            case 40: {
-              bitField0_ |= 0x00000008;
-              skipAddingParentColumns_ = input.readBool();
-              break;
-            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -9602,28 +9407,11 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
-    // optional bool skipAddingParentColumns = 5;
-    public static final int SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER = 5;
-    private boolean skipAddingParentColumns_;
-    /**
-     * <code>optional bool skipAddingParentColumns = 5;</code>
-     */
-    public boolean hasSkipAddingParentColumns() {
-      return ((bitField0_ & 0x00000008) == 0x00000008);
-    }
-    /**
-     * <code>optional bool skipAddingParentColumns = 5;</code>
-     */
-    public boolean getSkipAddingParentColumns() {
-      return skipAddingParentColumns_;
-    }
-
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       tableType_ = "";
       cascade_ = false;
       clientVersion_ = 0;
-      skipAddingParentColumns_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -9653,9 +9441,6 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt32(4, clientVersion_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        output.writeBool(5, skipAddingParentColumns_);
-      }
       getUnknownFields().writeTo(output);
     }
 
@@ -9686,10 +9471,6 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(4, clientVersion_);
       }
-      if (((bitField0_ & 0x00000008) == 0x00000008)) {
-        size += com.google.protobuf.CodedOutputStream
-          .computeBoolSize(5, skipAddingParentColumns_);
-      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -9730,11 +9511,6 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
-      result = result && (hasSkipAddingParentColumns() == other.hasSkipAddingParentColumns());
-      if (hasSkipAddingParentColumns()) {
-        result = result && (getSkipAddingParentColumns()
-            == other.getSkipAddingParentColumns());
-      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -9764,10 +9540,6 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
-      if (hasSkipAddingParentColumns()) {
-        hash = (37 * hash) + SKIPADDINGPARENTCOLUMNS_FIELD_NUMBER;
-        hash = (53 * hash) + hashBoolean(getSkipAddingParentColumns());
-      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -9885,8 +9657,6 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000004);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000008);
-        skipAddingParentColumns_ = false;
-        bitField0_ = (bitField0_ & ~0x00000010);
         return this;
       }
 
@@ -9932,10 +9702,6 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000004;
         }
         result.clientVersion_ = clientVersion_;
-        if (((from_bitField0_ & 0x00000010) == 0x00000010)) {
-          to_bitField0_ |= 0x00000008;
-        }
-        result.skipAddingParentColumns_ = skipAddingParentColumns_;
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -9973,9 +9739,6 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
-        if (other.hasSkipAddingParentColumns()) {
-          setSkipAddingParentColumns(other.getSkipAddingParentColumns());
-        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -10191,63 +9954,30 @@ public final class MetaDataProtos {
       /**
        * <code>optional int32 clientVersion = 4;</code>
        */
-      public boolean hasClientVersion() {
-        return ((bitField0_ & 0x00000008) == 0x00000008);
-      }
-      /**
-       * <code>optional int32 clientVersion = 4;</code>
-       */
-      public int getClientVersion() {
-        return clientVersion_;
-      }
-      /**
-       * <code>optional int32 clientVersion = 4;</code>
-       */
-      public Builder setClientVersion(int value) {
-        bitField0_ |= 0x00000008;
-        clientVersion_ = value;
-        onChanged();
-        return this;
-      }
-      /**
-       * <code>optional int32 clientVersion = 4;</code>
-       */
-      public Builder clearClientVersion() {
-        bitField0_ = (bitField0_ & ~0x00000008);
-        clientVersion_ = 0;
-        onChanged();
-        return this;
-      }
-
-      // optional bool skipAddingParentColumns = 5;
-      private boolean skipAddingParentColumns_ ;
-      /**
-       * <code>optional bool skipAddingParentColumns = 5;</code>
-       */
-      public boolean hasSkipAddingParentColumns() {
-        return ((bitField0_ & 0x00000010) == 0x00000010);
+      public boolean hasClientVersion() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
       }
       /**
-       * <code>optional bool skipAddingParentColumns = 5;</code>
+       * <code>optional int32 clientVersion = 4;</code>
        */
-      public boolean getSkipAddingParentColumns() {
-        return skipAddingParentColumns_;
+      public int getClientVersion() {
+        return clientVersion_;
       }
       /**
-       * <code>optional bool skipAddingParentColumns = 5;</code>
+       * <code>optional int32 clientVersion = 4;</code>
        */
-      public Builder setSkipAddingParentColumns(boolean value) {
-        bitField0_ |= 0x00000010;
-        skipAddingParentColumns_ = value;
+      public Builder setClientVersion(int value) {
+        bitField0_ |= 0x00000008;
+        clientVersion_ = value;
         onChanged();
         return this;
       }
       /**
-       * <code>optional bool skipAddingParentColumns = 5;</code>
+       * <code>optional int32 clientVersion = 4;</code>
        */
-      public Builder clearSkipAddingParentColumns() {
-        bitField0_ = (bitField0_ & ~0x00000010);
-        skipAddingParentColumns_ = false;
+      public Builder clearClientVersion() {
+        bitField0_ = (bitField0_ & ~0x00000008);
+        clientVersion_ = 0;
         onChanged();
         return this;
       }
@@ -11060,6 +10790,20 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 2;</code>
      */
     int getClientVersion();
+
+    // optional .PTable parentTable = 3;
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    boolean hasParentTable();
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable();
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder();
   }
   /**
    * Protobuf type {@code AddColumnRequest}
@@ -11125,6 +10869,19 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 26: {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = parentTable_.toBuilder();
+              }
+              parentTable_ = input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(parentTable_);
+                parentTable_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -11207,15 +10964,44 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional .PTable parentTable = 3;
+    public static final int PARENTTABLE_FIELD_NUMBER = 3;
+    private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable parentTable_;
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    public boolean hasParentTable() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable() {
+      return parentTable_;
+    }
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder() {
+      return parentTable_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       clientVersion_ = 0;
+      parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
+      if (hasParentTable()) {
+        if (!getParentTable().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -11229,6 +11015,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(2, clientVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(3, parentTable_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -11251,6 +11040,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, clientVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, parentTable_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -11281,6 +11074,11 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasParentTable() == other.hasParentTable());
+      if (hasParentTable()) {
+        result = result && getParentTable()
+            .equals(other.getParentTable());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -11302,6 +11100,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasParentTable()) {
+        hash = (37 * hash) + PARENTTABLE_FIELD_NUMBER;
+        hash = (53 * hash) + getParentTable().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -11403,6 +11205,7 @@ public final class MetaDataProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getParentTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -11415,6 +11218,12 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
+        if (parentTableBuilder_ == null) {
+          parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+        } else {
+          parentTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -11452,6 +11261,14 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (parentTableBuilder_ == null) {
+          result.parentTable_ = parentTable_;
+        } else {
+          result.parentTable_ = parentTableBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -11481,11 +11298,20 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasParentTable()) {
+          mergeParentTable(other.getParentTable());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
+        if (hasParentTable()) {
+          if (!getParentTable().isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -11613,6 +11439,123 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional .PTable parentTable = 3;
+      private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> parentTableBuilder_;
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public boolean hasParentTable() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable() {
+        if (parentTableBuilder_ == null) {
+          return parentTable_;
+        } else {
+          return parentTableBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder setParentTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (parentTableBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          parentTable_ = value;
+          onChanged();
+        } else {
+          parentTableBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder setParentTable(
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder builderForValue) {
+        if (parentTableBuilder_ == null) {
+          parentTable_ = builderForValue.build();
+          onChanged();
+        } else {
+          parentTableBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder mergeParentTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (parentTableBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              parentTable_ != org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance()) {
+            parentTable_ =
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.newBuilder(parentTable_).mergeFrom(value).buildPartial();
+          } else {
+            parentTable_ = value;
+          }
+          onChanged();
+        } else {
+          parentTableBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder clearParentTable() {
+        if (parentTableBuilder_ == null) {
+          parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+          onChanged();
+        } else {
+          parentTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder getParentTableBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getParentTableFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder() {
+        if (parentTableBuilder_ != null) {
+          return parentTableBuilder_.getMessageOrBuilder();
+        } else {
+          return parentTable_;
+        }
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> 
+          getParentTableFieldBuilder() {
+        if (parentTableBuilder_ == null) {
+          parentTableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder>(
+                  parentTable_,
+                  getParentForChildren(),
+                  isClean());
+          parentTable_ = null;
+        }
+        return parentTableBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:AddColumnRequest)
     }
 
@@ -11650,6 +11593,20 @@ public final class MetaDataProtos {
      * <code>optional int32 clientVersion = 2;</code>
      */
     int getClientVersion();
+
+    // optional .PTable parentTable = 3;
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    boolean hasParentTable();
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable();
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder();
   }
   /**
    * Protobuf type {@code DropColumnRequest}
@@ -11715,6 +11672,19 @@ public final class MetaDataProtos {
               clientVersion_ = input.readInt32();
               break;
             }
+            case 26: {
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000002) == 0x00000002)) {
+                subBuilder = parentTable_.toBuilder();
+              }
+              parentTable_ = input.readMessage(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(parentTable_);
+                parentTable_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000002;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -11797,15 +11767,44 @@ public final class MetaDataProtos {
       return clientVersion_;
     }
 
+    // optional .PTable parentTable = 3;
+    public static final int PARENTTABLE_FIELD_NUMBER = 3;
+    private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable parentTable_;
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    public boolean hasParentTable() {
+      return ((bitField0_ & 0x00000002) == 0x00000002);
+    }
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable() {
+      return parentTable_;
+    }
+    /**
+     * <code>optional .PTable parentTable = 3;</code>
+     */
+    public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder() {
+      return parentTable_;
+    }
+
     private void initFields() {
       tableMetadataMutations_ = java.util.Collections.emptyList();
       clientVersion_ = 0;
+      parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
       byte isInitialized = memoizedIsInitialized;
       if (isInitialized != -1) return isInitialized == 1;
 
+      if (hasParentTable()) {
+        if (!getParentTable().isInitialized()) {
+          memoizedIsInitialized = 0;
+          return false;
+        }
+      }
       memoizedIsInitialized = 1;
       return true;
     }
@@ -11819,6 +11818,9 @@ public final class MetaDataProtos {
       if (((bitField0_ & 0x00000001) == 0x00000001)) {
         output.writeInt32(2, clientVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        output.writeMessage(3, parentTable_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -11841,6 +11843,10 @@ public final class MetaDataProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(2, clientVersion_);
       }
+      if (((bitField0_ & 0x00000002) == 0x00000002)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(3, parentTable_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -11871,6 +11877,11 @@ public final class MetaDataProtos {
         result = result && (getClientVersion()
             == other.getClientVersion());
       }
+      result = result && (hasParentTable() == other.hasParentTable());
+      if (hasParentTable()) {
+        result = result && getParentTable()
+            .equals(other.getParentTable());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -11892,6 +11903,10 @@ public final class MetaDataProtos {
         hash = (37 * hash) + CLIENTVERSION_FIELD_NUMBER;
         hash = (53 * hash) + getClientVersion();
       }
+      if (hasParentTable()) {
+        hash = (37 * hash) + PARENTTABLE_FIELD_NUMBER;
+        hash = (53 * hash) + getParentTable().hashCode();
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -11993,6 +12008,7 @@ public final class MetaDataProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getParentTableFieldBuilder();
         }
       }
       private static Builder create() {
@@ -12005,6 +12021,12 @@ public final class MetaDataProtos {
         bitField0_ = (bitField0_ & ~0x00000001);
         clientVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000002);
+        if (parentTableBuilder_ == null) {
+          parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+        } else {
+          parentTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
         return this;
       }
 
@@ -12042,6 +12064,14 @@ public final class MetaDataProtos {
           to_bitField0_ |= 0x00000001;
         }
         result.clientVersion_ = clientVersion_;
+        if (((from_bitField0_ & 0x00000004) == 0x00000004)) {
+          to_bitField0_ |= 0x00000002;
+        }
+        if (parentTableBuilder_ == null) {
+          result.parentTable_ = parentTable_;
+        } else {
+          result.parentTable_ = parentTableBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -12071,11 +12101,20 @@ public final class MetaDataProtos {
         if (other.hasClientVersion()) {
           setClientVersion(other.getClientVersion());
         }
+        if (other.hasParentTable()) {
+          mergeParentTable(other.getParentTable());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
 
       public final boolean isInitialized() {
+        if (hasParentTable()) {
+          if (!getParentTable().isInitialized()) {
+            
+            return false;
+          }
+        }
         return true;
       }
 
@@ -12203,6 +12242,123 @@ public final class MetaDataProtos {
         return this;
       }
 
+      // optional .PTable parentTable = 3;
+      private org.apache.phoenix.coprocessor.generated.PTableProtos.PTable parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> parentTableBuilder_;
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public boolean hasParentTable() {
+        return ((bitField0_ & 0x00000004) == 0x00000004);
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable getParentTable() {
+        if (parentTableBuilder_ == null) {
+          return parentTable_;
+        } else {
+          return parentTableBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder setParentTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (parentTableBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          parentTable_ = value;
+          onChanged();
+        } else {
+          parentTableBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder setParentTable(
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder builderForValue) {
+        if (parentTableBuilder_ == null) {
+          parentTable_ = builderForValue.build();
+          onChanged();
+        } else {
+          parentTableBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder mergeParentTable(org.apache.phoenix.coprocessor.generated.PTableProtos.PTable value) {
+        if (parentTableBuilder_ == null) {
+          if (((bitField0_ & 0x00000004) == 0x00000004) &&
+              parentTable_ != org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance()) {
+            parentTable_ =
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.newBuilder(parentTable_).mergeFrom(value).buildPartial();
+          } else {
+            parentTable_ = value;
+          }
+          onChanged();
+        } else {
+          parentTableBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000004;
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public Builder clearParentTable() {
+        if (parentTableBuilder_ == null) {
+          parentTable_ = org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.getDefaultInstance();
+          onChanged();
+        } else {
+          parentTableBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000004);
+        return this;
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder getParentTableBuilder() {
+        bitField0_ |= 0x00000004;
+        onChanged();
+        return getParentTableFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      public org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder getParentTableOrBuilder() {
+        if (parentTableBuilder_ != null) {
+          return parentTableBuilder_.getMessageOrBuilder();
+        } else {
+          return parentTable_;
+        }
+      }
+      /**
+       * <code>optional .PTable parentTable = 3;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder> 
+          getParentTableFieldBuilder() {
+        if (parentTableBuilder_ == null) {
+          parentTableBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.phoenix.coprocessor.generated.PTableProtos.PTable, org.apache.phoenix.coprocessor.generated.PTableProtos.PTable.Builder, org.apache.phoenix.coprocessor.generated.PTableProtos.PTableOrBuilder>(
+                  parentTable_,
+                  getParentForChildren(),
+                  isClean());
+          parentTable_ = null;
+        }
+        return parentTableBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:DropColumnRequest)
     }
 
@@ -17841,91 +17997,90 @@ public final class MetaDataProtos {
       "ToDelete\030\t \003(\0132\021.SharedTableState\022\030\n\006sch" +
       "ema\030\n \001(\0132\010.PSchema\022\030\n\020autoPartitionNum\030" +
       "\013 \001(\003\022\023\n\013viewIndexId\030\014 \001(\003\022\032\n\017viewIndexI" +
-      "dType\030\r \001(\005:\0015\"\364\001\n\017GetTableRequest\022\020\n\010te" +
+      "dType\030\r \001(\005:\0015\"\222\001\n\017GetTableRequest\022\020\n\010te" +
       "nantId\030\001 \002(\014\022\022\n\nschemaName\030\002 \002(\014\022\021\n\ttabl" +
       "eName\030\003 \002(\014\022\026\n\016tableTimestamp\030\004 \002(\003\022\027\n\017c" +
       "lientTimestamp\030\005 \002(\003\022\025\n\rclientVersion\030\006 " +
-      "\001(\005\022\037\n\027skipAddingParentColumns\030\007 \001(\010\022\031\n\021",
-      "skipAddingIndexes\030\010 \001(\010\022$\n\023lockedAncesto" +
-      "rTable\030\t \001(\0132\007.PTable\"\212\001\n\023GetFunctionsRe" +
-      "quest\022\020\n\010tenantId\030\001 \002(\014\022\025\n\rfunctionNames" +
-      "\030\002 \003(\014\022\032\n\022functionTimestamps\030\003 \003(\003\022\027\n\017cl" +
-      "ientTimestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001" +
-      "(\005\"V\n\020GetSchemaRequest\022\022\n\nschemaName\030\001 \002" +
-      "(\t\022\027\n\017clientTimestamp\030\002 \002(\003\022\025\n\rclientVer" +
-      "sion\030\003 \002(\005\"d\n\022CreateTableRequest\022\036\n\026tabl" +
-      "eMetadataMutations\030\001 \003(\014\022\025\n\rclientVersio" +
-      "n\030\002 \001(\005\022\027\n\017allocateIndexId\030\003 \001(\010\"r\n\025Crea",
-      "teFunctionRequest\022\036\n\026tableMetadataMutati" +
-      "ons\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(\010\022\017\n\007replace\030" +
-      "\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\"`\n\023CreateSc" +
-      "hemaRequest\022\036\n\026tableMetadataMutations\030\001 " +
-      "\003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersion" +
-      "\030\003 \002(\005\"\216\001\n\020DropTableRequest\022\036\n\026tableMeta" +
-      "dataMutations\030\001 \003(\014\022\021\n\ttableType\030\002 \002(\t\022\017" +
-      "\n\007cascade\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(\005\022\037" +
-      "\n\027skipAddingParentColumns\030\005 \001(\010\"_\n\021DropS" +
-      "chemaRequest\022\037\n\027schemaMetadataMutations\030",
-      "\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\rclientVersi" +
-      "on\030\003 \002(\005\"I\n\020AddColumnRequest\022\036\n\026tableMet" +
-      "adataMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 " +
-      "\001(\005\"J\n\021DropColumnRequest\022\036\n\026tableMetadat" +
-      "aMutations\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\"" +
-      "^\n\023DropFunctionRequest\022\036\n\026tableMetadataM" +
-      "utations\030\001 \003(\014\022\020\n\010ifExists\030\002 \001(\010\022\025\n\rclie" +
-      "ntVersion\030\003 \001(\005\"P\n\027UpdateIndexStateReque" +
-      "st\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rcl" +
-      "ientVersion\030\002 \001(\005\"*\n\021ClearCacheRequest\022\025",
-      "\n\rclientVersion\030\001 \001(\005\"*\n\022ClearCacheRespo" +
-      "nse\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n\021GetVersionR" +
-      "equest\022\025\n\rclientVersion\030\001 \001(\005\"E\n\022GetVers" +
-      "ionResponse\022\017\n\007version\030\001 \002(\003\022\036\n\026systemCa" +
-      "talogTimestamp\030\002 \001(\003\"\205\001\n\032ClearTableFromC" +
-      "acheRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschemaN" +
-      "ame\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\027\n\017clientTi" +
-      "mestamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"\035\n\033" +
-      "ClearTableFromCacheResponse*\271\005\n\014Mutation" +
-      "Code\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_",
-      "NOT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COL" +
-      "UMN_ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE" +
-      "_MUTATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021" +
-      "NEWER_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_M" +
-      "UTATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARENT_T" +
-      "ABLE_NOT_FOUND\020\t\022\033\n\027FUNCTION_ALREADY_EXI" +
-      "STS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020\013\022\030\n\024NEWER_F" +
-      "UNCTION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_REGIO" +
-      "N\020\r\022\031\n\025SCHEMA_ALREADY_EXISTS\020\016\022\026\n\022NEWER_" +
-      "SCHEMA_FOUND\020\017\022\024\n\020SCHEMA_NOT_FOUND\020\020\022\030\n\024",
-      "SCHEMA_NOT_IN_REGION\020\021\022\032\n\026TABLES_EXIST_O" +
-      "N_SCHEMA\020\022\022\035\n\031UNALLOWED_SCHEMA_MUTATION\020" +
-      "\023\022%\n!AUTO_PARTITION_SEQUENCE_NOT_FOUND\020\024" +
-      "\022#\n\037CANNOT_COERCE_AUTO_PARTITION_ID\020\025\022\024\n" +
-      "\020TOO_MANY_INDEXES\020\026\022\037\n\033UNABLE_TO_CREATE_" +
-      "CHILD_LINK\020\027\022!\n\035UNABLE_TO_UPDATE_PARENT_" +
-      "TABLE\020\0302\345\006\n\017MetaDataService\022/\n\010getTable\022" +
-      "\020.GetTableRequest\032\021.MetaDataResponse\0227\n\014" +
-      "getFunctions\022\024.GetFunctionsRequest\032\021.Met" +
-      "aDataResponse\0221\n\tgetSchema\022\021.GetSchemaRe",
-      "quest\032\021.MetaDataResponse\0225\n\013createTable\022" +
-      "\023.CreateTableRequest\032\021.MetaDataResponse\022" +
-      ";\n\016createFunction\022\026.CreateFunctionReques" +
-      "t\032\021.MetaDataResponse\0227\n\014createSchema\022\024.C" +
-      "reateSchemaRequest\032\021.MetaDataResponse\0221\n" +
-      "\tdropTable\022\021.DropTableRequest\032\021.MetaData" +
-      "Response\0223\n\ndropSchema\022\022.DropSchemaReque" +
-      "st\032\021.MetaDataResponse\0227\n\014dropFunction\022\024." +
-      "DropFunctionRequest\032\021.MetaDataResponse\0221" +
-      "\n\taddColumn\022\021.AddColumnRequest\032\021.MetaDat",
-      "aResponse\0223\n\ndropColumn\022\022.DropColumnRequ" +
-      "est\032\021.MetaDataResponse\022?\n\020updateIndexSta" +
-      "te\022\030.UpdateIndexStateRequest\032\021.MetaDataR" +
-      "esponse\0225\n\nclearCache\022\022.ClearCacheReques" +
-      "t\032\023.ClearCacheResponse\0225\n\ngetVersion\022\022.G" +
-      "etVersionRequest\032\023.GetVersionResponse\022P\n" +
-      "\023clearTableFromCache\022\033.ClearTableFromCac" +
-      "heRequest\032\034.ClearTableFromCacheResponseB" +
-      "B\n(org.apache.phoenix.coprocessor.genera" +
-      "tedB\016MetaDataProtosH\001\210\001\001\240\001\001"
+      "\001(\005\"\212\001\n\023GetFunctionsRequest\022\020\n\010tenantId\030",
+      "\001 \002(\014\022\025\n\rfunctionNames\030\002 \003(\014\022\032\n\022function" +
+      "Timestamps\030\003 \003(\003\022\027\n\017clientTimestamp\030\004 \002(" +
+      "\003\022\025\n\rclientVersion\030\005 \001(\005\"V\n\020GetSchemaReq" +
+      "uest\022\022\n\nschemaName\030\001 \002(\t\022\027\n\017clientTimest" +
+      "amp\030\002 \002(\003\022\025\n\rclientVersion\030\003 \002(\005\"\202\001\n\022Cre" +
+      "ateTableRequest\022\036\n\026tableMetadataMutation" +
+      "s\030\001 \003(\014\022\025\n\rclientVersion\030\002 \001(\005\022\027\n\017alloca" +
+      "teIndexId\030\003 \001(\010\022\034\n\013parentTable\030\004 \001(\0132\007.P" +
+      "Table\"r\n\025CreateFunctionRequest\022\036\n\026tableM" +
+      "etadataMutations\030\001 \003(\014\022\021\n\ttemporary\030\002 \002(",
+      "\010\022\017\n\007replace\030\003 \001(\010\022\025\n\rclientVersion\030\004 \001(" +
+      "\005\"`\n\023CreateSchemaRequest\022\036\n\026tableMetadat" +
+      "aMutations\030\001 \003(\014\022\022\n\nschemaName\030\002 \002(\t\022\025\n\r" +
+      "clientVersion\030\003 \002(\005\"m\n\020DropTableRequest\022" +
+      "\036\n\026tableMetadataMutations\030\001 \003(\014\022\021\n\ttable" +
+      "Type\030\002 \002(\t\022\017\n\007cascade\030\003 \001(\010\022\025\n\rclientVer" +
+      "sion\030\004 \001(\005\"_\n\021DropSchemaRequest\022\037\n\027schem" +
+      "aMetadataMutations\030\001 \003(\014\022\022\n\nschemaName\030\002" +
+      " \002(\t\022\025\n\rclientVersion\030\003 \002(\005\"g\n\020AddColumn" +
+      "Request\022\036\n\026tableMetadataMutations\030\001 \003(\014\022",
+      "\025\n\rclientVersion\030\002 \001(\005\022\034\n\013parentTable\030\003 " +
+      "\001(\0132\007.PTable\"h\n\021DropColumnRequest\022\036\n\026tab" +
+      "leMetadataMutations\030\001 \003(\014\022\025\n\rclientVersi" +
+      "on\030\002 \001(\005\022\034\n\013parentTable\030\003 \001(\0132\007.PTable\"^" +
+      "\n\023DropFunctionRequest\022\036\n\026tableMetadataMu" +
+      "tations\030\001 \003(\014\022\020\n\010ifExists\030\002 \001(\010\022\025\n\rclien" +
+      "tVersion\030\003 \001(\005\"P\n\027UpdateIndexStateReques" +
+      "t\022\036\n\026tableMetadataMutations\030\001 \003(\014\022\025\n\rcli" +
+      "entVersion\030\002 \001(\005\"*\n\021ClearCacheRequest\022\025\n" +
+      "\rclientVersion\030\001 \001(\005\"*\n\022ClearCacheRespon",
+      "se\022\024\n\014unfreedBytes\030\001 \001(\003\"*\n\021GetVersionRe" +
+      "quest\022\025\n\rclientVersion\030\001 \001(\005\"E\n\022GetVersi" +
+      "onResponse\022\017\n\007version\030\001 \002(\003\022\036\n\026systemCat" +
+      "alogTimestamp\030\002 \001(\003\"\205\001\n\032ClearTableFromCa" +
+      "cheRequest\022\020\n\010tenantId\030\001 \002(\014\022\022\n\nschemaNa" +
+      "me\030\002 \002(\014\022\021\n\ttableName\030\003 \002(\014\022\027\n\017clientTim" +
+      "estamp\030\004 \002(\003\022\025\n\rclientVersion\030\005 \001(\005\"\035\n\033C" +
+      "learTableFromCacheResponse*\271\005\n\014MutationC" +
+      "ode\022\030\n\024TABLE_ALREADY_EXISTS\020\000\022\023\n\017TABLE_N" +
+      "OT_FOUND\020\001\022\024\n\020COLUMN_NOT_FOUND\020\002\022\031\n\025COLU",
+      "MN_ALREADY_EXISTS\020\003\022\035\n\031CONCURRENT_TABLE_" +
+      "MUTATION\020\004\022\027\n\023TABLE_NOT_IN_REGION\020\005\022\025\n\021N" +
+      "EWER_TABLE_FOUND\020\006\022\034\n\030UNALLOWED_TABLE_MU" +
+      "TATION\020\007\022\021\n\rNO_PK_COLUMNS\020\010\022\032\n\026PARENT_TA" +
+      "BLE_NOT_FOUND\020\t\022\033\n\027FUNCTION_ALREADY_EXIS" +
+      "TS\020\n\022\026\n\022FUNCTION_NOT_FOUND\020\013\022\030\n\024NEWER_FU" +
+      "NCTION_FOUND\020\014\022\032\n\026FUNCTION_NOT_IN_REGION" +
+      "\020\r\022\031\n\025SCHEMA_ALREADY_EXISTS\020\016\022\026\n\022NEWER_S" +
+      "CHEMA_FOUND\020\017\022\024\n\020SCHEMA_NOT_FOUND\020\020\022\030\n\024S" +
+      "CHEMA_NOT_IN_REGION\020\021\022\032\n\026TABLES_EXIST_ON",
+      "_SCHEMA\020\022\022\035\n\031UNALLOWED_SCHEMA_MUTATION\020\023" +
+      "\022%\n!AUTO_PARTITION_SEQUENCE_NOT_FOUND\020\024\022" +
+      "#\n\037CANNOT_COERCE_AUTO_PARTITION_ID\020\025\022\024\n\020" +
+      "TOO_MANY_INDEXES\020\026\022\037\n\033UNABLE_TO_CREATE_C" +
+      "HILD_LINK\020\027\022!\n\035UNABLE_TO_UPDATE_PARENT_T" +
+      "ABLE\020\0302\345\006\n\017MetaDataService\022/\n\010getTable\022\020" +
+      ".GetTableRequest\032\021.MetaDataResponse\0227\n\014g" +
+      "etFunctions\022\024.GetFunctionsRequest\032\021.Meta" +
+      "DataResponse\0221\n\tgetSchema\022\021.GetSchemaReq" +
+      "uest\032\021.MetaDataResponse\0225\n\013createTable\022\023",
+      ".CreateTableRequest\032\021.MetaDataResponse\022;" +
+      "\n\016createFunction\022\026.CreateFunctionRequest" +
+      "\032\021.MetaDataResponse\0227\n\014createSchema\022\024.Cr" +
+      "eateSchemaRequest\032\021.MetaDataResponse\0221\n\t" +
+      "dropTable\022\021.DropTableRequest\032\021.MetaDataR" +
+      "esponse\0223\n\ndropSchema\022\022.DropSchemaReques" +
+      "t\032\021.MetaDataResponse\0227\n\014dropFunction\022\024.D" +
+      "ropFunctionRequest\032\021.MetaDataResponse\0221\n" +
+      "\taddColumn\022\021.AddColumnRequest\032\021.MetaData" +
+      "Response\0223\n\ndropColumn\022\022.DropColumnReque",
+      "st\032\021.MetaDataResponse\022?\n\020updateIndexStat" +
+      "e\022\030.UpdateIndexStateRequest\032\021.MetaDataRe" +
+      "sponse\0225\n\nclearCache\022\022.ClearCacheRequest" +
+      "\032\023.ClearCacheResponse\0225\n\ngetVersion\022\022.Ge" +
+      "tVersionRequest\032\023.GetVersionResponse\022P\n\023" +
+      "clearTableFromCache\022\033.ClearTableFromCach" +
+      "eRequest\032\034.ClearTableFromCacheResponseBB" +
+      "\n(org.apache.phoenix.coprocessor.generat" +
+      "edB\016MetaDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -17949,7 +18104,7 @@ public final class MetaDataProtos {
           internal_static_GetTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_GetTableRequest_descriptor,
-              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", "SkipAddingParentColumns", "SkipAddingIndexes", "LockedAncestorTable", });
+              new java.lang.String[] { "TenantId", "SchemaName", "TableName", "TableTimestamp", "ClientTimestamp", "ClientVersion", });
           internal_static_GetFunctionsRequest_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_GetFunctionsRequest_fieldAccessorTable = new
@@ -17967,7 +18122,7 @@ public final class MetaDataProtos {
           internal_static_CreateTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_CreateTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", "AllocateIndexId", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", "AllocateIndexId", "ParentTable", });
           internal_static_CreateFunctionRequest_descriptor =
             getDescriptor().getMessageTypes().get(6);
           internal_static_CreateFunctionRequest_fieldAccessorTable = new
@@ -17985,7 +18140,7 @@ public final class MetaDataProtos {
           internal_static_DropTableRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropTableRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", "SkipAddingParentColumns", });
+              new java.lang.String[] { "TableMetadataMutations", "TableType", "Cascade", "ClientVersion", });
           internal_static_DropSchemaRequest_descriptor =
             getDescriptor().getMessageTypes().get(9);
           internal_static_DropSchemaRequest_fieldAccessorTable = new
@@ -17997,13 +18152,13 @@ public final class MetaDataProtos {
           internal_static_AddColumnRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_AddColumnRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", "ParentTable", });
           internal_static_DropColumnRequest_descriptor =
             getDescriptor().getMessageTypes().get(11);
           internal_static_DropColumnRequest_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_DropColumnRequest_descriptor,
-              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", });
+              new java.lang.String[] { "TableMetadataMutations", "ClientVersion", "ParentTable", });
           internal_static_DropFunctionRequest_descriptor =
             getDescriptor().getMessageTypes().get(12);
           internal_static_DropFunctionRequest_fieldAccessorTable = new
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
index 4a3b43f..6ffedeb 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/PTableProtos.java
@@ -3704,6 +3704,26 @@ public final class PTableProtos {
      * <code>optional int32 viewIndexIdType = 39 [default = 5];</code>
      */
     int getViewIndexIdType();
+
+    // optional bool viewModifiedUpdateCacheFrequency = 40;
+    /**
+     * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+     */
+    boolean hasViewModifiedUpdateCacheFrequency();
+    /**
+     * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+     */
+    boolean getViewModifiedUpdateCacheFrequency();
+
+    // optional bool viewModifiedUseStatsForParallelization = 41;
+    /**
+     * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+     */
+    boolean hasViewModifiedUseStatsForParallelization();
+    /**
+     * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+     */
+    boolean getViewModifiedUseStatsForParallelization();
   }
   /**
    * Protobuf type {@code PTable}
@@ -3965,6 +3985,16 @@ public final class PTableProtos {
               viewIndexIdType_ = input.readInt32();
               break;
             }
+            case 320: {
+              bitField1_ |= 0x00000004;
+              viewModifiedUpdateCacheFrequency_ = input.readBool();
+              break;
+            }
+            case 328: {
+              bitField1_ |= 0x00000008;
+              viewModifiedUseStatsForParallelization_ = input.readBool();
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -4761,6 +4791,38 @@ public final class PTableProtos {
       return viewIndexIdType_;
     }
 
+    // optional bool viewModifiedUpdateCacheFrequency = 40;
+    public static final int VIEWMODIFIEDUPDATECACHEFREQUENCY_FIELD_NUMBER = 40;
+    private boolean viewModifiedUpdateCacheFrequency_;
+    /**
+     * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+     */
+    public boolean hasViewModifiedUpdateCacheFrequency() {
+      return ((bitField1_ & 0x00000004) == 0x00000004);
+    }
+    /**
+     * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+     */
+    public boolean getViewModifiedUpdateCacheFrequency() {
+      return viewModifiedUpdateCacheFrequency_;
+    }
+
+    // optional bool viewModifiedUseStatsForParallelization = 41;
+    public static final int VIEWMODIFIEDUSESTATSFORPARALLELIZATION_FIELD_NUMBER = 41;
+    private boolean viewModifiedUseStatsForParallelization_;
+    /**
+     * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+     */
+    public boolean hasViewModifiedUseStatsForParallelization() {
+      return ((bitField1_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+     */
+    public boolean getViewModifiedUseStatsForParallelization() {
+      return viewModifiedUseStatsForParallelization_;
+    }
+
     private void initFields() {
       schemaNameBytes_ = com.google.protobuf.ByteString.EMPTY;
       tableNameBytes_ = com.google.protobuf.ByteString.EMPTY;
@@ -4800,6 +4862,8 @@ public final class PTableProtos {
       useStatsForParallelization_ = false;
       transactionProvider_ = 0;
       viewIndexIdType_ = 5;
+      viewModifiedUpdateCacheFrequency_ = false;
+      viewModifiedUseStatsForParallelization_ = false;
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -4981,6 +5045,12 @@ public final class PTableProtos {
       if (((bitField1_ & 0x00000002) == 0x00000002)) {
         output.writeInt32(39, viewIndexIdType_);
       }
+      if (((bitField1_ & 0x00000004) == 0x00000004)) {
+        output.writeBool(40, viewModifiedUpdateCacheFrequency_);
+      }
+      if (((bitField1_ & 0x00000008) == 0x00000008)) {
+        output.writeBool(41, viewModifiedUseStatsForParallelization_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -5147,6 +5217,14 @@ public final class PTableProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(39, viewIndexIdType_);
       }
+      if (((bitField1_ & 0x00000004) == 0x00000004)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(40, viewModifiedUpdateCacheFrequency_);
+      }
+      if (((bitField1_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBoolSize(41, viewModifiedUseStatsForParallelization_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -5348,6 +5426,16 @@ public final class PTableProtos {
         result = result && (getViewIndexIdType()
             == other.getViewIndexIdType());
       }
+      result = result && (hasViewModifiedUpdateCacheFrequency() == other.hasViewModifiedUpdateCacheFrequency());
+      if (hasViewModifiedUpdateCacheFrequency()) {
+        result = result && (getViewModifiedUpdateCacheFrequency()
+            == other.getViewModifiedUpdateCacheFrequency());
+      }
+      result = result && (hasViewModifiedUseStatsForParallelization() == other.hasViewModifiedUseStatsForParallelization());
+      if (hasViewModifiedUseStatsForParallelization()) {
+        result = result && (getViewModifiedUseStatsForParallelization()
+            == other.getViewModifiedUseStatsForParallelization());
+      }
       result = result &&
           getUnknownFields().equals(other.getUnknownFields());
       return result;
@@ -5513,6 +5601,14 @@ public final class PTableProtos {
         hash = (37 * hash) + VIEWINDEXIDTYPE_FIELD_NUMBER;
         hash = (53 * hash) + getViewIndexIdType();
       }
+      if (hasViewModifiedUpdateCacheFrequency()) {
+        hash = (37 * hash) + VIEWMODIFIEDUPDATECACHEFREQUENCY_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getViewModifiedUpdateCacheFrequency());
+      }
+      if (hasViewModifiedUseStatsForParallelization()) {
+        hash = (37 * hash) + VIEWMODIFIEDUSESTATSFORPARALLELIZATION_FIELD_NUMBER;
+        hash = (53 * hash) + hashBoolean(getViewModifiedUseStatsForParallelization());
+      }
       hash = (29 * hash) + getUnknownFields().hashCode();
       memoizedHashCode = hash;
       return hash;
@@ -5713,6 +5809,10 @@ public final class PTableProtos {
         bitField1_ = (bitField1_ & ~0x00000010);
         viewIndexIdType_ = 5;
         bitField1_ = (bitField1_ & ~0x00000020);
+        viewModifiedUpdateCacheFrequency_ = false;
+        bitField1_ = (bitField1_ & ~0x00000040);
+        viewModifiedUseStatsForParallelization_ = false;
+        bitField1_ = (bitField1_ & ~0x00000080);
         return this;
       }
 
@@ -5911,6 +6011,14 @@ public final class PTableProtos {
           to_bitField1_ |= 0x00000002;
         }
         result.viewIndexIdType_ = viewIndexIdType_;
+        if (((from_bitField1_ & 0x00000040) == 0x00000040)) {
+          to_bitField1_ |= 0x00000004;
+        }
+        result.viewModifiedUpdateCacheFrequency_ = viewModifiedUpdateCacheFrequency_;
+        if (((from_bitField1_ & 0x00000080) == 0x00000080)) {
+          to_bitField1_ |= 0x00000008;
+        }
+        result.viewModifiedUseStatsForParallelization_ = viewModifiedUseStatsForParallelization_;
         result.bitField0_ = to_bitField0_;
         result.bitField1_ = to_bitField1_;
         onBuilt();
@@ -6122,6 +6230,12 @@ public final class PTableProtos {
         if (other.hasViewIndexIdType()) {
           setViewIndexIdType(other.getViewIndexIdType());
         }
+        if (other.hasViewModifiedUpdateCacheFrequency()) {
+          setViewModifiedUpdateCacheFrequency(other.getViewModifiedUpdateCacheFrequency());
+        }
+        if (other.hasViewModifiedUseStatsForParallelization()) {
+          setViewModifiedUseStatsForParallelization(other.getViewModifiedUseStatsForParallelization());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -8267,6 +8381,72 @@ public final class PTableProtos {
         return this;
       }
 
+      // optional bool viewModifiedUpdateCacheFrequency = 40;
+      private boolean viewModifiedUpdateCacheFrequency_ ;
+      /**
+       * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+       */
+      public boolean hasViewModifiedUpdateCacheFrequency() {
+        return ((bitField1_ & 0x00000040) == 0x00000040);
+      }
+      /**
+       * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+       */
+      public boolean getViewModifiedUpdateCacheFrequency() {
+        return viewModifiedUpdateCacheFrequency_;
+      }
+      /**
+       * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+       */
+      public Builder setViewModifiedUpdateCacheFrequency(boolean value) {
+        bitField1_ |= 0x00000040;
+        viewModifiedUpdateCacheFrequency_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool viewModifiedUpdateCacheFrequency = 40;</code>
+       */
+      public Builder clearViewModifiedUpdateCacheFrequency() {
+        bitField1_ = (bitField1_ & ~0x00000040);
+        viewModifiedUpdateCacheFrequency_ = false;
+        onChanged();
+        return this;
+      }
+
+      // optional bool viewModifiedUseStatsForParallelization = 41;
+      private boolean viewModifiedUseStatsForParallelization_ ;
+      /**
+       * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+       */
+      public boolean hasViewModifiedUseStatsForParallelization() {
+        return ((bitField1_ & 0x00000080) == 0x00000080);
+      }
+      /**
+       * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+       */
+      public boolean getViewModifiedUseStatsForParallelization() {
+        return viewModifiedUseStatsForParallelization_;
+      }
+      /**
+       * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+       */
+      public Builder setViewModifiedUseStatsForParallelization(boolean value) {
+        bitField1_ |= 0x00000080;
+        viewModifiedUseStatsForParallelization_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional bool viewModifiedUseStatsForParallelization = 41;</code>
+       */
+      public Builder clearViewModifiedUseStatsForParallelization() {
+        bitField1_ = (bitField1_ & ~0x00000080);
+        viewModifiedUseStatsForParallelization_ = false;
+        onChanged();
+        return this;
+      }
+
       // @@protoc_insertion_point(builder_scope:PTable)
     }
 
@@ -8936,7 +9116,7 @@ public final class PTableProtos {
       "es\030\002 \003(\014\022\033\n\023guidePostsByteCount\030\003 \001(\003\022\025\n" +
       "\rkeyBytesCount\030\004 \001(\003\022\027\n\017guidePostsCount\030" +
       "\005 \001(\005\022!\n\013pGuidePosts\030\006 \001(\0132\014.PGuidePosts" +
-      "\"\311\007\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016" +
+      "\"\243\010\n\006PTable\022\027\n\017schemaNameBytes\030\001 \002(\014\022\026\n\016" +
       "tableNameBytes\030\002 \002(\014\022\036\n\ttableType\030\003 \002(\0162" +
       "\013.PTableType\022\022\n\nindexState\030\004 \001(\t\022\026\n\016sequ" +
       "enceNumber\030\005 \002(\003\022\021\n\ttimeStamp\030\006 \002(\003\022\023\n\013p" +
@@ -8960,12 +9140,14 @@ public final class PTableProtos {
       "odedCQCounters\030$ \003(\0132\021.EncodedCQCounter\022" +
       "\"\n\032useStatsForParallelization\030% \001(\010\022\033\n\023t" +
       "ransactionProvider\030& \001(\005\022\032\n\017viewIndexIdT" +
-      "ype\030\' \001(\005:\0015\"6\n\020EncodedCQCounter\022\021\n\tcolF" +
-      "amily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A\n\nPTableTy",
-      "pe\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VIEW\020\002\022\t\n\005IN" +
-      "DEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache.phoenix.co" +
-      "processor.generatedB\014PTableProtosH\001\210\001\001\240\001" +
-      "\001"
+      "ype\030\' \001(\005:\0015\022(\n viewModifiedUpdateCacheF" +
+      "requency\030( \001(\010\022.\n&viewModifiedUseStatsFo",
+      "rParallelization\030) \001(\010\"6\n\020EncodedCQCount" +
+      "er\022\021\n\tcolFamily\030\001 \002(\t\022\017\n\007counter\030\002 \002(\005*A" +
+      "\n\nPTableType\022\n\n\006SYSTEM\020\000\022\010\n\004USER\020\001\022\010\n\004VI" +
+      "EW\020\002\022\t\n\005INDEX\020\003\022\010\n\004JOIN\020\004B@\n(org.apache." +
+      "phoenix.coprocessor.generatedB\014PTablePro" +
+      "tosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -8989,7 +9171,7 @@ public final class PTableProtos {
           internal_static_PTable_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_PTable_descriptor,
-              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisable [...]
+              new java.lang.String[] { "SchemaNameBytes", "TableNameBytes", "TableType", "IndexState", "SequenceNumber", "TimeStamp", "PkNameBytes", "BucketNum", "Columns", "Indexes", "IsImmutableRows", "DataTableNameBytes", "DefaultFamilyName", "DisableWAL", "MultiTenant", "ViewType", "ViewStatement", "PhysicalNames", "TenantId", "ViewIndexId", "IndexType", "StatsTimeStamp", "StoreNulls", "BaseColumnCount", "RowKeyOrderOptimizable", "Transactional", "UpdateCacheFrequency", "IndexDisable [...]
           internal_static_EncodedCQCounter_descriptor =
             getDescriptor().getMessageTypes().get(3);
           internal_static_EncodedCQCounter_fieldAccessorTable = new
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java
index 79d90f0..62a0f28 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/tasks/DropChildViewsTask.java
@@ -17,9 +17,9 @@
  */
 package org.apache.phoenix.coprocessor.tasks;
 
-import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.TaskRegionObserver;
+import org.apache.phoenix.util.ViewUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.schema.MetaDataClient;
 import org.apache.phoenix.schema.task.Task;
@@ -47,7 +47,8 @@ public class DropChildViewsTask extends BaseTask {
             if (tenantId != null) {
                 Properties tenantProps = new Properties();
                 tenantProps.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
-                pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration()).unwrap(PhoenixConnection.class);
+                pconn = QueryUtil.getConnectionOnServer(tenantProps, env.getConfiguration())
+                        .unwrap(PhoenixConnection.class);
             }
             else {
                 pconn = QueryUtil.getConnectionOnServer(env.getConfiguration()).unwrap(PhoenixConnection.class);
@@ -56,8 +57,8 @@ public class DropChildViewsTask extends BaseTask {
             MetaDataProtocol.MetaDataMutationResult result = new MetaDataClient(pconn).updateCache(pconn.getTenantId(),
                     taskRecord.getSchemaName(), taskRecord.getTableName(), true);
             if (result.getMutationCode() != MetaDataProtocol.MutationCode.TABLE_ALREADY_EXISTS) {
-                MetaDataEndpointImpl
-                        .dropChildViews(env, taskRecord.getTenantIdBytes(), taskRecord.getSchemaNameBytes(), taskRecord.getTableNameBytes());
+                ViewUtil.dropChildViews(env, taskRecord.getTenantIdBytes(),
+                        taskRecord.getSchemaNameBytes(), taskRecord.getTableNameBytes());
                 return new TaskRegionObserver.TaskResult(TaskRegionObserver.TaskResultCode.SUCCESS, "");
             } else if (System.currentTimeMillis() < timeMaxInterval + timestamp.getTime()) {
                 // skip this task as it has not been expired and its parent table has not been dropped yet
@@ -77,7 +78,7 @@ public class DropChildViewsTask extends BaseTask {
             }
         }
         catch (Throwable t) {
-            LOGGER.warn("Exception while dropping a child view task. " +
+            LOGGER.error("Exception while dropping a child view task. " +
                     taskRecord.getSchemaName()  + "." + taskRecord.getTableName() +
                     " with tenant id " + (taskRecord.getTenantId() == null ? " IS NULL" : taskRecord.getTenantId()) +
                     " and timestamp " + timestamp.toString(), t);
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
index 4c59fc6..af97d92 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/exception/SQLExceptionCode.java
@@ -397,6 +397,7 @@ public enum SQLExceptionCode {
     
     UPDATE_CACHE_FREQUENCY_INVALID(1130, "XCL30", "UPDATE_CACHE_FREQUENCY cannot be set to ALWAYS if APPEND_ONLY_SCHEMA is true."),
     CANNOT_DROP_COL_APPEND_ONLY_SCHEMA(1131, "XCL31", "Cannot drop column from table that with append only schema."),
+    CANNOT_DROP_VIEW_REFERENCED_COL(1132, "XCL32", "Cannot drop column that is referenced in view where clause."),
     
     CANNOT_ALTER_IMMUTABLE_ROWS_PROPERTY(1133, "XCL33", "IMMUTABLE_ROWS property can be changed only if the table storage scheme is ONE_CELL_PER_KEYVALUE_COLUMN"),
     CANNOT_ALTER_TABLE_PROPERTY_ON_VIEW(1134, "XCL34", "Altering this table property on a view is not allowed"),
@@ -424,6 +425,8 @@ public enum SQLExceptionCode {
     CONCURRENT_UPGRADE_IN_PROGRESS(2010, "INT12", ""),
     UPGRADE_REQUIRED(2011, "INT13", ""),
     UPGRADE_NOT_REQUIRED(2012, "INT14", ""),
+    GET_TABLE_ERROR(2013, "INT15", "MetadataEndpointImpl doGetTable called for table not present " +
+            "on region"),
     OPERATION_TIMED_OUT(6000, "TIM01", "Operation timed out.", new Factory() {
         @Override
         public SQLException newException(SQLExceptionInfo info) {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
index d5a5199..18009ad 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServices.java
@@ -83,25 +83,27 @@ public interface ConnectionQueryServices extends QueryServices, MetaDataMutated
      * @param tableTimestamp timestamp of table if its present in the client side cache
      * @param clientTimetamp if the client connection has an scn, or of the table is transactional
      *            the txn write pointer
-     * @param skipAddingIndexes if true will the returned PTable will not include any indexes
-     * @param skipAddingParentColumns if true will the returned PTable will not include any columns
-     *            derived from ancestors
-     * @param lockedAncestorTable ancestor table table that is being mutated (as we won't be able to
-     *            resolve this table as its locked)
      * @return PTable for the given tenant id, schema and table name
      */
     public MetaDataMutationResult getTable(PName tenantId, byte[] schemaName, byte[] tableName,
-            long tableTimestamp, long clientTimetamp, boolean skipAddingIndexes,
-            boolean skipAddingParentColumns, PTable lockedAncestorTable) throws SQLException;
+            long tableTimestamp, long clientTimetamp) throws SQLException;
     public MetaDataMutationResult getFunctions(PName tenantId, List<Pair<byte[], Long>> functionNameAndTimeStampPairs, long clientTimestamp) throws SQLException;
 
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] tableName, PTableType tableType,
-            Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet) throws SQLException;
-    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade, boolean skipAddingParentColumns) throws SQLException;
+                                              Map<String, Object> tableProps,
+                                              List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
+                                              boolean isNamespaceMapped, boolean allocateIndexId,
+                                              boolean isDoNotUpgradePropSet, PTable parentTable) throws SQLException;
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException;
     public MetaDataMutationResult dropFunction(List<Mutation> tableMetadata, boolean ifExists) throws SQLException;
-    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException;
-    public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException;
+    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData,
+                                            PTable table,
+                                            PTable parentTable,
+                                            Map<String, List<Pair<String, Object>>> properties,
+                                            Set<String> colFamiliesForPColumnsToBeAdded,
+                                            List<PColumn> columns) throws SQLException;
+    public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata,
+                                             PTableType tableType, PTable parentTable) throws SQLException;
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName) throws SQLException;
     public MetaDataMutationResult updateIndexState(List<Mutation> tableMetadata, String parentTableName,  Map<String, List<Pair<String,Object>>> stmtProperties,  PTable table) throws SQLException;
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
index 7d7860c..c084e67 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionQueryServicesImpl.java
@@ -638,7 +638,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 // If existing table isn't older than new table, don't replace
                 // If a client opens a connection at an earlier timestamp, this can happen
                 PTable existingTable = latestMetaData.getTableRef(new PTableKey(table.getTenantId(), table.getName().getString())).getTable();
-                if (existingTable.getTimeStamp() >= table.getTimeStamp()) {
+                if (existingTable.getTimeStamp() > table.getTimeStamp()) {
                     return;
                 }
             } catch (TableNotFoundException e) {}
@@ -1444,8 +1444,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             }
 
             ht = this.getTable(metaTable);
+            final byte[] tablekey = PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES;
             final Map<byte[], GetVersionResponse> results =
-                    ht.coprocessorService(MetaDataService.class, null, null, new Batch.Call<MetaDataService,GetVersionResponse>() {
+                    ht.coprocessorService(MetaDataService.class, tablekey, tablekey, new Batch.Call<MetaDataService,GetVersionResponse>() {
                         @Override
                         public GetVersionResponse call(MetaDataService instance) throws IOException {
                             ServerRpcController controller = new ServerRpcController();
@@ -1649,9 +1650,11 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     @Override
     public MetaDataMutationResult createTable(final List<Mutation> tableMetaData, final byte[] physicalTableName,
-            PTableType tableType, Map<String, Object> tableProps,
-            final List<Pair<byte[], Map<String, Object>>> families, byte[][] splits, boolean isNamespaceMapped,
-      final boolean allocateIndexId, final boolean isDoNotUpgradePropSet) throws SQLException {
+                                              PTableType tableType, Map<String, Object> tableProps,
+                                              final List<Pair<byte[], Map<String, Object>>> families,
+                                              byte[][] splits, boolean isNamespaceMapped,
+                                              final boolean allocateIndexId, final boolean isDoNotUpgradePropSet,
+                                              final PTable parentTable) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         Mutation m = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetaData);
         byte[] key = m.getRow();
@@ -1724,6 +1727,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         if (allocateIndexId) {
                             builder.setAllocateIndexId(allocateIndexId);
                         }
+                if (parentTable!=null) {
+                    builder.setParentTable(PTableImpl.toProto(parentTable));
+                }
                 CreateTableRequest build = builder.build();
                 instance.createTable(controller, build, rpcCallback);
                 if(controller.getFailedOn() != null) {
@@ -1737,9 +1743,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     @Override
     public MetaDataMutationResult getTable(final PName tenantId, final byte[] schemaBytes,
-            final byte[] tableBytes, final long tableTimestamp, final long clientTimestamp,
-            final boolean skipAddingIndexes, final boolean skipAddingParentColumns,
-            final PTable lockedAncestorTable) throws SQLException {
+            final byte[] tableBytes, final long tableTimestamp, final long clientTimestamp) throws SQLException {
         final byte[] tenantIdBytes = tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId.getBytes();
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
         return metaDataCoprocessorExec(tableKey,
@@ -1756,10 +1760,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 builder.setTableTimestamp(tableTimestamp);
                 builder.setClientTimestamp(clientTimestamp);
                 builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                builder.setSkipAddingParentColumns(skipAddingParentColumns);
-                builder.setSkipAddingIndexes(skipAddingIndexes);
-                if (lockedAncestorTable!=null)
-                    builder.setLockedAncestorTable(PTableImpl.toProto(lockedAncestorTable));
                 instance.getTable(controller, builder.build(), rpcCallback);
                 if(controller.getFailedOn() != null) {
                     throw controller.getFailedOn();
@@ -1771,7 +1771,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
 
     @Override
     public MetaDataMutationResult dropTable(final List<Mutation> tableMetaData, final PTableType tableType,
-            final boolean cascade, final boolean skipAddingParentColumns) throws SQLException {
+            final boolean cascade) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
@@ -1793,7 +1793,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 builder.setTableType(tableType.getSerializedValue());
                 builder.setCascade(cascade);
                 builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
-                builder.setSkipAddingParentColumns(skipAddingParentColumns);
                 instance.dropTable(controller, builder.build(), rpcCallback);
                 if(controller.getFailedOn() != null) {
                     throw controller.getFailedOn();
@@ -1953,7 +1952,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             byte[] schemaName = Bytes.toBytes(SchemaUtil.getSchemaNameFromFullName(fullTableName));
             byte[] tableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(fullTableName));
             MetaDataMutationResult result = this.getTable(tenantId, schemaName, tableName, HConstants.LATEST_TIMESTAMP,
-                    timestamp, false, false, null);
+                    timestamp);
             table = result.getTable();
             if (table == null) { throw e; }
         }
@@ -1992,7 +1991,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> stmtProperties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException {
+    public MetaDataMutationResult addColumn(final List<Mutation> tableMetaData,
+                                            PTable table,
+                                            final PTable parentTable,
+                                            Map<String, List<Pair<String, Object>>> stmtProperties,
+                                            Set<String> colFamiliesForPColumnsToBeAdded,
+                                            List<PColumn> columns) throws SQLException {
         List<Pair<byte[], Map<String, Object>>> families = new ArrayList<>(stmtProperties.size());
         Map<String, Object> tableProps = new HashMap<>();
         Set<HTableDescriptor> tableDescriptors = Collections.emptySet();
@@ -2070,6 +2074,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                         builder.addTableMetadataMutations(mp.toByteString());
                     }
                     builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                    if (parentTable!=null)
+                        builder.setParentTable(PTableImpl.toProto(parentTable));
                     instance.addColumn(controller, builder.build(), rpcCallback);
                     if(controller.getFailedOn() != null) {
                         throw controller.getFailedOn();
@@ -2738,7 +2744,9 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     @Override
-    public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData, PTableType tableType) throws SQLException {
+    public MetaDataMutationResult dropColumn(final List<Mutation> tableMetaData,
+                                             final PTableType tableType,
+                                             final PTable parentTable) throws SQLException {
         byte[][] rowKeyMetadata = new byte[3][];
         SchemaUtil.getVarChars(tableMetaData.get(0).getRow(), rowKeyMetadata);
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
@@ -2758,6 +2766,8 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                     builder.addTableMetadataMutations(mp.toByteString());
                 }
                 builder.setClientVersion(VersionUtil.encodeVersion(PHOENIX_MAJOR_VERSION, PHOENIX_MINOR_VERSION, PHOENIX_PATCH_NUMBER));
+                if (parentTable!=null)
+                    builder.setParentTable(PTableImpl.toProto(parentTable));
                 instance.dropColumn(controller, builder.build(), rpcCallback);
                 if(controller.getFailedOn() != null) {
                     throw controller.getFailedOn();
@@ -3854,7 +3864,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         }
         tableMetadata.addAll(metaConnection.getMutationState().toMutations(metaConnection.getSCN()).next().getSecond());
         metaConnection.rollback();
-        metaConnection.getQueryServices().addColumn(tableMetadata, sysCatalogPTable, Collections.<String,List<Pair<String,Object>>>emptyMap(), Collections.<String>emptySet(), Lists.newArrayList(column));
+        metaConnection.getQueryServices().addColumn(tableMetadata, sysCatalogPTable, null, Collections.<String,List<Pair<String,Object>>>emptyMap(), Collections.<String>emptySet(), Lists.newArrayList(column));
         metaConnection.removeTable(null, SYSTEM_CATALOG_NAME, null, timestamp);
         ConnectionQueryServicesImpl.this.removeTable(null,
                 SYSTEM_CATALOG_NAME, null,
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
index 48edd1b..62b1def 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/ConnectionlessQueryServicesImpl.java
@@ -251,13 +251,13 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp, boolean skipAddingIndexes, boolean skipCombiningColumns, PTable ancestorTable) throws SQLException {
+    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp) throws SQLException {
         // Return result that will cause client to use it's own metadata instead of needing
         // to get anything from the server (since we don't have a connection)
         try {
             String fullTableName = SchemaUtil.getTableName(schemaBytes, tableBytes);
             PTable table = metaData.getTableRef(new PTableKey(tenantId, fullTableName)).getTable();
-            return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, table, true);
+            return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, table);
         } catch (TableNotFoundException e) {
             return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND, 0, null);
         }
@@ -293,8 +293,8 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     
     @Override
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType,
-            Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet) throws SQLException {
+                                              Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
+                                              boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet, PTable parentTable) throws SQLException {
         if (tableType == PTableType.INDEX && IndexUtil.isLocalIndexFamily(Bytes.toString(families.iterator().next().getFirst()))) {
             Object dataTableName = tableProps.get(PhoenixDatabaseMetaData.DATA_TABLE_NAME);
             List<HRegionLocation> regionLocations = tableSplits.get(dataTableName);
@@ -312,14 +312,19 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
+    public MetaDataMutationResult dropTable(List<Mutation> tableMetadata, PTableType tableType, boolean cascade) throws SQLException {
         byte[] tableName = getTableName(tableMetadata, null);
         tableSplits.remove(Bytes.toString(tableName));
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
     }
 
     @Override
-    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columnsToBeAdded) throws SQLException {
+    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData,
+                                            PTable table,
+                                            PTable parentTable,
+                                            Map<String, List<Pair<String, Object>>> properties,
+                                            Set<String> colFamiliesForPColumnsToBeAdded,
+                                            List<PColumn> columnsToBeAdded) throws SQLException {
         List<PColumn> columns = Lists.newArrayList(table.getColumns());
         columns.addAll(columnsToBeAdded);
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0,
@@ -327,7 +332,9 @@ public class ConnectionlessQueryServicesImpl extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata, PTableType tableType) throws SQLException {
+    public MetaDataMutationResult dropColumn(List<Mutation> tableMetadata,
+                                             PTableType tableType,
+                                             PTable parentTable) throws SQLException {
         return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, 0, null);
     }
     
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
index 4be4af8..c228270 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/query/DelegateConnectionQueryServices.java
@@ -107,32 +107,42 @@ public class DelegateConnectionQueryServices extends DelegateQueryServices imple
     }
 
     @Override
-    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp, boolean skipAddingIndexes, boolean skipAddingParentColumns, PTable ancestorTable) throws SQLException {
-        return getDelegate().getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, clientTimestamp, skipAddingIndexes, skipAddingParentColumns, ancestorTable);
+    public MetaDataMutationResult getTable(PName tenantId, byte[] schemaBytes, byte[] tableBytes, long tableTimestamp, long clientTimestamp) throws SQLException {
+        return getDelegate().getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, clientTimestamp);
     }
 
     @Override
     public MetaDataMutationResult createTable(List<Mutation> tableMetaData, byte[] physicalName, PTableType tableType,
-            Map<String, Object> tableProps, List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
-            boolean isNamespaceMapped, boolean allocateIndexId, boolean isDoNotUpgradePropSet) throws SQLException {
+                                              Map<String, Object> tableProps,
+                                              List<Pair<byte[], Map<String, Object>>> families, byte[][] splits,
+                                              boolean isNamespaceMapped, boolean allocateIndexId,
+                                              boolean isDoNotUpgradePropSet, PTable parentTable) throws SQLException {
         return getDelegate().createTable(tableMetaData, physicalName, tableType, tableProps, families, splits,
-                isNamespaceMapped, allocateIndexId, isDoNotUpgradePropSet);
+                isNamespaceMapped, allocateIndexId, isDoNotUpgradePropSet, parentTable);
     }
 
     @Override
-    public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType, boolean cascade, boolean skipAddingParentColumns) throws SQLException {
-        return getDelegate().dropTable(tabeMetaData, tableType, cascade, skipAddingParentColumns);
+    public MetaDataMutationResult dropTable(List<Mutation> tabeMetaData, PTableType tableType, boolean cascade) throws SQLException {
+        return getDelegate().dropTable(tabeMetaData, tableType, cascade);
     }
 
     @Override
-    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData, PTable table, Map<String, List<Pair<String,Object>>> properties, Set<String> colFamiliesForPColumnsToBeAdded, List<PColumn> columns) throws SQLException {
-        return getDelegate().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded, columns);
+    public MetaDataMutationResult addColumn(List<Mutation> tableMetaData,
+                                            PTable table,
+                                            PTable parentTable,
+                                            Map<String, List<Pair<String, Object>>> properties,
+                                            Set<String> colFamiliesForPColumnsToBeAdded,
+                                            List<PColumn> columns) throws SQLException {
+        return getDelegate().addColumn(tableMetaData, table, parentTable,
+                properties, colFamiliesForPColumnsToBeAdded, columns);
     }
 
 
     @Override
-    public MetaDataMutationResult dropColumn(List<Mutation> tabeMetaData, PTableType tableType) throws SQLException {
-        return getDelegate().dropColumn(tabeMetaData, tableType);
+    public MetaDataMutationResult dropColumn(List<Mutation> tabeMetaData,
+                                             PTableType tableType,
+                                             PTable parentTable) throws SQLException {
+        return getDelegate().dropColumn(tabeMetaData, tableType, parentTable);
     }
 
     @Override
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 b796f51..9084251 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
@@ -104,7 +104,6 @@ import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.ONE_CELL_P
 import static org.apache.phoenix.schema.PTable.ImmutableStorageScheme.SINGLE_CELL_ARRAY_WITH_OFFSETS;
 import static org.apache.phoenix.schema.PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS;
 import static org.apache.phoenix.schema.PTable.ViewType.MAPPED;
-import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
 import static org.apache.phoenix.schema.PTableType.TABLE;
 import static org.apache.phoenix.schema.PTableType.VIEW;
 import static org.apache.phoenix.schema.types.PDataType.FALSE_BYTES;
@@ -134,7 +133,6 @@ import java.util.Properties;
 import java.util.Set;
 
 import org.apache.hadoop.hbase.HColumnDescriptor;
-import com.google.common.base.Objects;
 import com.google.gson.JsonObject;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -164,6 +162,7 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.SharedTableState;
+import org.apache.phoenix.util.ViewUtil;
 import org.apache.phoenix.exception.SQLExceptionCode;
 import org.apache.phoenix.exception.SQLExceptionInfo;
 import org.apache.phoenix.execute.MutationState;
@@ -219,7 +218,6 @@ import org.apache.phoenix.schema.PTable.QualifierEncodingScheme;
 import org.apache.phoenix.schema.PTable.QualifierEncodingScheme.QualifierOutOfRangeException;
 import org.apache.phoenix.schema.PTable.ViewType;
 import org.apache.phoenix.schema.stats.GuidePostsKey;
-import org.apache.phoenix.schema.stats.StatisticsUtil;
 import org.apache.phoenix.schema.task.Task;
 import org.apache.phoenix.schema.types.PDataType;
 import org.apache.phoenix.schema.types.PDate;
@@ -659,8 +657,7 @@ public class MetaDataClient {
                 final byte[] tableBytes = PVarchar.INSTANCE.toBytes(tableName);
                 ConnectionQueryServices queryServices = connection.getQueryServices();
                 result =
-                        queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp,
-                            resolvedTimestamp, false, false, null);
+                        queryServices.getTable(tenantId, schemaBytes, tableBytes, tableTimestamp, resolvedTimestamp);
                 // if the table was assumed to be non transactional, but is actually transactional
                 // then re-resolve as of the right timestamp
                 if (result.getTable() != null
@@ -671,9 +668,8 @@ public class MetaDataClient {
                             HConstants.LATEST_TIMESTAMP);
                     // Reresolve if table timestamp is past timestamp as of which we should see data
                     if (result.getTable().getTimeStamp() >= resolveTimestamp) {
-                        result =
-                                queryServices.getTable(tenantId, schemaBytes, tableBytes,
-                                    tableTimestamp, resolveTimestamp, false, false, null);
+                        result = queryServices.getTable(tenantId, schemaBytes,
+                                tableBytes, tableTimestamp, resolveTimestamp);
                     }
                 }
 
@@ -694,7 +690,7 @@ public class MetaDataClient {
                     // Otherwise, a tenant would be required to create a VIEW first
                     // which is not really necessary unless you want to filter or add
                     // columns
-                    addTableToCache(result);
+                    PTable temp = addTableToCache(result);
                     return result;
                 } else {
                     // if (result.getMutationCode() == MutationCode.NEWER_TABLE_FOUND) {
@@ -710,7 +706,7 @@ public class MetaDataClient {
                             // In this case, we update the parent table which may in turn pull
                             // in indexes to add to this table.
                             long resolvedTime = TransactionUtil.getResolvedTime(connection, result);
-                            if (addIndexesFromParentTable(result, resolvedTimestamp, true)) {
+                            if (addColumnsAndIndexesFromAncestors(result, resolvedTimestamp, true)) {
                                 connection.addTable(result.getTable(), resolvedTime);
                             } else {
                                 // if we aren't adding the table, we still need to update the
@@ -831,129 +827,49 @@ public class MetaDataClient {
     }
 
     /**
-     * Fault in the parent table to the cache and add any indexes it has to the indexes
-     * of the table for which we just updated.
-     * TODO: combine this round trip with the one that updates the cache for the child table.
+     * Looks up the ancestors of views and view indexes and adds inherited columns and
+     * also any indexes of the ancestors that can be used
+     *
      * @param result the result from updating the cache for the current table.
      * @param resolvedTimestamp timestamp at which child table was resolved
-     * @param alwaysAddIndexes flag that determines whether we should recalculate
-     *        all indexes that can be used in the view
+     * @param alwaysAddAncestorColumnsAndIndexes flag that determines whether we should recalculate
+     *        all inherited columns and indexes that can be used in the view and
      * @return true if the PTable contained by result was modified and false otherwise
      * @throws SQLException if the physical table cannot be found
      */
-    private boolean addIndexesFromParentTable(MetaDataMutationResult result, Long resolvedTimestamp, boolean alwaysAddIndexes) throws SQLException {
-        PTable view = result.getTable();
-        // If not a view or if a view directly over an HBase table, there's nothing to do
-        if (view.getType() != PTableType.VIEW || view.getViewType() == ViewType.MAPPED) {
-            return false;
-        }
-        // a view on a table will not have a parent name but will have a physical table name (which is the parent)
-        String parentName = view.getParentName().getString();
-        String schemaName = SchemaUtil.getSchemaNameFromFullName(parentName);
-        String tableName = SchemaUtil.getTableNameFromFullName(parentName);
-        MetaDataMutationResult parentResult = updateCache(connection.getTenantId(), schemaName, tableName, false, resolvedTimestamp);
-        PTable parentTable = parentResult.getTable();
-        if (parentTable == null) {
-            throw new TableNotFoundException(schemaName, tableName);
-        }
-        // if alwaysAddIndexes is false we only add indexes if the parent table or view was updated from the server
-        if (!alwaysAddIndexes && !result.wasUpdated() && !parentResult.wasUpdated()) {
-            return false;
-        }
-        List<PTable> parentTableIndexes = parentTable.getIndexes();
-        if (parentTableIndexes.isEmpty()) {
-            return false;
-        }
-        // Filter out indexes if column doesn't exist in view
-        List<PTable> indexesToAdd = Lists.newArrayListWithExpectedSize(parentTableIndexes.size() + view.getIndexes().size());
-        if (result.wasUpdated()) { // Table from server never contains inherited indexes
-            indexesToAdd.addAll(view.getIndexes());
-        } else { // Only add original ones, as inherited ones may have changed
-            for (PTable index : view.getIndexes()) {
-                // Original indexes will not have a view statement while inherited ones will
-                if (index.getViewStatement() == null) {
-                    indexesToAdd.add(index);
-                }
-            }
-        }
-        for (PTable index : parentTableIndexes) {
-            boolean containsAllReqdCols = true;
-            // Ensure that all columns required to create index exist in the view too,
-            // since view columns may be removed.
-            IndexMaintainer indexMaintainer = index.getIndexMaintainer(parentTable, connection);
-            // Check that the columns required for the index pk are present in the view
-            Set<Pair<String, String>> indexedColInfos = indexMaintainer.getIndexedColumnInfo();
-            for (Pair<String, String> colInfo : indexedColInfos) {
-                try {
-                    String colFamily = colInfo.getFirst();
-                    String colName = colInfo.getSecond();
-                    if (colFamily == null) {
-                        view.getColumnForColumnName(colName);
-                    } else {
-                        view.getColumnFamily(colFamily).getPColumnForColumnName(colName);
-                    }
-                } catch (ColumnNotFoundException e) {
-                    containsAllReqdCols = false;
-                    break;
-                }
-            }
-            
-            // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
-            // all exist in the index on the parent table.
-            for (PColumn col : view.getColumns()) {
-                if (col.getViewConstant() != null) {
-                    try {
-                        // It'd be possible to use a local index that doesn't have all view constants,
-                        // but the WHERE clause for the view statement (which is added to the index below)
-                        // would fail to compile.
-                        String indexColumnName = IndexUtil.getIndexColumnName(col);
-                        index.getColumnForColumnName(indexColumnName);
-                    } catch (ColumnNotFoundException e1) {
-                        PColumn indexCol = null;
-                        try {
-                            String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
-                            String colName = col.getName().getString();
-                            if (cf != null) {
-                                indexCol = parentTable.getColumnFamily(cf).getPColumnForColumnName(colName);
-                            }
-                            else {
-                                indexCol = parentTable.getColumnForColumnName(colName);
-                            }
-                        } catch (ColumnNotFoundException e2) { // Ignore this index and continue with others
-                            containsAllReqdCols = false;
-                            break;
-                        }
-                        if (indexCol.getViewConstant()==null || Bytes.compareTo(indexCol.getViewConstant(), col.getViewConstant())!=0) {
-                            containsAllReqdCols = false;
-                            break;
-                        }
-                    }
-                }
-            }
-            if (containsAllReqdCols) {
-                // Tack on view statement to index to get proper filtering for view
-                String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
-                PName modifiedIndexName = PNameFactory.newName(view.getName().getString() 
-                    + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + index.getName().getString());
-                // add the index table with a new name so that it does not conflict with the existing index table
-                // and set update cache frequency to that of the view
-                if (Objects.equal(viewStatement, index.getViewStatement())) {
-                    indexesToAdd.add(index);
-                } else {
-                    indexesToAdd.add(PTableImpl.builderWithColumns(index, getColumnsToClone(index))
-                            .setTableName(modifiedIndexName)
-                            .setViewStatement(viewStatement)
-                            .setUpdateCacheFrequency(view.getUpdateCacheFrequency())
-                            .setTenantId(view.getTenantId())
-                            .build());
-                }
-            }
+    private boolean addColumnsAndIndexesFromAncestors(MetaDataMutationResult result, Long resolvedTimestamp,
+                                                      boolean alwaysAddAncestorColumnsAndIndexes) throws SQLException {
+        PTable table = result.getTable();
+        boolean hasIndexId = table.getViewIndexId() != null;
+        // only need to inherit columns and indexes for view indexes and views
+        if ( (table.getType()==PTableType.INDEX && hasIndexId)
+                || (table.getType() == PTableType.VIEW && table.getViewType() != ViewType.MAPPED)) {
+            String parentName = table.getParentName().getString();
+            String parentSchemaName = SchemaUtil.getSchemaNameFromFullName(parentName);
+            String parentTableName = SchemaUtil.getTableNameFromFullName(parentName);
+            MetaDataMutationResult parentResult = updateCache(connection.getTenantId(), parentSchemaName, parentTableName,
+                    false, resolvedTimestamp);
+            PTable parentTable = parentResult.getTable();
+            if (parentResult.getMutationCode() == MutationCode.TABLE_NOT_FOUND || parentTable == null) {
+                // this mean the parent table was dropped and the child views have not yet been
+                // dropped by the TaskRegionObserver
+                String schemaName = table.getSchemaName()!=null ? table.getSchemaName().getString() : null;
+                throw new TableNotFoundException(schemaName, parentName);
+            }
+            // only inherit columns view indexes (and not local indexes on regular tables which also have a viewIndexId)
+            if (hasIndexId && parentTable.getType() != PTableType.VIEW) {
+                return false;
+            }
+            // if alwaysAddAncestorColumnsAndIndexes is false we only recalculate if the ancestor table or table
+            // was updated from the server
+            if (!alwaysAddAncestorColumnsAndIndexes && !result.wasUpdated() && !parentResult.wasUpdated()) {
+                return false;
+            }
+            PTable resolvedTable = ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, table, parentTable);
+            result.setTable(resolvedTable);
+            return true;
         }
-        PTable allIndexesTable = PTableImpl.builderWithColumns(view, getColumnsToClone(view))
-                .setIndexes(indexesToAdd == null ? Collections.<PTable>emptyList() : indexesToAdd)
-                .build();
-        result.setTable(allIndexesTable);
-        return true;
+        return false;
     }
 
     private void addColumnMutation(String schemaName, String tableName, PColumn column, PreparedStatement colUpsert, String parentTableName, String pkName, Short keySeq, boolean isSalted) throws SQLException {
@@ -1072,8 +988,9 @@ public class MetaDataClient {
                 isNull = false;
             }
             PColumn column = new PColumnImpl(PNameFactory.newName(columnName), familyName, def.getDataType(),
-                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(), null, false, def.getExpression(), isRowTimestamp, false, columnQualifierBytes,
-                HConstants.LATEST_TIMESTAMP);
+                    def.getMaxLength(), def.getScale(), isNull, position, sortOrder, def.getArraySize(),
+                    null, false, def.getExpression(), isRowTimestamp,
+                    false, columnQualifierBytes, EnvironmentEdgeManager.currentTimeMillis());
             return column;
         } catch (IllegalArgumentException e) { // Based on precondition check in constructor
             throw new SQLException(e);
@@ -1537,7 +1454,6 @@ public class MetaDataClient {
         List<ColumnName> includedColumns = statement.getIncludeColumns();
         TableRef tableRef = null;
         PTable table = null;
-        int numRetries = 0;
         boolean allocateIndexId = false;
         boolean isLocalIndex = statement.getIndexType() == IndexType.LOCAL;
         int hbaseVersion = connection.getQueryServices().getLowestClusterHBaseVersion();
@@ -1549,181 +1465,196 @@ public class MetaDataClient {
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_LOCAL_INDEXES).setTableName(indexTableName.getTableName()).build().buildException();
             }
         }
-        while (true) {
-            try {
-                ColumnResolver resolver = FromCompiler.getResolver(statement, connection, statement.getUdfParseNodes());
-                tableRef = resolver.getTables().get(0);
-                Date asyncCreatedDate = null;
-                if (statement.isAsync()) {
-                    asyncCreatedDate = new Date(tableRef.getTimeStamp());
-                }
-                PTable dataTable = tableRef.getTable();
-                boolean isTenantConnection = connection.getTenantId() != null;
-                if (isTenantConnection) {
-                    if (dataTable.getType() != PTableType.VIEW) {
-                        throw new SQLFeatureNotSupportedException("An index may only be created for a VIEW through a tenant-specific connection");
-                    }
-                }
-                if (!dataTable.isImmutableRows()) {
-                    if (hbaseVersion < MetaDataProtocol.MUTABLE_SI_VERSION_THRESHOLD) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES).setTableName(indexTableName.getTableName()).build().buildException();
-                    }
-                    if (!connection.getQueryServices().hasIndexWALCodec() && !dataTable.isTransactional()) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_MUTABLE_INDEX_CONFIG).setTableName(indexTableName.getTableName()).build().buildException();
-                    }
-                    boolean tableWithRowTimestampCol = dataTable.getRowTimestampColPos() != -1;
-                    if (tableWithRowTimestampCol) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_INDEX_ON_MUTABLE_TABLE_WITH_ROWTIMESTAMP).setTableName(indexTableName.getTableName()).build().buildException();
-                    }
+        Set<String> acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3);
+        String physicalSchemaName = null;
+        String physicalTableName = null;
+        try {
+            ColumnResolver resolver = FromCompiler.getResolver(statement, connection, statement.getUdfParseNodes());
+            tableRef = resolver.getTables().get(0);
+            Date asyncCreatedDate = null;
+            if (statement.isAsync()) {
+                asyncCreatedDate = new Date(tableRef.getTimeStamp());
+            }
+            PTable dataTable = tableRef.getTable();
+            boolean isTenantConnection = connection.getTenantId() != null;
+            if (isTenantConnection) {
+                if (dataTable.getType() != PTableType.VIEW) {
+                    throw new SQLFeatureNotSupportedException("An index may only be created for a VIEW through a tenant-specific connection");
                 }
-                if (dataTable.isTransactional() 
-                        && isLocalIndex 
-                        && dataTable.getTransactionProvider().getTransactionProvider().isUnsupported(PhoenixTransactionProvider.Feature.ALLOW_LOCAL_INDEX)) {
-                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_LOCAL_INDEX_FOR_TXN_TABLE).setMessage(dataTable.getTransactionProvider().name()).setTableName(indexTableName.getTableName()).build().buildException();
+            }
+            if (!dataTable.isImmutableRows()) {
+                if (hbaseVersion < MetaDataProtocol.MUTABLE_SI_VERSION_THRESHOLD) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.NO_MUTABLE_INDEXES).setTableName(indexTableName.getTableName()).build().buildException();
                 }
-                int posOffset = 0;
-                List<PColumn> pkColumns = dataTable.getPKColumns();
-                Set<RowKeyColumnExpression> unusedPkColumns;
-                if (dataTable.getBucketNum() != null) { // Ignore SALT column
-                    unusedPkColumns = Sets.newLinkedHashSetWithExpectedSize(pkColumns.size()-1);
-                    posOffset++;
-                } else {
-                    unusedPkColumns = Sets.newLinkedHashSetWithExpectedSize(pkColumns.size());
+                if (!connection.getQueryServices().hasIndexWALCodec() && !dataTable.isTransactional()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.INVALID_MUTABLE_INDEX_CONFIG).setTableName(indexTableName.getTableName()).build().buildException();
                 }
-                for (int i = posOffset; i < pkColumns.size(); i++) {
-                    PColumn column = pkColumns.get(i);
-                    unusedPkColumns.add(new RowKeyColumnExpression(column, new RowKeyValueAccessor(pkColumns, i), "\""+column.getName().getString()+"\""));
+                boolean tableWithRowTimestampCol = dataTable.getRowTimestampColPos() != -1;
+                if (tableWithRowTimestampCol) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_INDEX_ON_MUTABLE_TABLE_WITH_ROWTIMESTAMP).setTableName(indexTableName.getTableName()).build().buildException();
                 }
-                List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
-                List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
+            }
+            if (dataTable.isTransactional()
+                    && isLocalIndex
+                    && dataTable.getTransactionProvider().getTransactionProvider().isUnsupported(PhoenixTransactionProvider.Feature.ALLOW_LOCAL_INDEX)) {
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_CREATE_LOCAL_INDEX_FOR_TXN_TABLE).setMessage(dataTable.getTransactionProvider().name()).setTableName(indexTableName.getTableName()).build().buildException();
+            }
+            int posOffset = 0;
+            List<PColumn> pkColumns = dataTable.getPKColumns();
+            Set<RowKeyColumnExpression> unusedPkColumns;
+            if (dataTable.getBucketNum() != null) { // Ignore SALT column
+                unusedPkColumns = Sets.newLinkedHashSetWithExpectedSize(pkColumns.size()-1);
+                posOffset++;
+            } else {
+                unusedPkColumns = Sets.newLinkedHashSetWithExpectedSize(pkColumns.size());
+            }
+            for (int i = posOffset; i < pkColumns.size(); i++) {
+                PColumn column = pkColumns.get(i);
+                unusedPkColumns.add(new RowKeyColumnExpression(column, new RowKeyValueAccessor(pkColumns, i), "\""+column.getName().getString()+"\""));
+            }
+            List<ColumnDefInPkConstraint> allPkColumns = Lists.newArrayListWithExpectedSize(unusedPkColumns.size());
+            List<ColumnDef> columnDefs = Lists.newArrayListWithExpectedSize(includedColumns.size() + indexParseNodeAndSortOrderList.size());
 
-                /*
-                 * Allocate an index ID in two circumstances:
-                 * 1) for a local index, as all local indexes will reside in the same HBase table
-                 * 2) for a view on an index.
-                 */
-                if (isLocalIndex || (dataTable.getType() == PTableType.VIEW && dataTable.getViewType() != ViewType.MAPPED)) {
-                    allocateIndexId = true;
-                    PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
-                    ColumnName colName = ColumnName.caseSensitiveColumnName(MetaDataUtil.getViewIndexIdColumnName());
-                    allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
-                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
+            /*
+             * Allocate an index ID in two circumstances:
+             * 1) for a local index, as all local indexes will reside in the same HBase table
+             * 2) for a view on an index.
+             */
+            if (isLocalIndex || (dataTable.getType() == PTableType.VIEW && dataTable.getViewType() != ViewType.MAPPED)) {
+                allocateIndexId = true;
+                PDataType dataType = MetaDataUtil.getViewIndexIdDataType();
+                ColumnName colName = ColumnName.caseSensitiveColumnName(MetaDataUtil.getViewIndexIdColumnName());
+                allPkColumns.add(new ColumnDefInPkConstraint(colName, SortOrder.getDefault(), false));
+                columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), false, null, null, false, SortOrder.getDefault(), null, false));
+            }
+
+            if (dataTable.isMultiTenant()) {
+                PColumn col = dataTable.getPKColumns().get(posOffset);
+                RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
+                unusedPkColumns.remove(columnExpression);
+                PDataType dataType = IndexUtil.getIndexColumnDataType(col);
+                ColumnName colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
+                allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
+                columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
+            }
+
+            PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
+            StatementContext context = new StatementContext(phoenixStatment, resolver);
+            IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
+            Set<ColumnName> indexedColumnNames = Sets.newHashSetWithExpectedSize(indexParseNodeAndSortOrderList.size());
+            for (Pair<ParseNode, SortOrder> pair : indexParseNodeAndSortOrderList) {
+                ParseNode parseNode = pair.getFirst();
+                // normalize the parse node
+                parseNode = StatementNormalizer.normalize(parseNode, resolver);
+                // compile the parseNode to get an expression
+                expressionIndexCompiler.reset();
+                Expression expression = parseNode.accept(expressionIndexCompiler);
+                if (expressionIndexCompiler.isAggregate()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                 }
-
-                if (dataTable.isMultiTenant()) {
-                    PColumn col = dataTable.getPKColumns().get(posOffset);
-                    RowKeyColumnExpression columnExpression = new RowKeyColumnExpression(col, new RowKeyValueAccessor(pkColumns, posOffset), col.getName().getString());
-                    unusedPkColumns.remove(columnExpression);
-                    PDataType dataType = IndexUtil.getIndexColumnDataType(col);
-                    ColumnName colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
-                    allPkColumns.add(new ColumnDefInPkConstraint(colName, col.getSortOrder(), false));
-                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, SortOrder.getDefault(), col.getName().getString(), col.isRowTimestamp()));
+                if (expression.getDeterminism() != Determinism.ALWAYS) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.NON_DETERMINISTIC_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                 }
-
-                PhoenixStatement phoenixStatment = new PhoenixStatement(connection);
-                StatementContext context = new StatementContext(phoenixStatment, resolver);
-                IndexExpressionCompiler expressionIndexCompiler = new IndexExpressionCompiler(context);
-                Set<ColumnName> indexedColumnNames = Sets.newHashSetWithExpectedSize(indexParseNodeAndSortOrderList.size());
-                for (Pair<ParseNode, SortOrder> pair : indexParseNodeAndSortOrderList) {
-                    ParseNode parseNode = pair.getFirst();
-                    // normalize the parse node
-                    parseNode = StatementNormalizer.normalize(parseNode, resolver);
-                    // compile the parseNode to get an expression
-                    expressionIndexCompiler.reset();
-                    Expression expression = parseNode.accept(expressionIndexCompiler);
-                    if (expressionIndexCompiler.isAggregate()) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.AGGREGATE_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
-                    }
-                    if (expression.getDeterminism() != Determinism.ALWAYS) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.NON_DETERMINISTIC_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
-                    }
-                    if (expression.isStateless()) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
-                    }
-                    unusedPkColumns.remove(expression);
-
-                    // Go through parse node to get string as otherwise we
-                    // can lose information during compilation
-                    StringBuilder buf = new StringBuilder();
-                    parseNode.toSQL(resolver, buf);
-                    // need to escape backslash as this expression will be re-parsed later
-                    String expressionStr = StringUtil.escapeBackslash(buf.toString());
-
-                    ColumnName colName = null;
-                    ColumnRef colRef = expressionIndexCompiler.getColumnRef();
-                    boolean isRowTimestamp = false;
-                    if (colRef!=null) {
-                        // if this is a regular column
-                        PColumn column = colRef.getColumn();
-                        String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
-                        colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
-                        isRowTimestamp = column.isRowTimestamp();
-                        if (colRef.getColumn().getExpressionStr() != null) {
-                            expressionStr = colRef.getColumn().getExpressionStr();
-                        }
-                    }
-                    else {
-                        // if this is an expression
-                        // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
-                        String name = expressionStr.replaceAll("\"", "'");
-                        colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, name));
-                    }
-                    indexedColumnNames.add(colName);
-                    PDataType dataType = IndexUtil.getIndexColumnDataType(expression.isNullable(), expression.getDataType());
-                    allPkColumns.add(new ColumnDefInPkConstraint(colName, pair.getSecond(), isRowTimestamp));
-                    columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), expression.isNullable(), expression.getMaxLength(), expression.getScale(), false, pair.getSecond(), expressionStr, isRowTimestamp));
+                if (expression.isStateless()) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.STATELESS_EXPRESSION_NOT_ALLOWED_IN_INDEX).build().buildException();
                 }
-
-                // Next all the PK columns from the data table that aren't indexed
-                if (!unusedPkColumns.isEmpty()) {
-                    for (RowKeyColumnExpression colExpression : unusedPkColumns) {
-                        PColumn col = dataTable.getPKColumns().get(colExpression.getPosition());
-                        // Don't add columns with constant values from updatable views, as
-                        // we don't need these in the index
-                        if (col.getViewConstant() == null) {
-                            ColumnName colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
-                            allPkColumns.add(new ColumnDefInPkConstraint(colName, colExpression.getSortOrder(), col.isRowTimestamp()));
-                            PDataType dataType = IndexUtil.getIndexColumnDataType(colExpression.isNullable(), colExpression.getDataType());
-                            columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(),
-                                    colExpression.isNullable(), colExpression.getMaxLength(), colExpression.getScale(),
-                                    false, colExpression.getSortOrder(), colExpression.toString(), col.isRowTimestamp()));
-                        }
+                unusedPkColumns.remove(expression);
+
+                // Go through parse node to get string as otherwise we
+                // can lose information during compilation
+                StringBuilder buf = new StringBuilder();
+                parseNode.toSQL(resolver, buf);
+                // need to escape backslash as this expression will be re-parsed later
+                String expressionStr = StringUtil.escapeBackslash(buf.toString());
+
+                ColumnName colName = null;
+                ColumnRef colRef = expressionIndexCompiler.getColumnRef();
+                boolean isRowTimestamp = false;
+                if (colRef!=null) {
+                    // if this is a regular column
+                    PColumn column = colRef.getColumn();
+                    String columnFamilyName = column.getFamilyName()!=null ? column.getFamilyName().getString() : null;
+                    colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(columnFamilyName, column.getName().getString()));
+                    isRowTimestamp = column.isRowTimestamp();
+                    if (colRef.getColumn().getExpressionStr() != null) {
+                        expressionStr = colRef.getColumn().getExpressionStr();
                     }
                 }
-
-                // Last all the included columns (minus any PK columns)
-                for (ColumnName colName : includedColumns) {
-                    PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
-                    colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
-                    // Check for duplicates between indexed and included columns
-                    if (indexedColumnNames.contains(colName)) {
-                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_EXIST_IN_DEF).build().buildException();
-                    }
-                    if (!SchemaUtil.isPKColumn(col) && col.getViewConstant() == null) {
-                        // Need to re-create ColumnName, since the above one won't have the column family name
-                        colName = ColumnName.caseSensitiveColumnName(isLocalIndex?IndexUtil.getLocalIndexColumnFamily(col.getFamilyName().getString()):col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
-                        columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder(), col.getExpressionStr(), col.isRowTimestamp()));
+                else {
+                    // if this is an expression
+                    // TODO column names cannot have double quotes, remove this once this PHOENIX-1621 is fixed
+                    String name = expressionStr.replaceAll("\"", "'");
+                    colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(null, name));
+                }
+                indexedColumnNames.add(colName);
+                PDataType dataType = IndexUtil.getIndexColumnDataType(expression.isNullable(), expression.getDataType());
+                allPkColumns.add(new ColumnDefInPkConstraint(colName, pair.getSecond(), isRowTimestamp));
+                columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(), expression.isNullable(), expression.getMaxLength(), expression.getScale(), false, pair.getSecond(), expressionStr, isRowTimestamp));
+            }
+
+            // Next all the PK columns from the data table that aren't indexed
+            if (!unusedPkColumns.isEmpty()) {
+                for (RowKeyColumnExpression colExpression : unusedPkColumns) {
+                    PColumn col = dataTable.getPKColumns().get(colExpression.getPosition());
+                    // Don't add columns with constant values from updatable views, as
+                    // we don't need these in the index
+                    if (col.getViewConstant() == null) {
+                        ColumnName colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
+                        allPkColumns.add(new ColumnDefInPkConstraint(colName, colExpression.getSortOrder(), col.isRowTimestamp()));
+                        PDataType dataType = IndexUtil.getIndexColumnDataType(colExpression.isNullable(), colExpression.getDataType());
+                        columnDefs.add(FACTORY.columnDef(colName, dataType.getSqlTypeName(),
+                                colExpression.isNullable(), colExpression.getMaxLength(), colExpression.getScale(),
+                                false, colExpression.getSortOrder(), colExpression.toString(), col.isRowTimestamp()));
                     }
                 }
+            }
 
-                // Set DEFAULT_COLUMN_FAMILY_NAME of index to match data table
-                // We need this in the props so that the correct column family is created
-                if (dataTable.getDefaultFamilyName() != null && dataTable.getType() != PTableType.VIEW && !allocateIndexId) {
-                    statement.getProps().put("", new Pair<String,Object>(DEFAULT_COLUMN_FAMILY_NAME,dataTable.getDefaultFamilyName().getString()));
+            // Last all the included columns (minus any PK columns)
+            for (ColumnName colName : includedColumns) {
+                PColumn col = resolver.resolveColumn(null, colName.getFamilyName(), colName.getColumnName()).getColumn();
+                colName = ColumnName.caseSensitiveColumnName(IndexUtil.getIndexColumnName(col));
+                // Check for duplicates between indexed and included columns
+                if (indexedColumnNames.contains(colName)) {
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.COLUMN_EXIST_IN_DEF).build().buildException();
+                }
+                if (!SchemaUtil.isPKColumn(col) && col.getViewConstant() == null) {
+                    // Need to re-create ColumnName, since the above one won't have the column family name
+                    colName = ColumnName.caseSensitiveColumnName(isLocalIndex?IndexUtil.getLocalIndexColumnFamily(col.getFamilyName().getString()):col.getFamilyName().getString(), IndexUtil.getIndexColumnName(col));
+                    columnDefs.add(FACTORY.columnDef(colName, col.getDataType().getSqlTypeName(), col.isNullable(), col.getMaxLength(), col.getScale(), false, col.getSortOrder(), col.getExpressionStr(), col.isRowTimestamp()));
                 }
-                PrimaryKeyConstraint pk = FACTORY.primaryKey(null, allPkColumns);
-                tableProps.put(MetaDataUtil.DATA_TABLE_NAME_PROP_NAME, dataTable.getName().getString());
+            }
 
-                CreateTableStatement tableStatement = FACTORY.createTable(indexTableName, statement.getProps(), columnDefs, pk, statement.getSplitNodes(), PTableType.INDEX, statement.ifNotExists(), null, null, statement.getBindCount(), null);
-                table = createTableInternal(tableStatement, splits, dataTable, null, null, MetaDataUtil.getViewIndexIdDataType(),null, null, allocateIndexId, statement.getIndexType(), asyncCreatedDate, tableProps, commonFamilyProps);
-                break;
-            } catch (ConcurrentTableMutationException e) { // Can happen if parent data table changes while above is in progress
-                if (numRetries<5) {
-                    numRetries++;
-                    continue;
+            // for view indexes
+            if (dataTable.getType() == PTableType.VIEW) {
+                String physicalName = dataTable.getPhysicalName().getString();
+                physicalSchemaName = SchemaUtil.getSchemaNameFromFullName(physicalName);
+                physicalTableName = SchemaUtil.getTableNameFromFullName(physicalName);
+                List<ColumnName> requiredCols = Lists.newArrayList(indexedColumnNames);
+                requiredCols.addAll(includedColumns);
+                for (ColumnName colName : requiredCols) {
+                    // acquire the mutex using the global physical table name to
+                    // prevent this column from being dropped while the index is being created
+                    boolean acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName,
+                            colName.toString());
+                    if (!acquiredMutex) {
+                        throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
+                    }
+                    acquiredColumnMutexSet.add(colName.toString());
                 }
-                throw e;
             }
+
+            // Set DEFAULT_COLUMN_FAMILY_NAME of index to match data table
+            // We need this in the props so that the correct column family is created
+            if (dataTable.getDefaultFamilyName() != null && dataTable.getType() != PTableType.VIEW && !allocateIndexId) {
+                statement.getProps().put("", new Pair<String,Object>(DEFAULT_COLUMN_FAMILY_NAME,dataTable.getDefaultFamilyName().getString()));
+            }
+            PrimaryKeyConstraint pk = FACTORY.primaryKey(null, allPkColumns);
+            tableProps.put(MetaDataUtil.DATA_TABLE_NAME_PROP_NAME, dataTable.getName().getString());
+            CreateTableStatement tableStatement = FACTORY.createTable(indexTableName, statement.getProps(), columnDefs, pk, statement.getSplitNodes(), PTableType.INDEX, statement.ifNotExists(), null, null, statement.getBindCount(), null);
+            table = createTableInternal(tableStatement, splits, dataTable, null, null, MetaDataUtil.getViewIndexIdDataType(),null, null, allocateIndexId, statement.getIndexType(), asyncCreatedDate, tableProps, commonFamilyProps);
+        }
+        finally {
+            deleteMutexCells(physicalSchemaName, physicalTableName, acquiredColumnMutexSet);
         }
         if (table == null) {
             return new MutationState(0, 0, connection);
@@ -1915,8 +1846,10 @@ public class MetaDataClient {
     }
     
     /**
-     * If we are creating a view we write a cell to the SYSTEM.MUTEX table with the rowkey of the
-     * parent table to prevent concurrent modifications
+     * While adding or dropping columns we write a cell to the SYSTEM.MUTEX table with the rowkey of the
+     * physical table to prevent conflicting concurrent modifications. For eg two client adding a column
+     * with the same name but different datatype, or once client dropping a column on a base table
+     * while another client creating a view or view index that requires the dropped column
      */
     private boolean writeCell(String tenantId, String schemaName, String tableName, String columnName)
             throws SQLException {
@@ -1924,8 +1857,7 @@ public class MetaDataClient {
     }
 
     /**
-     * Remove the cell that was written to to the SYSTEM.MUTEX table with the rowkey of the
-     * parent table to prevent concurrent modifications
+     * Remove the cell that was written to to the SYSTEM.MUTEX table with the rowkey of the physical table
      */
     private void deleteCell(String tenantId, String schemaName, String tableName, String columnName)
             throws SQLException {
@@ -1995,6 +1927,13 @@ public class MetaDataClient {
                 connection.getQueryServices().getProps().getBoolean(
                     QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK,
                     QueryServicesOptions.DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
+        Set<String> acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3);
+        String parentPhysicalName =
+                (parent!=null &&  parent.getPhysicalName()!=null) ? parent.getPhysicalName().getString() : null;
+        String parentPhysicalSchemaName = parentPhysicalName!=null ?
+                SchemaUtil.getSchemaNameFromFullName(parentPhysicalName) : null;
+        String parentPhysicalTableName = parentPhysicalName!=null ?
+                SchemaUtil.getTableNameFromFullName(parentPhysicalName) : null;
         connection.rollback();
         try {
             connection.setAutoCommit(false);
@@ -2786,6 +2725,18 @@ 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 index is being created
+                            boolean acquiredMutex = writeCell(null, parentPhysicalSchemaName, parentPhysicalTableName,
+                                    column.getName().getString());
+                            if (!acquiredMutex) {
+                                throw new ConcurrentTableMutationException(parentPhysicalSchemaName, parentPhysicalTableName);
+                            }
+                            acquiredColumnMutexSet.add(column.getName().getString());
+                        }
                     }
                     Short keySeq = SchemaUtil.isPKColumn(column) ? ++nextKeySeq : null;
                     // Prior to PHOENIX-3534 we were sending the parent table column metadata while creating a
@@ -2914,7 +2865,7 @@ public class MetaDataClient {
                     tableMetaData,
                     viewType == ViewType.MAPPED || allocateIndexId ? physicalNames.get(0).getBytes() : null,
                     tableType, tableProps, familyPropList, splits, isNamespaceMapped, allocateIndexId,
-                    UpgradeUtil.isNoUpgradeSet(connection.getClientInfo()));
+                    UpgradeUtil.isNoUpgradeSet(connection.getClientInfo()), parent);
             MutationCode code = result.getMutationCode();
             switch(code) {
             case TABLE_ALREADY_EXISTS:
@@ -2925,8 +2876,6 @@ public class MetaDataClient {
                     throw new TableAlreadyExistsException(schemaName, tableName, result.getTable());
                 }
                 return null;
-            case PARENT_TABLE_NOT_FOUND:
-                throw new TableNotFoundException(schemaName, parent.getName().getString());
             case NEWER_TABLE_FOUND:
                 // Add table to ConnectionQueryServices so it's cached, but don't add
                 // it to this connection as we can't see it.
@@ -3041,6 +2990,7 @@ public class MetaDataClient {
             }
         } finally {
             connection.setAutoCommit(wasAutoCommit);
+            deleteMutexCells(parentPhysicalSchemaName, parentPhysicalTableName, acquiredColumnMutexSet);
         }
     }
 
@@ -3183,7 +3133,7 @@ public class MetaDataClient {
                 Delete linkDelete = new Delete(linkKey, clientTimeStamp);
                 tableMetaData.add(linkDelete);
             }
-            MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade, skipAddingParentColumns);
+            MetaDataMutationResult result = connection.getQueryServices().dropTable(tableMetaData, tableType, cascade);
             MutationCode code = result.getMutationCode();
             PTable table = result.getTable();
             switch (code) {
@@ -3534,7 +3484,7 @@ public class MetaDataClient {
         String physicalTableName =
                 SchemaUtil.getTableNameFromFullName(physicalName.getString());
         Set<String> acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3);
-        boolean acquiredMutex = false;
+        boolean acquiredBaseTableMutex = false;
         try {
             connection.setAutoCommit(false);
 
@@ -3827,25 +3777,25 @@ public class MetaDataClient {
                 if (EncodedColumnsUtil.usesEncodedColumnNames(table)) {
                     // for tables that use column encoding acquire a mutex on the base table as we
                     // need to update the encoded column qualifier counter on the base table
-                    acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName, null);
-                    if (!acquiredMutex) {
+                    acquiredBaseTableMutex = writeCell(null, physicalSchemaName, physicalTableName, null);
+                    if (!acquiredBaseTableMutex) {
                         throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
                     }
                 }
-                else {
-                    for (PColumn pColumn : columns) {
-                        // acquire the mutex using the global physical table name to
-                        // prevent creating the same column on a table or view with
-                        // a conflicting type etc
-                        acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName,
-                            pColumn.getName().getString());
-                        if (!acquiredMutex) {
-                            throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
-                        }
-                        acquiredColumnMutexSet.add(pColumn.getName().getString());
+                for (PColumn pColumn : columns) {
+                    // acquire the mutex using the global physical table name to
+                    // prevent creating the same column on a table or view with
+                    // a conflicting type etc
+                    boolean acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName,
+                        pColumn.getName().getString());
+                    if (!acquiredMutex) {
+                        throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
                     }
+                    acquiredColumnMutexSet.add(pColumn.getName().getString());
                 }
-                MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, table, properties, colFamiliesForPColumnsToBeAdded, columns);
+                MetaDataMutationResult result = connection.getQueryServices().addColumn(tableMetaData, table,
+                        getParentTable(table), properties, colFamiliesForPColumnsToBeAdded, columns);
+
                 try {
                     MutationCode code = processMutationResult(schemaName, tableName, result);
                     if (code == MutationCode.COLUMN_ALREADY_EXISTS) {
@@ -3861,7 +3811,7 @@ public class MetaDataClient {
                     String fullTableName = SchemaUtil.getTableName(schemaName, tableName);
                     long resolvedTimeStamp = TransactionUtil.getResolvedTime(connection, result);
                     if (table.getIndexes().isEmpty() || (numPkColumnsAdded==0 && ! metaProperties.getNonTxToTx())) {
-                        connection.addTable(result.getTable(), resolvedTimeStamp);
+                        addTableToCache(result, resolvedTimeStamp);
                         table = result.getTable();
                     } else  {
                         // remove the table from the cache, it will be fetched from the server the
@@ -3934,16 +3884,20 @@ public class MetaDataClient {
             }
         } finally {
             connection.setAutoCommit(wasAutoCommit);
-            if (EncodedColumnsUtil.usesEncodedColumnNames(table) && acquiredMutex) {
+            if (acquiredBaseTableMutex) {
                 // release the mutex on the physical table (used to prevent concurrent conflicting
                 // add column changes)
                 deleteCell(null, physicalSchemaName, physicalTableName, null);
             }
-            else if (!acquiredColumnMutexSet.isEmpty()) {
-                for (String columnName : acquiredColumnMutexSet) {
-                    // release the mutex (used to prevent concurrent conflicting add column changes)
-                    deleteCell(null, physicalSchemaName, physicalTableName, columnName);
-                }
+            deleteMutexCells(physicalSchemaName, physicalTableName, acquiredColumnMutexSet);
+        }
+    }
+
+    private void deleteMutexCells(String physicalSchemaName, String physicalTableName, Set<String> acquiredColumnMutexSet) throws SQLException {
+        if (!acquiredColumnMutexSet.isEmpty()) {
+            for (String columnName : acquiredColumnMutexSet) {
+                // release the mutex (used to prevent concurrent conflicting add column changes)
+                deleteCell(null, physicalSchemaName, physicalTableName, columnName);
             }
         }
     }
@@ -4016,9 +3970,31 @@ public class MetaDataClient {
         return null;
     }
 
+    private PTable getParentTable(PTable table) throws SQLException {
+        PTable parentTable = null;
+        boolean hasIndexId = table.getViewIndexId() != null;
+        if ( (table.getType()==PTableType.INDEX && hasIndexId)
+                || (table.getType() == PTableType.VIEW && table.getViewType() != ViewType.MAPPED)) {
+            parentTable = PhoenixRuntime.getTable(connection, table.getParentName().getString());
+            if (parentTable==null) {
+                String schemaName = table.getSchemaName()!=null ? table.getSchemaName().getString() : null;
+                throw new TableNotFoundException(schemaName, table.getTableName().getString());
+            }
+            // only inherit columns view indexes (and not local indexes
+            // on regular tables which also have a viewIndexId)
+            if (hasIndexId && parentTable.getType() != PTableType.VIEW) {
+                return null;
+            }
+        }
+        return parentTable;
+    }
+
     public MutationState dropColumn(DropColumnStatement statement) throws SQLException {
         connection.rollback();
         boolean wasAutoCommit = connection.getAutoCommit();
+        Set<String> acquiredColumnMutexSet = Sets.newHashSetWithExpectedSize(3);
+        String physicalSchemaName = null;
+        String physicalTableName  = null;
         try {
             connection.setAutoCommit(false);
             PName tenantId = connection.getTenantId();
@@ -4031,6 +4007,9 @@ public class MetaDataClient {
                 final ColumnResolver resolver = FromCompiler.getResolver(statement, connection);
                 TableRef tableRef = resolver.getTables().get(0);
                 PTable table = tableRef.getTable();
+                PName physicalName = table.getPhysicalName();
+                physicalSchemaName = SchemaUtil.getSchemaNameFromFullName(physicalName.getString());
+                physicalTableName = SchemaUtil.getTableNameFromFullName(physicalName.getString());
 
                 List<ColumnName> columnRefs = statement.getColumnRefs();
                 if(columnRefs == null) {
@@ -4061,7 +4040,16 @@ public class MetaDataClient {
                         throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_COL_APPEND_ONLY_SCHEMA)
                         .setColumnName(columnToDrop.getName().getString()).build().buildException();
                     }
+                    else if (columnToDrop.isViewReferenced()) {
+                        throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_DROP_VIEW_REFERENCED_COL)
+                                .setColumnName(columnToDrop.getName().getString()).build().buildException();
+                    }
                     columnsToDrop.add(new ColumnRef(columnRef.getTableRef(), columnToDrop.getPosition()));
+                    boolean acquiredMutex = writeCell(null, physicalSchemaName, physicalTableName, columnToDrop.getName().getString());
+                    if (!acquiredMutex) {
+                        throw new ConcurrentTableMutationException(physicalSchemaName, physicalTableName);
+                    }
+                    acquiredColumnMutexSet.add(columnToDrop.getName().getString());
                 }
 
                 dropColumnMutations(table, tableColumnsToDrop);
@@ -4138,12 +4126,14 @@ public class MetaDataClient {
                                     Collections.<Mutation>singletonList(new Put(SchemaUtil.getTableKey
                                             (tenantIdBytes, tableContainingColumnToDrop.getSchemaName().getBytes(),
                                                     tableContainingColumnToDrop.getTableName().getBytes()))),
-                                                    tableContainingColumnToDrop, family, Sets.newHashSet(Bytes.toString(emptyCF)), Collections.<PColumn>emptyList());
+                                                    tableContainingColumnToDrop, null, family, Sets.newHashSet(Bytes.toString(emptyCF)), Collections.<PColumn>emptyList());
 
                         }
                     }
                 }
-                MetaDataMutationResult result = connection.getQueryServices().dropColumn(tableMetaData, statement.getTableType());
+
+                MetaDataMutationResult result = connection.getQueryServices().dropColumn(tableMetaData,
+                        statement.getTableType(), getParentTable(table));
                 try {
                     MutationCode code = processMutationResult(schemaName, tableName, result);
                     if (code == MutationCode.COLUMN_NOT_FOUND) {
@@ -4288,6 +4278,7 @@ public class MetaDataClient {
             }
         } finally {
             connection.setAutoCommit(wasAutoCommit);
+            deleteMutexCells(physicalSchemaName, physicalTableName, acquiredColumnMutexSet);
         }
     }
 
@@ -4441,9 +4432,13 @@ public class MetaDataClient {
     }
 
     private PTable addTableToCache(MetaDataMutationResult result) throws SQLException {
-        addIndexesFromParentTable(result, null, false);
+        return addTableToCache(result, TransactionUtil.getResolvedTime(connection, result));
+    }
+
+    private PTable addTableToCache(MetaDataMutationResult result, long timestamp) throws SQLException {
+        addColumnsAndIndexesFromAncestors(result, null, false);
         PTable table = result.getTable();
-        connection.addTable(table, TransactionUtil.getResolvedTime(connection, result));
+        connection.addTable(table, timestamp);
         return table;
     }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
index a11c31a..365ff5c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/MetaDataSplitPolicy.java
@@ -26,8 +26,12 @@ public class MetaDataSplitPolicy extends SplitOnLeadingVarCharColumnsPolicy {
 	@Override
 	protected boolean shouldSplit() {
 		Configuration conf = getConf();
-		return super.shouldSplit() && conf.getBoolean(QueryServices.SYSTEM_CATALOG_SPLITTABLE,
+		boolean allowSplittableSystemCatalogRollback = conf.getBoolean
+				(QueryServices.ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK,
+						QueryServicesOptions.DEFAULT_ALLOW_SPLITTABLE_SYSTEM_CATALOG_ROLLBACK);
+		boolean allowSystemCatalogToSplit = conf.getBoolean(QueryServices.SYSTEM_CATALOG_SPLITTABLE,
 				QueryServicesOptions.DEFAULT_SYSTEM_CATALOG_SPLITTABLE);
+		return super.shouldSplit() && allowSystemCatalogToSplit && !allowSplittableSystemCatalogRollback;
 	}
 
 	@Override
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
index 5f499d8..5c0e12c 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/schema/PTableImpl.java
@@ -1603,18 +1603,15 @@ public class PTableImpl implements PTable {
         }
         ViewType viewType = null;
         String viewStatement = null;
-        List<PName> physicalNames = Collections.emptyList();
         if (tableType == PTableType.VIEW) {
             viewType = ViewType.fromSerializedValue(table.getViewType().toByteArray()[0]);
         }
         if(table.hasViewStatement()){
             viewStatement = (String) PVarchar.INSTANCE.toObject(table.getViewStatement().toByteArray());
         }
-        if (tableType == PTableType.VIEW || viewIndexId != null) {
-            physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
-            for(int i = 0; i < table.getPhysicalNamesCount(); i++) {
-                physicalNames.add(PNameFactory.newName(table.getPhysicalNames(i).toByteArray()));
-            }
+        List<PName> physicalNames = Lists.newArrayListWithExpectedSize(table.getPhysicalNamesCount());
+        for(int i = 0; i < table.getPhysicalNamesCount(); i++) {
+            physicalNames.add(PNameFactory.newName(table.getPhysicalNames(i).toByteArray()));
         }
         int baseColumnCount = -1;
         if (table.hasBaseColumnCount()) {
@@ -1667,6 +1664,15 @@ public class PTableImpl implements PTable {
         if (table.hasUseStatsForParallelization()) {
             useStatsForParallelization = table.getUseStatsForParallelization();
         }
+        // for older clients just use the value of the properties that are set on the view
+        boolean viewModifiedUpdateCacheFrequency = true;
+        boolean viewModifiedUseStatsForParallelization = true;
+        if (table.hasViewModifiedUpdateCacheFrequency()) {
+            viewModifiedUpdateCacheFrequency = table.getViewModifiedUpdateCacheFrequency();
+        }
+        if (table.hasViewModifiedUseStatsForParallelization()) {
+            viewModifiedUseStatsForParallelization = table.getViewModifiedUseStatsForParallelization();
+        }
         try {
             return new PTableImpl.Builder()
                     .setType(tableType)
@@ -1711,6 +1717,8 @@ public class PTableImpl implements PTable {
                     .setPhysicalNames(physicalNames == null ?
                             ImmutableList.<PName>of() : ImmutableList.copyOf(physicalNames))
                     .setColumns(columns)
+                    .setViewModifiedUpdateCacheFrequency(viewModifiedUpdateCacheFrequency)
+                    .setViewModifiedUseStatsForParallelization(viewModifiedUseStatsForParallelization)
                     .build();
         } catch (SQLException e) {
             throw new RuntimeException(e); // Impossible
@@ -1784,10 +1792,8 @@ public class PTableImpl implements PTable {
       if(table.getViewStatement()!=null){
         builder.setViewStatement(ByteStringer.wrap(PVarchar.INSTANCE.toBytes(table.getViewStatement())));
       }
-      if(table.getType() == PTableType.VIEW || table.getViewIndexId() != null){
-        for (int i = 0; i < table.getPhysicalNames().size(); i++) {
-          builder.addPhysicalNames(ByteStringer.wrap(table.getPhysicalNames().get(i).getBytes()));
-        }
+      for (int i = 0; i < table.getPhysicalNames().size(); i++) {
+        builder.addPhysicalNames(ByteStringer.wrap(table.getPhysicalNames().get(i).getBytes()));
       }
       builder.setBaseColumnCount(table.getBaseColumnCount());
       builder.setRowKeyOrderOptimizable(table.rowKeyOrderOptimizable());
@@ -1816,6 +1822,8 @@ public class PTableImpl implements PTable {
       if (table.useStatsForParallelization() != null) {
           builder.setUseStatsForParallelization(table.useStatsForParallelization());
       }
+      builder.setViewModifiedUpdateCacheFrequency(table.hasViewModifiedUpdateCacheFrequency());
+      builder.setViewModifiedUseStatsForParallelization(table.hasViewModifiedUseStatsForParallelization());
       return builder.build();
     }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java b/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
deleted file mode 100644
index 2b66559..0000000
--- a/phoenix-core/src/main/java/org/apache/phoenix/schema/ParentTableNotFoundException.java
+++ /dev/null
@@ -1,47 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.phoenix.schema;
-
-import org.apache.phoenix.coprocessor.TableInfo;
-
-public class ParentTableNotFoundException extends TableNotFoundException {
-    private static final long serialVersionUID = 1L;
-    private final byte[] parentTenantId;
-    private final byte[] parentSchemaName;
-    private final byte[] parentTableName;
-
-    public ParentTableNotFoundException(TableInfo parentTableInfo, String tableName) {
-        super(tableName);
-        this.parentTenantId = parentTableInfo.getTenantId();
-        this.parentSchemaName = parentTableInfo.getSchemaName();
-        this.parentTableName = parentTableInfo.getTableName();
-    }
-
-    public byte[] getParentTenantId() {
-        return parentTenantId;
-    }
-
-    public byte[] getParentSchemaName() {
-        return parentSchemaName;
-    }
-
-    public byte[] getParentTableName() {
-        return parentTableName;
-    }
-
-}
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
index 3e41997..239018b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/MetaDataUtil.java
@@ -17,6 +17,8 @@
  */
 package org.apache.phoenix.util;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.COLUMN_NAME_INDEX;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.FAMILY_NAME_INDEX;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
 import java.io.IOException;
@@ -63,7 +65,10 @@ import org.apache.phoenix.hbase.index.util.VersionUtil;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnFamilyNotFoundException;
+import org.apache.phoenix.schema.ColumnNotFoundException;
 import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnFamily;
 import org.apache.phoenix.schema.PName;
 import org.apache.phoenix.schema.PNameFactory;
 import org.apache.phoenix.schema.PTable;
@@ -382,16 +387,6 @@ public class MetaDataUtil {
         return false;
     }
 
-    
-    public static ViewType getViewType(List<Mutation> tableMetaData, KeyValueBuilder builder,
-    	      ImmutableBytesWritable value) {
-    	        if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
-    	            PhoenixDatabaseMetaData.VIEW_TYPE_BYTES, builder, value)) {
-    	            return ViewType.fromSerializedValue(value.get()[value.getOffset()]);
-    	        }
-    	        return null;
-    	    }
-    
     public static int getSaltBuckets(List<Mutation> tableMetaData, KeyValueBuilder builder,
       ImmutableBytesWritable value) {
         if (getMutationValue(getPutOnlyTableHeaderRow(tableMetaData),
@@ -939,4 +934,19 @@ public class MetaDataUtil {
                 value)) { return IndexType.fromSerializedValue(value.get()[value.getOffset()]); }
         return null;
     }
+
+    public static PColumn getColumn(int pkCount, byte[][] rowKeyMetaData, PTable table) throws ColumnFamilyNotFoundException, ColumnNotFoundException {
+        PColumn col = null;
+        if (pkCount > FAMILY_NAME_INDEX
+            && rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX].length > 0) {
+            PColumnFamily family =
+                table.getColumnFamily(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]);
+            col =
+                family.getPColumnForColumnNameBytes(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]);
+        } else if (pkCount > COLUMN_NAME_INDEX
+            && rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX].length > 0) {
+            col = table.getPKColumn(new String(rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX]));
+        }
+        return col;
+    }
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
index 24a0e12..b2140e7 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/SchemaUtil.java
@@ -278,7 +278,8 @@ public class SchemaUtil {
      * @param tableName
      */
     public static byte[] getTableKey(byte[] tenantId, byte[] schemaName, byte[] tableName) {
-        return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId, SEPARATOR_BYTE_ARRAY, schemaName, SEPARATOR_BYTE_ARRAY, tableName);
+        return ByteUtil.concat(tenantId == null ? ByteUtil.EMPTY_BYTE_ARRAY : tenantId, SEPARATOR_BYTE_ARRAY,
+                schemaName == null ? ByteUtil.EMPTY_BYTE_ARRAY : schemaName, SEPARATOR_BYTE_ARRAY, tableName);
     }
 
     /**
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java b/phoenix-core/src/main/java/org/apache/phoenix/util/TableViewFinderResult.java
similarity index 94%
rename from phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
rename to phoenix-core/src/main/java/org/apache/phoenix/util/TableViewFinderResult.java
index 269c5cb..5d693ab 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/TableViewFinderResult.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/TableViewFinderResult.java
@@ -13,11 +13,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.phoenix.coprocessor;
+package org.apache.phoenix.util;
 
 import java.util.List;
 
 import com.google.common.collect.Lists;
+import org.apache.phoenix.coprocessor.TableInfo;
 
 /**
  * This class wraps the results of a scanning SYSTEM.CATALOG or SYSTEM.CHILD_LINK
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
index b79bfe6..254f78e 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/UpgradeUtil.java
@@ -94,8 +94,6 @@ import org.apache.phoenix.coprocessor.MetaDataProtocol;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MetaDataMutationResult;
 import org.apache.phoenix.coprocessor.MetaDataProtocol.MutationCode;
 import org.apache.phoenix.coprocessor.TableInfo;
-import org.apache.phoenix.coprocessor.TableViewFinderResult;
-import org.apache.phoenix.coprocessor.ViewFinder;
 import org.apache.phoenix.jdbc.PhoenixConnection;
 import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
 import org.apache.phoenix.query.ConnectionQueryServices;
@@ -1355,7 +1353,7 @@ public class UpgradeUtil {
                             PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
                             newConn.getQueryServices().getProps())
                             .getName())) {
-                ViewFinder.findAllRelatives(childLinkTable, tenantId,
+                ViewUtil.findAllRelatives(childLinkTable, tenantId,
                         table.getSchemaName().getBytes(), table.getTableName().getBytes(),
                         LinkType.CHILD_TABLE, childViewsResult);
 
@@ -2118,7 +2116,7 @@ public class UpgradeUtil {
                                 PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, readOnlyProps)
                                     .getName())) {
                 byte[] tenantId = conn.getTenantId() != null ? conn.getTenantId().getBytes() : null;
-                ViewFinder.findAllRelatives(childLinkTable, tenantId, schemaName.getBytes(),
+                ViewUtil.findAllRelatives(childLinkTable, tenantId, schemaName.getBytes(),
                     tableName.getBytes(), LinkType.CHILD_TABLE, childViewsResult);
             }
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java
new file mode 100644
index 0000000..6badf1f
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/util/ViewUtil.java
@@ -0,0 +1,578 @@
+/**
+ * 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.util;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.client.Put;
+import org.apache.hadoop.hbase.client.Result;
+import org.apache.hadoop.hbase.client.ResultScanner;
+import org.apache.hadoop.hbase.client.Scan;
+import org.apache.hadoop.hbase.client.Table;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.hadoop.hbase.filter.CompareFilter;
+import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.Pair;
+import org.apache.phoenix.compile.ColumnNameTrackingExpressionCompiler;
+import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
+import org.apache.phoenix.coprocessor.MetaDataProtocol;
+import org.apache.phoenix.coprocessor.TableInfo;
+import org.apache.phoenix.coprocessor.WhereConstantParser;
+import org.apache.phoenix.index.IndexMaintainer;
+import org.apache.phoenix.jdbc.PhoenixConnection;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.parse.DropTableStatement;
+import org.apache.phoenix.parse.ParseNode;
+import org.apache.phoenix.parse.SQLParser;
+import org.apache.phoenix.query.QueryConstants;
+import org.apache.phoenix.schema.ColumnNotFoundException;
+import org.apache.phoenix.schema.MetaDataClient;
+import org.apache.phoenix.schema.PColumn;
+import org.apache.phoenix.schema.PColumnImpl;
+import org.apache.phoenix.schema.PName;
+import org.apache.phoenix.schema.PNameFactory;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTable.LinkType;
+import org.apache.phoenix.schema.PTableImpl;
+import org.apache.phoenix.schema.PTableType;
+import org.apache.phoenix.schema.SaltingUtil;
+import org.apache.phoenix.schema.TableNotFoundException;
+import org.apache.phoenix.schema.types.PBoolean;
+import org.apache.phoenix.schema.types.PLong;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.LINK_TYPE_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PARENT_TENANT_ID_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_TYPE_BYTES;
+import static org.apache.phoenix.schema.PTableImpl.getColumnsToClone;
+import static org.apache.phoenix.util.SchemaUtil.getVarChars;
+
+public class ViewUtil {
+
+    private static final Logger logger = LoggerFactory.getLogger(ViewUtil.class);
+
+    public static void findAllRelatives(Table systemTable, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, TableViewFinderResult result) throws IOException {
+        findAllRelatives(systemTable, tenantId, schema, table, linkType, HConstants.LATEST_TIMESTAMP, result);
+    }
+
+    static void findAllRelatives(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, long timestamp, TableViewFinderResult result) throws IOException {
+        TableViewFinderResult currentResult =
+            findRelatedViews(systemCatalog, tenantId, schema, table, linkType, timestamp);
+        result.addResult(currentResult);
+        for (TableInfo viewInfo : currentResult.getLinks()) {
+            findAllRelatives(systemCatalog, viewInfo.getTenantId(), viewInfo.getSchemaName(), viewInfo.getTableName(), linkType, timestamp, result);
+        }
+    }
+
+    /**
+     * Runs a scan on SYSTEM.CATALOG or SYSTEM.CHILD_LINK to get the related tables/views
+     */
+    static TableViewFinderResult findRelatedViews(Table systemCatalog, byte[] tenantId, byte[] schema, byte[] table,
+        PTable.LinkType linkType, long timestamp) throws IOException {
+        if (linkType==PTable.LinkType.INDEX_TABLE || linkType==PTable.LinkType.EXCLUDED_COLUMN) {
+            throw new IllegalArgumentException("findAllRelatives does not support link type "+linkType);
+        }
+        byte[] key = SchemaUtil.getTableKey(tenantId, schema, table);
+		Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        SingleColumnValueFilter linkFilter =
+            new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES, CompareFilter.CompareOp.EQUAL,
+                linkType.getSerializedValueAsByteArray());
+        linkFilter.setFilterIfMissing(true);
+        scan.setFilter(linkFilter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        if (linkType==PTable.LinkType.PARENT_TABLE)
+            scan.addColumn(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+        if (linkType==PTable.LinkType.PHYSICAL_TABLE)
+            scan.addColumn(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES);
+        List<TableInfo> tableInfoList = Lists.newArrayList();
+        try (ResultScanner scanner = systemCatalog.getScanner(scan))  {
+            for (Result result = scanner.next(); (result != null); result = scanner.next()) {
+                byte[][] rowKeyMetaData = new byte[5][];
+                byte[] viewTenantId = null;
+                getVarChars(result.getRow(), 5, rowKeyMetaData);
+                if (linkType==PTable.LinkType.PARENT_TABLE) {
+                    viewTenantId = result.getValue(TABLE_FAMILY_BYTES, PARENT_TENANT_ID_BYTES);
+                } else if (linkType==PTable.LinkType.CHILD_TABLE) {
+                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
+                } else if (linkType==PTable.LinkType.VIEW_INDEX_PARENT_TABLE) {
+                    viewTenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
+                } 
+                else if (linkType==PTable.LinkType.PHYSICAL_TABLE && result.getValue(TABLE_FAMILY_BYTES, TABLE_TYPE_BYTES)!=null) {
+                    // do not links from indexes to their physical table
+                    continue;
+                }
+                byte[] viewSchemaName = SchemaUtil.getSchemaNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
+                byte[] viewName = SchemaUtil.getTableNameFromFullName(rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX]).getBytes();
+                tableInfoList.add(new TableInfo(viewTenantId, viewSchemaName, viewName));
+            }
+            return new TableViewFinderResult(tableInfoList);
+        } 
+    }
+    
+    /**
+     * @return true if the given table has at least one child view
+     * @throws IOException 
+     */
+    public static boolean hasChildViews(Table systemCatalog, byte[] tenantId, byte[] schemaName, byte[] tableName, long timestamp) throws IOException {
+        byte[] key = SchemaUtil.getTableKey(tenantId, schemaName, tableName);
+        Scan scan = MetaDataUtil.newTableRowsScan(key, MetaDataProtocol.MIN_TABLE_TIMESTAMP, timestamp);
+        SingleColumnValueFilter linkFilter =
+                new SingleColumnValueFilter(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES,
+                        CompareFilter.CompareOp.EQUAL,
+                        LinkType.CHILD_TABLE.getSerializedValueAsByteArray()) {
+                    // if we found a row with the CHILD_TABLE link type we are done and can
+                    // terminate the scan
+                    @Override
+                    public boolean filterAllRemaining() throws IOException {
+                        return matchedColumn;
+                    }
+                };
+        linkFilter.setFilterIfMissing(true);
+        scan.setFilter(linkFilter);
+        scan.addColumn(TABLE_FAMILY_BYTES, LINK_TYPE_BYTES);
+        try (ResultScanner scanner = systemCatalog.getScanner(scan)) {
+            Result result = scanner.next();
+            return result!=null; 
+        }
+    }
+
+    public static void dropChildViews(RegionCoprocessorEnvironment env, byte[] tenantIdBytes, byte[] schemaName, byte[] tableName)
+            throws IOException, SQLException, ClassNotFoundException {
+        Table hTable = null;
+        try {
+            hTable =
+                    ServerUtil.getHTableForCoprocessorScan(env,
+                            SchemaUtil.getPhysicalTableName(
+                                    PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
+                                    env.getConfiguration()).getName());
+        }
+        catch (Exception e){
+        }
+        // if the SYSTEM.CHILD_LINK doesn't exist just return
+        if (hTable==null) {
+            return;
+        }
+
+        TableViewFinderResult childViewsResult = findRelatedViews(hTable, tenantIdBytes, schemaName, tableName,
+                LinkType.CHILD_TABLE, HConstants.LATEST_TIMESTAMP);
+
+        for (TableInfo viewInfo : childViewsResult.getLinks()) {
+            byte[] viewTenantId = viewInfo.getTenantId();
+            byte[] viewSchemaName = viewInfo.getSchemaName();
+            byte[] viewName = viewInfo.getTableName();
+            if (logger.isDebugEnabled()) {
+                logger.debug("dropChildViews :" + Bytes.toString(schemaName) + "." + Bytes.toString(tableName) +
+                        " -> " + Bytes.toString(viewSchemaName) + "." + Bytes.toString(viewName) +
+                        "with tenant id :" + Bytes.toString(viewTenantId));
+            }
+            Properties props = new Properties();
+            if (viewTenantId != null && viewTenantId.length != 0)
+                props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, Bytes.toString(viewTenantId));
+            try (PhoenixConnection connection = QueryUtil.getConnectionOnServer(props, env.getConfiguration())
+                    .unwrap(PhoenixConnection.class)) {
+                MetaDataClient client = new MetaDataClient(connection);
+                org.apache.phoenix.parse.TableName viewTableName = org.apache.phoenix.parse.TableName
+                        .create(Bytes.toString(viewSchemaName), Bytes.toString(viewName));
+                try {
+                    client.dropTable(
+                            new DropTableStatement(viewTableName, PTableType.VIEW, true, true, true));
+                }
+                catch (TableNotFoundException e) {
+                    logger.info("Ignoring view "+viewTableName+" as it has already been dropped");
+                }
+            }
+        }
+    }
+
+    public static boolean isDivergedView(PTable view) {
+        return view.getBaseColumnCount() == QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT;
+    }
+
+    /**
+     * Adds indexes of the parent table to inheritedIndexes if the index contains all required columns
+     */
+    public static void addIndexesFromParent(PhoenixConnection connection, PTable view,
+                                            PTable parentTable, List<PTable> inheritedIndexes) throws SQLException {
+        List<PTable> parentTableIndexes = parentTable.getIndexes();
+        for (PTable index : parentTableIndexes) {
+            boolean containsAllReqdCols = true;
+            // Ensure that all columns required to create index exist in the view too,
+            // since view columns may be removed.
+            IndexMaintainer indexMaintainer = index.getIndexMaintainer(parentTable, connection);
+            // Check that the columns required for the index pk are present in the view
+            Set<Pair<String, String>> indexedColInfos = indexMaintainer.getIndexedColumnInfo();
+            for (Pair<String, String> colInfo : indexedColInfos) {
+                try {
+                    String colFamily = colInfo.getFirst();
+                    String colName = colInfo.getSecond();
+                    if (colFamily == null) {
+                        view.getColumnForColumnName(colName);
+                    } else {
+                        view.getColumnFamily(colFamily).getPColumnForColumnName(colName);
+                    }
+                } catch (ColumnNotFoundException e) {
+                    containsAllReqdCols = false;
+                    break;
+                }
+            }
+
+            // Ensure that constant columns (i.e. columns matched in the view WHERE clause)
+            // all exist in the index on the parent table.
+            for (PColumn col : view.getColumns()) {
+                if (col.getViewConstant() != null) {
+                    try {
+                        // It'd be possible to use a local index that doesn't have all view constants,
+                        // but the WHERE clause for the view statement (which is added to the index below)
+                        // would fail to compile.
+                        String indexColumnName = IndexUtil.getIndexColumnName(col);
+                        index.getColumnForColumnName(indexColumnName);
+                    } catch (ColumnNotFoundException e1) {
+                        PColumn indexCol = null;
+                        try {
+                            String cf = col.getFamilyName()!=null ? col.getFamilyName().getString() : null;
+                            String colName = col.getName().getString();
+                            if (cf != null) {
+                                indexCol = parentTable.getColumnFamily(cf).getPColumnForColumnName(colName);
+                            }
+                            else {
+                                indexCol = parentTable.getColumnForColumnName(colName);
+                            }
+                        } catch (ColumnNotFoundException e2) { // Ignore this index and continue with others
+                            containsAllReqdCols = false;
+                            break;
+                        }
+                        if (indexCol.getViewConstant()==null || Bytes.compareTo(indexCol.getViewConstant(), col.getViewConstant())!=0) {
+                            containsAllReqdCols = false;
+                            break;
+                        }
+                    }
+                }
+            }
+            if (containsAllReqdCols) {
+                // Tack on view statement to index to get proper filtering for view
+                String viewStatement = IndexUtil.rewriteViewStatement(connection, index, parentTable, view.getViewStatement());
+                PName modifiedIndexName = PNameFactory.newName(view.getName().getString()
+                        + QueryConstants.CHILD_VIEW_INDEX_NAME_SEPARATOR + index.getName().getString());
+                // add the index table with a new name so that it does not conflict with the existing index table
+                // and set update cache frequency to that of the view
+                if (Objects.equal(viewStatement, index.getViewStatement())) {
+                    inheritedIndexes.add(index);
+                } else {
+                    inheritedIndexes.add(PTableImpl.builderWithColumns(index, getColumnsToClone(index))
+                            .setTableName(modifiedIndexName)
+                            .setViewStatement(viewStatement)
+                            .setUpdateCacheFrequency(view.getUpdateCacheFrequency())
+                            .setTenantId(view.getTenantId())
+                            .setPhysicalNames(Collections.singletonList(index.getPhysicalName()))
+                            .build());
+                }
+            }
+        }
+    }
+
+    /**
+     * Inherit all columns from the parent unless its an excluded column if the same columns is present in the parent
+     * and child (for table metadata created before PHOENIX-3534) we chose the child column over the parent column
+     * @return table with inherited columns and indexes
+     */
+    public static PTable addDerivedColumnsAndIndexesFromParent(PhoenixConnection connection,
+                                                               PTable table, PTable parentTable) throws SQLException {
+        // combine columns for view and view indexes
+        boolean hasIndexId = table.getViewIndexId() != null;
+        boolean isSalted = table.getBucketNum() != null;
+        boolean isDiverged = isDivergedView(table);
+        List<PColumn> allColumns = Lists.newArrayList();
+        List<PColumn> excludedColumns = Lists.newArrayList();
+        // add my own columns first in reverse order
+        List<PColumn> myColumns = table.getColumns();
+        // skip salted column as it will be created automatically
+        myColumns = myColumns.subList(isSalted ? 1 : 0, myColumns.size());
+        for (int i = myColumns.size() - 1; i >= 0; i--) {
+            PColumn pColumn = myColumns.get(i);
+            if (pColumn.isExcluded()) {
+                excludedColumns.add(pColumn);
+            }
+            allColumns.add(pColumn);
+        }
+
+        // initialize map from with indexed expression to list of required data columns
+        // then remove the data columns that have not been dropped, so that we get the columns that
+        // have been dropped
+        Map<PColumn, List<String>> indexRequiredDroppedDataColMap =
+                Maps.newHashMapWithExpectedSize(table.getColumns().size());
+        if (hasIndexId) {
+            int indexPosOffset = (isSalted ? 1 : 0) + (table.isMultiTenant() ? 1 : 0) + 1;
+            ColumnNameTrackingExpressionCompiler expressionCompiler =
+                    new ColumnNameTrackingExpressionCompiler();
+            for (int i = indexPosOffset; i < table.getPKColumns().size(); i++) {
+                PColumn indexColumn = table.getPKColumns().get(i);
+                try {
+                    expressionCompiler.reset();
+                    String expressionStr = IndexUtil.getIndexColumnExpressionStr(indexColumn);
+                    ParseNode parseNode = SQLParser.parseCondition(expressionStr);
+                    parseNode.accept(expressionCompiler);
+                    indexRequiredDroppedDataColMap.put(indexColumn,
+                            Lists.newArrayList(expressionCompiler.getDataColumnNames()));
+                } catch (SQLException e) {
+                    throw new RuntimeException(e); // Impossible
+                }
+            }
+        }
+
+        long maxTableTimestamp = table.getTimeStamp();
+        int numPKCols = table.getPKColumns().size();
+        // set the final table timestamp as the max timestamp of the view/view index or its ancestors
+        maxTableTimestamp = Math.max(maxTableTimestamp, parentTable.getTimeStamp());
+        if (hasIndexId) {
+            // add all pk columns of parent tables to indexes
+            // skip salted column as it will be added from the base table columns
+            int startIndex = parentTable.getBucketNum() != null ? 1 : 0;
+            for (int index=startIndex; index<parentTable.getPKColumns().size(); index++) {
+                PColumn pkColumn = parentTable.getPKColumns().get(index);
+                // don't add the salt column of ancestor tables for view indexes, or deleted columns
+                // or constant columns from the view where statement
+                if (pkColumn.equals(SaltingUtil.SALTING_COLUMN) || pkColumn.isExcluded()
+                        || pkColumn.getViewConstant()!=null) {
+                    continue;
+                }
+                pkColumn = IndexUtil.getIndexPKColumn(++numPKCols, pkColumn);
+                int existingColumnIndex = allColumns.indexOf(pkColumn);
+                if (existingColumnIndex == -1) {
+                    allColumns.add(0, pkColumn);
+                }
+            }
+            for (int j = 0; j < parentTable.getColumns().size(); j++) {
+                PColumn tableColumn = parentTable.getColumns().get(j);
+                if (tableColumn.isExcluded()) {
+                    continue;
+                }
+                String dataColumnName = tableColumn.getName().getString();
+                // remove from list of columns since it has not been dropped
+                for (Map.Entry<PColumn, List<String>> entry : indexRequiredDroppedDataColMap
+                        .entrySet()) {
+                    entry.getValue().remove(dataColumnName);
+                }
+            }
+        } else {
+            List<PColumn> currAncestorTableCols = PTableImpl.getColumnsToClone(parentTable);
+            if (currAncestorTableCols != null) {
+                // add the ancestor columns in reverse order so that the final column list
+                // contains ancestor columns and then the view columns in the right order
+                for (int j = currAncestorTableCols.size() - 1; j >= 0; j--) {
+                    PColumn column = currAncestorTableCols.get(j);
+                    // for diverged views we always include pk columns of the base table. We
+                    // have to include these pk columns to be able to support adding pk
+                    // columns to the diverged view
+                    // we only include regular columns that were created before the view
+                    // diverged
+                    if (isDiverged && column.getFamilyName() != null
+                            && column.getTimestamp() > table.getTimeStamp()) {
+                        continue;
+                    }
+                    // need to check if this column is in the list of excluded (dropped)
+                    // columns of the view
+                    int existingIndex = excludedColumns.indexOf(column);
+                    if (existingIndex != -1) {
+                        // if it is, only exclude the column if was created before the
+                        // column was dropped in the view in order to handle the case where
+                        // a base table column is dropped in a view, then dropped in the
+                        // base table and then added back to the base table
+                        if (column.getTimestamp() <= excludedColumns.get(existingIndex)
+                                .getTimestamp()) {
+                            continue;
+                        }
+                    }
+                    if (column.isExcluded()) {
+                        excludedColumns.add(column);
+                    } else {
+                        int existingColumnIndex = allColumns.indexOf(column);
+                        if (existingColumnIndex != -1) {
+                            // for diverged views if the view was created before
+                            // PHOENIX-3534 the parent table columns will be present in the
+                            // view PTable (since the base column count is
+                            // QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT we can't
+                            // filter them out) so we always pick the parent column
+                            // for non diverged views if the same column exists in a parent
+                            // and child, we keep the latest column
+                            PColumn existingColumn = allColumns.get(existingColumnIndex);
+                            if (isDiverged || column.getTimestamp() > existingColumn.getTimestamp()) {
+                                allColumns.remove(existingColumnIndex);
+                                allColumns.add(column);
+                            }
+                        } else {
+                            allColumns.add(column);
+                        }
+                    }
+                }
+            }
+        }
+        // at this point indexRequiredDroppedDataColMap only contain the columns required by a view
+        // index that have dropped
+        for (Map.Entry<PColumn, List<String>> entry : indexRequiredDroppedDataColMap.entrySet()) {
+            if (!entry.getValue().isEmpty()) {
+                PColumn indexColumnToBeDropped = entry.getKey();
+                if (SchemaUtil.isPKColumn(indexColumnToBeDropped)) {
+                    // if an indexed column was dropped in an ancestor then we
+                    // cannot use this index an more
+                    // TODO figure out a way to actually drop this view index
+                    return null;
+                } else {
+                    allColumns.remove(indexColumnToBeDropped);
+                }
+            }
+        }
+        // remove the excluded columns if the timestamp of the excludedColumn is newer
+        for (PColumn excludedColumn : excludedColumns) {
+            int index = allColumns.indexOf(excludedColumn);
+            if (index != -1) {
+                if (allColumns.get(index).getTimestamp() <= excludedColumn.getTimestamp()) {
+                    allColumns.remove(excludedColumn);
+                }
+            }
+        }
+        List<PColumn> columnsToAdd = Lists.newArrayList();
+        int position = isSalted ? 1 : 0;
+        // allColumns contains the columns in the reverse order
+        for (int i = allColumns.size() - 1; i >= 0; i--) {
+            PColumn column = allColumns.get(i);
+            if (table.getColumns().contains(column)) {
+                // for views this column is not derived from an ancestor
+                columnsToAdd.add(new PColumnImpl(column, position++));
+            } else {
+                columnsToAdd.add(new PColumnImpl(column, true, position++));
+            }
+        }
+        // we need to include the salt column when setting the base table column count in order to
+        // maintain b/w compatibility
+        int baseTableColumnCount =
+                isDiverged ? QueryConstants.DIVERGED_VIEW_BASE_COLUMN_COUNT
+                        : columnsToAdd.size() - myColumns.size() + (isSalted ? 1 : 0);
+        // Inherit view-modifiable properties from the parent table/view if the current view has
+        // not previously modified this property
+        Long updateCacheFreq = (table.getType() != PTableType.VIEW ||
+                table.hasViewModifiedUpdateCacheFrequency()) ?
+                table.getUpdateCacheFrequency() : parentTable.getUpdateCacheFrequency();
+        Boolean useStatsForParallelization = (table.getType() != PTableType.VIEW ||
+                table.hasViewModifiedUseStatsForParallelization()) ?
+                table.useStatsForParallelization() : parentTable.useStatsForParallelization();
+
+        // When creating a PTable for views or view indexes, use the baseTable PTable for attributes
+        // inherited from the physical base table.
+        // if a TableProperty is not valid on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is not mutable on a view we set it to the base table value
+        // if a TableProperty is valid on a view and is mutable on a view, we use the value set
+        // on the view if the view had previously modified the property, otherwise we propagate the
+        // value from the base table (see PHOENIX-4763)
+        PTable pTable = PTableImpl.builderWithColumns(table, columnsToAdd)
+                .setImmutableRows(parentTable.isImmutableRows())
+                .setDisableWAL(parentTable.isWALDisabled())
+                .setMultiTenant(parentTable.isMultiTenant())
+                .setStoreNulls(parentTable.getStoreNulls())
+                .setTransactionProvider(parentTable.getTransactionProvider())
+                .setAutoPartitionSeqName(parentTable.getAutoPartitionSeqName())
+                .setAppendOnlySchema(parentTable.isAppendOnlySchema())
+                .setImmutableStorageScheme(parentTable.getImmutableStorageScheme() == null ?
+                        PTable.ImmutableStorageScheme.ONE_CELL_PER_COLUMN : parentTable.getImmutableStorageScheme())
+                .setQualifierEncodingScheme(parentTable.getEncodingScheme() == null ?
+                        PTable.QualifierEncodingScheme.NON_ENCODED_QUALIFIERS : parentTable.getEncodingScheme())
+                .setBaseColumnCount(baseTableColumnCount)
+                .setTimeStamp(maxTableTimestamp)
+                .setExcludedColumns(excludedColumns == null ?
+                        ImmutableList.<PColumn>of() : ImmutableList.copyOf(excludedColumns))
+                .setUpdateCacheFrequency(updateCacheFreq)
+                .setUseStatsForParallelization(useStatsForParallelization)
+                .build();
+        pTable = WhereConstantParser.addViewInfoToPColumnsIfNeeded(pTable);
+
+        // For views :
+        if (!hasIndexId) {
+            // 1. need to resolve the views's own indexes so that any columns added by ancestors are included
+            List<PTable> allIndexes = Lists.newArrayList();
+            if (!pTable.getIndexes().isEmpty()) {
+                for (PTable viewIndex : pTable.getIndexes()) {
+                    PTable resolvedViewIndex =
+                            ViewUtil.addDerivedColumnsAndIndexesFromParent(connection, viewIndex, pTable);
+                    if (resolvedViewIndex!=null)
+                        allIndexes.add(resolvedViewIndex);
+                }
+            }
+            // 2. include any indexes from ancestors that can be used by this view
+            List<PTable> inheritedIndexes = Lists.newArrayList();
+            addIndexesFromParent(connection, pTable, parentTable, inheritedIndexes);
+            allIndexes.addAll(inheritedIndexes);
+            if (!allIndexes.isEmpty()) {
+                pTable = PTableImpl.builderWithColumns(pTable, getColumnsToClone(pTable))
+                        .setIndexes(allIndexes).build();
+            }
+        }
+        return pTable;
+    }
+
+    /**
+     * See PHOENIX-4763. If we are modifying any table-level properties that are mutable on a view,
+     * we mark these cells in SYSTEM.CATALOG with tags to indicate that this view property should
+     * not be kept in-sync with the base table and so we shouldn't propagate the base table's
+     * property value when resolving the view
... 140 lines suppressed ...