You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by ch...@apache.org on 2019/10/01 01:36:32 UTC

[phoenix] branch 4.x-HBase-1.5 updated: PHOENIX-4810: Send parent->child link mutations to SYSTEM.CHILD_LINK table in MetdataClient.createTableInternal

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

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


The following commit(s) were added to refs/heads/4.x-HBase-1.5 by this push:
     new 0db6313  PHOENIX-4810: Send parent->child link mutations to SYSTEM.CHILD_LINK table in MetdataClient.createTableInternal
0db6313 is described below

commit 0db6313987b8473f296a952978697145168356ec
Author: Chinmay Kulkarni <ch...@gmail.com>
AuthorDate: Mon Sep 30 18:32:07 2019 -0700

    PHOENIX-4810: Send parent->child link mutations to SYSTEM.CHILD_LINK table in MetdataClient.createTableInternal
---
 .../apache/phoenix/end2end/BasePermissionsIT.java  | 100 +--
 .../phoenix/end2end/PermissionNSEnabledIT.java     |  72 ++
 .../it/java/org/apache/phoenix/end2end/ViewIT.java |  67 +-
 .../coprocessor/BaseMetaDataEndpointObserver.java  |   5 +
 .../coprocessor/ChildLinkMetaDataEndpoint.java     | 121 ++++
 .../phoenix/coprocessor/MetaDataEndpointImpl.java  |  74 +-
 .../coprocessor/MetaDataEndpointObserver.java      |   3 +
 .../phoenix/coprocessor/MetaDataProtocol.java      |   3 +
 .../PhoenixMetaDataCoprocessorHost.java            |  33 +-
 .../generated/ChildLinkMetaDataProtos.java         | 786 +++++++++++++++++++++
 .../coprocessor/generated/MetaDataProtos.java      |  60 +-
 .../apache/phoenix/exception/SQLExceptionCode.java |   7 +
 .../org/apache/phoenix/protobuf/ProtobufUtil.java  |   6 +
 .../phoenix/query/ConnectionQueryServicesImpl.java |  88 ++-
 .../org/apache/phoenix/schema/MetaDataClient.java  |  13 +-
 .../java/org/apache/phoenix/util/MetaDataUtil.java |  18 +-
 .../src/main/ChildLinkMetaDataService.proto        |  36 +
 phoenix-protocol/src/main/MetaDataService.proto    |   1 +
 18 files changed, 1350 insertions(+), 143 deletions(-)

diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
index c4e3907..4301ec8 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/BasePermissionsIT.java
@@ -76,26 +76,26 @@ public abstract class BasePermissionsIT extends BaseTest {
 
     private static final Logger LOGGER = LoggerFactory.getLogger(BasePermissionsIT.class);
 
-    static String SUPER_USER = System.getProperty("user.name");
+    private static String SUPER_USER = System.getProperty("user.name");
 
-    static HBaseTestingUtility testUtil;
-    static final Set<String> PHOENIX_SYSTEM_TABLES =
+    private static HBaseTestingUtility testUtil;
+    private static final Set<String> PHOENIX_SYSTEM_TABLES =
             new HashSet<>(Arrays.asList("SYSTEM.CATALOG", "SYSTEM.SEQUENCE", "SYSTEM.STATS",
-                "SYSTEM.FUNCTION", "SYSTEM.MUTEX"));
+                "SYSTEM.FUNCTION", "SYSTEM.MUTEX", "SYSTEM.CHILD_LINK"));
 
-    static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS =
+    private static final Set<String> PHOENIX_SYSTEM_TABLES_IDENTIFIERS =
             new HashSet<>(Arrays.asList("SYSTEM.\"CATALOG\"", "SYSTEM.\"SEQUENCE\"",
-                "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\"", "SYSTEM.\"MUTEX\""));
+                "SYSTEM.\"STATS\"", "SYSTEM.\"FUNCTION\"", "SYSTEM.\"MUTEX\"", "SYSTEM.\"CHILD_LINK\""));
 
-    static final String SYSTEM_SEQUENCE_IDENTIFIER =
+    private static final String SYSTEM_SEQUENCE_IDENTIFIER =
             QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\"" + PhoenixDatabaseMetaData.SYSTEM_SEQUENCE_TABLE+ "\"";
 
-    static final String SYSTEM_MUTEX_IDENTIFIER =
+    private static final String SYSTEM_MUTEX_IDENTIFIER =
             QueryConstants.SYSTEM_SCHEMA_NAME + "." + "\""
                     + PhoenixDatabaseMetaData.SYSTEM_MUTEX_TABLE_NAME + "\"";
 
     static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(Arrays.asList(
-            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION", "SYSTEM:MUTEX"));
+            "SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION", "SYSTEM:MUTEX", "SYSTEM:CHILD_LINK"));
 
     // Create Multiple users so that we can use Hadoop UGI to run tasks as various users
     // Permissions can be granted or revoke by superusers and admins only
@@ -104,23 +104,23 @@ public abstract class BasePermissionsIT extends BaseTest {
 
     // Super User has all the access
     static User superUser1 = null;
-    static User superUser2 = null;
+    private static User superUser2 = null;
 
     // Regular users are granted and revoked permissions as needed
     User regularUser1 = null;
-    User regularUser2 = null;
-    User regularUser3 = null;
-    User regularUser4 = null;
+    private User regularUser2 = null;
+    private User regularUser3 = null;
+    private User regularUser4 = null;
 
     // Group User is equivalent of regular user but inside a group
     // Permissions can be granted to group should affect this user
     static final String GROUP_SYSTEM_ACCESS = "group_system_access";
-    User groupUser = null;
+    private User groupUser = null;
 
     // Unpriviledged User doesn't have any access and is denied for every action
     User unprivilegedUser = null;
 
-    static final int NUM_RECORDS = 5;
+    private static final int NUM_RECORDS = 5;
 
     boolean isNamespaceMapped;
 
@@ -134,12 +134,12 @@ public abstract class BasePermissionsIT extends BaseTest {
     private String view1TableName;
     private String view2TableName;
 
-    public BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
+    BasePermissionsIT(final boolean isNamespaceMapped) throws Exception {
         this.isNamespaceMapped = isNamespaceMapped;
         this.tableName = generateUniqueName();
     }
 
-    public static void initCluster(boolean isNamespaceMapped) throws Exception {
+    static void initCluster(boolean isNamespaceMapped) throws Exception {
         if (null != testUtil) {
             testUtil.shutdownMiniCluster();
             testUtil = null;
@@ -234,7 +234,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         AccessControlClient.revoke(getUtility().getConnection(), unprivilegedUser.getShortName(), Permission.Action.values() );
     }
 
-    Properties getClientProperties(String tenantId) {
+    private Properties getClientProperties(String tenantId) {
         Properties props = new Properties();
         if(tenantId != null) {
             props.setProperty(PhoenixRuntime.TENANT_ID_ATTRIB, tenantId);
@@ -255,7 +255,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
     }
 
-    static Set<String> getHBaseTables() throws IOException {
+    private static Set<String> getHBaseTables() throws IOException {
         Set<String> tables = new HashSet<>();
         for (TableName tn : testUtil.getHBaseAdmin().listTableNames()) {
             tables.add(tn.getNameAsString());
@@ -266,12 +266,12 @@ public abstract class BasePermissionsIT extends BaseTest {
     // UG Object
     // 1. Instance of String --> represents GROUP name
     // 2. Instance of User --> represents HBase user
-    AccessTestAction grantPermissions(final String actions, final Object ug,
+    private AccessTestAction grantPermissions(final String actions, final Object ug,
                                       final String tableOrSchemaList, final boolean isSchema) throws SQLException {
         return grantPermissions(actions, ug, Collections.singleton(tableOrSchemaList), isSchema);
     }
 
-    AccessTestAction grantPermissions(final String actions, final Object ug,
+    private AccessTestAction grantPermissions(final String actions, final Object ug,
                                       final Set<String> tableOrSchemaList, final boolean isSchema) throws SQLException {
         return new AccessTestAction() {
             @Override
@@ -289,7 +289,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction grantPermissions(final String actions, final User user) throws SQLException {
+    private AccessTestAction grantPermissions(final String actions, final User user) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -303,12 +303,12 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction revokePermissions(final Object ug,
+    private AccessTestAction revokePermissions(final Object ug,
                                        final String tableOrSchemaList, final boolean isSchema) throws SQLException {
         return revokePermissions(ug, Collections.singleton(tableOrSchemaList), isSchema);
     }
 
-    AccessTestAction revokePermissions(final Object ug,
+    private AccessTestAction revokePermissions(final Object ug,
                                        final Set<String> tableOrSchemaList, final boolean isSchema) throws SQLException {
         return new AccessTestAction() {
             @Override
@@ -326,7 +326,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction revokePermissions(final Object ug) throws SQLException {
+    private AccessTestAction revokePermissions(final Object ug) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -343,7 +343,7 @@ public abstract class BasePermissionsIT extends BaseTest {
 
     // Attempts to get a Phoenix Connection
     // New connections could create SYSTEM tables if appropriate perms are granted
-    AccessTestAction getConnectionAction() throws SQLException {
+    private AccessTestAction getConnectionAction() throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -403,7 +403,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction createMultiTenantTable(final String tableName) throws SQLException {
+    private AccessTestAction createMultiTenantTable(final String tableName) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -426,7 +426,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction dropTable(final String tableName) throws SQLException {
+    private AccessTestAction dropTable(final String tableName) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -443,7 +443,7 @@ public abstract class BasePermissionsIT extends BaseTest {
     // AccessDeniedException is only triggered when ResultSet#next() method is called
     // The first call triggers HBase Scan object
     // The Statement#executeQuery() method returns an iterator and doesn't interact with HBase API at all
-    AccessTestAction readTableWithoutVerification(final String tableName) throws SQLException {
+    private AccessTestAction readTableWithoutVerification(final String tableName) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -458,11 +458,11 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction readTable(final String tableName) throws SQLException {
+    private AccessTestAction readTable(final String tableName) throws SQLException {
         return readTable(tableName,null);
     }
 
-    AccessTestAction readTable(final String tableName, final String indexName) throws SQLException {
+    private AccessTestAction readTable(final String tableName, final String indexName) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -484,11 +484,11 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction readMultiTenantTableWithoutIndex(final String tableName) throws SQLException {
+    private AccessTestAction readMultiTenantTableWithoutIndex(final String tableName) throws SQLException {
         return readMultiTenantTableWithoutIndex(tableName, null);
     }
 
-    AccessTestAction readMultiTenantTableWithoutIndex(final String tableName, final String tenantId) throws SQLException {
+    private AccessTestAction readMultiTenantTableWithoutIndex(final String tableName, final String tenantId) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -522,11 +522,11 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction readMultiTenantTableWithIndex(final String tableName) throws SQLException {
+    private AccessTestAction readMultiTenantTableWithIndex(final String tableName) throws SQLException {
         return readMultiTenantTableWithIndex(tableName, null);
     }
 
-    AccessTestAction readMultiTenantTableWithIndex(final String tableName, final String tenantId) throws SQLException {
+    private AccessTestAction readMultiTenantTableWithIndex(final String tableName, final String tenantId) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -559,7 +559,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction addProperties(final String tableName, final String property, final String value)
+    private AccessTestAction addProperties(final String tableName, final String property, final String value)
             throws SQLException {
         return new AccessTestAction() {
             @Override
@@ -584,7 +584,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction dropColumn(final String tableName, final String columnName) throws SQLException {
+    private AccessTestAction dropColumn(final String tableName, final String columnName) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -596,11 +596,11 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction createIndex(final String indexName, final String dataTable) throws SQLException {
+    private AccessTestAction createIndex(final String indexName, final String dataTable) throws SQLException {
         return createIndex(indexName, dataTable, null);
     }
 
-    AccessTestAction createIndex(final String indexName, final String dataTable, final String tenantId) throws SQLException {
+    private AccessTestAction createIndex(final String indexName, final String dataTable, final String tenantId) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -613,7 +613,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction createLocalIndex(final String indexName, final String dataTable) throws SQLException {
+    private AccessTestAction createLocalIndex(final String indexName, final String dataTable) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -626,7 +626,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction dropIndex(final String indexName, final String dataTable) throws SQLException {
+    private AccessTestAction dropIndex(final String indexName, final String dataTable) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -638,7 +638,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction rebuildIndex(final String indexName, final String dataTable) throws SQLException {
+    private AccessTestAction rebuildIndex(final String indexName, final String dataTable) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -651,7 +651,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    AccessTestAction dropView(final String viewName) throws SQLException {
+    private AccessTestAction dropView(final String viewName) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -667,7 +667,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         return createView(viewName, dataTable, null);
     }
 
-    AccessTestAction createView(final String viewName, final String dataTable, final String tenantId) throws SQLException {
+    private AccessTestAction createView(final String viewName, final String dataTable, final String tenantId) throws SQLException {
         return new AccessTestAction() {
             @Override
             public Object run() throws Exception {
@@ -680,7 +680,7 @@ public abstract class BasePermissionsIT extends BaseTest {
         };
     }
 
-    static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
+    interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
 
     /** This fails only in case of ADE or empty list for any of the users. */
     void verifyAllowed(AccessTestAction action, User... users) throws Exception {
@@ -692,13 +692,13 @@ public abstract class BasePermissionsIT extends BaseTest {
         }
     }
 
-    void verifyAllowed(User user, AccessTestAction... actions) throws Exception {
+    private void verifyAllowed(User user, AccessTestAction... actions) throws Exception {
         for (AccessTestAction action : actions) {
             try {
                 Object obj = user.runAs(action);
                 if (obj != null && obj instanceof List<?>) {
                     List<?> results = (List<?>) obj;
-                    if (results != null && results.isEmpty()) {
+                    if (results.isEmpty()) {
                         fail("Empty non null results from action for user '" + user.getShortName() + "'");
                     }
                 }
@@ -719,7 +719,7 @@ public abstract class BasePermissionsIT extends BaseTest {
     }
 
     /** This passes only if desired exception is caught for all users. */
-    <T> void verifyDenied(User user, Class<T> exception, AccessTestAction... actions) throws Exception {
+    private <T> void verifyDenied(User user, Class<T> exception, AccessTestAction... actions) throws Exception {
         for (AccessTestAction action : actions) {
             try {
                 user.runAs(action);
@@ -755,11 +755,11 @@ public abstract class BasePermissionsIT extends BaseTest {
         }
     }
 
-    String surroundWithDoubleQuotes(String input) {
+    private String surroundWithDoubleQuotes(String input) {
         return "\"" + input + "\"";
     }
 
-    void validateAccessDeniedException(AccessDeniedException ade) {
+    private void validateAccessDeniedException(AccessDeniedException ade) {
         String msg = ade.getMessage();
         assertTrue("Exception contained unexpected message: '" + msg + "'",
                 !msg.contains("is not the scanner owner"));
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionNSEnabledIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionNSEnabledIT.java
index 30f3a08..fffa536 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionNSEnabledIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/PermissionNSEnabledIT.java
@@ -29,8 +29,11 @@ import org.junit.Test;
 import java.security.PrivilegedExceptionAction;
 import java.sql.Connection;
 import java.sql.SQLException;
+import java.util.Collections;
 
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.PK_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_SCHEMA_NAME;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
@@ -124,4 +127,73 @@ public class PermissionNSEnabledIT extends BasePermissionsIT {
             revokeAll();
         }
     }
+
+    // After PHOENIX-4810, a user requires Exec permissions on SYSTEM.CHILD_LINK to create views
+    // since the user must invoke the ChildLinkMetaDataEndpoint to create parent->child links
+    @Test
+    public void testViewCreationFailsWhenNoExecPermsOnSystemChildLink() throws Throwable {
+        try {
+            grantSystemTableAccess();
+            final TableName systemChildLink = TableName.valueOf(SchemaUtil.getPhysicalHBaseTableName(
+                    SYSTEM_SCHEMA_NAME, SYSTEM_CHILD_LINK_TABLE, true).getString());
+            final String schemaName = "S_" + generateUniqueName();
+            final String tableName = "T_" + generateUniqueName();
+            final String fullTableName = schemaName + "." + tableName;
+            final String viewName = "V_" + generateUniqueName();
+            verifyAllowed(createSchema(schemaName), superUser1);
+            verifyAllowed(createTable(fullTableName), superUser1);
+
+            superUser1.runAs(new PrivilegedExceptionAction<Object>() {
+                @Override public Object run() throws Exception {
+                    try {
+                        // Revoke Exec permissions for SYSTEM CHILD_LINK for the unprivileged user
+                        AccessControlClient.revoke(getUtility().getConnection(), systemChildLink,
+                                unprivilegedUser.getShortName(), null, null,
+                                Permission.Action.EXEC);
+
+                        // Grant read and exec permissions to the user on the parent table so it
+                        // doesn't fail to getTable when resolving the parent
+                        PermissionNSEnabledIT.this.grantPermissions(unprivilegedUser.getShortName(),
+                                Collections.singleton(SchemaUtil
+                                        .getPhysicalHBaseTableName(schemaName, tableName, true)
+                                        .getString()), Permission.Action.READ,
+                                Permission.Action.EXEC);
+                    } catch (Throwable t) {
+                        if (t instanceof Exception) {
+                            throw (Exception) t;
+                        } else {
+                            throw new Exception(t);
+                        }
+                    }
+                    return null;
+                }
+            });
+
+            // Adding parent->child links fails for the unprivileged user thus failing view creation
+            verifyDenied(createView(viewName, fullTableName), AccessDeniedException.class,
+                    unprivilegedUser);
+
+            superUser1.runAs(new PrivilegedExceptionAction<Object>() {
+                @Override public Object run() throws Exception {
+                    try {
+                        // Grant Exec permissions for SYSTEM CHILD_LINK for the unprivileged user
+                        PermissionNSEnabledIT.this.grantPermissions(unprivilegedUser.getShortName(),
+                                Collections.singleton(systemChildLink.getNameAsString()),
+                                Permission.Action.EXEC);
+                    } catch (Throwable t) {
+                        if (t instanceof Exception) {
+                            throw (Exception) t;
+                        } else {
+                            throw new Exception(t);
+                        }
+                    }
+                    return null;
+                }
+            });
+            verifyAllowed(createView(viewName, fullTableName), unprivilegedUser);
+        } finally {
+            revokeAll();
+        }
+    }
+
 }
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
index 9ab7bb3..1ca6e07 100644
--- a/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/ViewIT.java
@@ -81,6 +81,7 @@ import org.apache.phoenix.util.QueryUtil;
 import org.apache.phoenix.util.ReadOnlyProps;
 import org.apache.phoenix.util.SchemaUtil;
 import org.apache.phoenix.util.TestUtil;
+import org.junit.After;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -100,6 +101,8 @@ public class ViewIT extends SplitSystemCatalogIT {
 
     private static volatile CountDownLatch latch1 = null;
     private static volatile CountDownLatch latch2 = null;
+    private static volatile boolean throwExceptionInChildLinkPreHook = false;
+    private static volatile boolean slowDownAddingChildLink = false;
 
     public ViewIT(String transactionProvider, boolean columnEncoded) {
         StringBuilder optionBuilder = new StringBuilder();
@@ -140,7 +143,15 @@ public class ViewIT extends SplitSystemCatalogIT {
             splitSystemCatalog();
         }
     }
-    
+
+    @After
+    public void cleanup() {
+        latch1 = null;
+        latch2 = null;
+        throwExceptionInChildLinkPreHook = false;
+        slowDownAddingChildLink = false;
+    }
+
     public static class TestMetaDataRegionObserver extends BaseMetaDataEndpointObserver {
         
         @Override
@@ -163,6 +174,16 @@ public class ViewIT extends SplitSystemCatalogIT {
             processTable(tableName);
         }
 
+        @Override
+        public void preCreateViewAddChildLink(
+                final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+                final String tableName) throws IOException {
+            if (throwExceptionInChildLinkPreHook) {
+                throw new IOException();
+            }
+            processTable(tableName);
+        }
+
         private void processTable(String tableName) throws DoNotRetryIOException {
             if (tableName.equals(FAILED_VIEWNAME)) {
                 // throwing anything other than instances of IOException result
@@ -170,8 +191,8 @@ public class ViewIT extends SplitSystemCatalogIT {
                 // DoNotRetryIOException tells HBase not to retry this mutation
                 // multiple times
                 throw new DoNotRetryIOException();
-            } else if (tableName.startsWith(SLOW_VIEWNAME_PREFIX)) {
-                // simulate a slow write to SYSTEM.CATALOG
+            } else if (tableName.startsWith(SLOW_VIEWNAME_PREFIX) || slowDownAddingChildLink) {
+                // simulate a slow write to SYSTEM.CATALOG or SYSTEM.CHILD_LINK
                 if (latch1 != null) {
                     latch1.countDown();
                 }
@@ -801,6 +822,12 @@ public class ViewIT extends SplitSystemCatalogIT {
                 }
             });
 
+            // When dropping a table, we check the parent->child links in the SYSTEM.CHILD_LINK
+            // table and check that cascade is set, if it isn't, we throw an exception (see
+            // ViewUtil.hasChildViews). After PHOENIX-4810, we first send a client-server RPC to add
+            // parent->child links to SYSTEM.CHILD_LINK and then add metadata for the view in
+            // SYSTEM.CATALOG, so we must delay link creation so that the drop table does not fail
+            slowDownAddingChildLink = true;
             // create the view in a separate thread (which will take some time
             // to complete)
             Future<Exception> future =
@@ -808,6 +835,9 @@ public class ViewIT extends SplitSystemCatalogIT {
             // wait till the thread makes the rpc to create the view
             latch1.await();
             tableDdl = "DROP TABLE " + fullTableName;
+
+            // Revert this flag since we don't want to wait in preDropTable
+            slowDownAddingChildLink = false;
             // drop table goes through first and so the view creation should fail
             conn.createStatement().execute(tableDdl);
             latch2.countDown();
@@ -821,10 +851,35 @@ public class ViewIT extends SplitSystemCatalogIT {
     }
 
     @Test
+    public void testChildLinkCreationFailThrowsException() throws Exception {
+        try (Connection conn = DriverManager.getConnection(getUrl())) {
+            String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
+            String fullViewName1 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
+            // create base table
+            String tableDdl = "CREATE TABLE " + fullTableName
+                    + "  (k INTEGER NOT NULL PRIMARY KEY, v1 DATE)" + tableDDLOptions;
+            conn.createStatement().execute(tableDdl);
+
+            // Throw an exception in ChildLinkMetaDataEndpoint while adding parent->child links
+            // to simulate a failure
+            throwExceptionInChildLinkPreHook = true;
+            // create a view
+            String ddl = "CREATE VIEW " + fullViewName1 + " (v2 VARCHAR) AS SELECT * FROM "
+                    + fullTableName + " WHERE k = 6";
+            try {
+                conn.createStatement().execute(ddl);
+                fail("Should have thrown an exception");
+            } catch(SQLException sqlE) {
+                assertEquals("Expected a different Error code",
+                        SQLExceptionCode.UNABLE_TO_CREATE_CHILD_LINK.getErrorCode(),
+                        sqlE.getErrorCode());
+            }
+        }
+    }
+
+    @Test
     public void testConcurrentAddSameColumnDifferentType() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            latch1 = null;
-            latch2 = null;
             String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
             String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
             String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
@@ -884,8 +939,6 @@ public class ViewIT extends SplitSystemCatalogIT {
     @Test
     public void testConcurrentAddDifferentColumn() throws Exception {
         try (Connection conn = DriverManager.getConnection(getUrl())) {
-            latch1 = null;
-            latch2 = null;
             String fullTableName = SchemaUtil.getTableName(SCHEMA1, generateUniqueName());
             String fullViewName1 = SLOW_VIEWNAME_PREFIX + "_" + generateUniqueName();
             String fullViewName2 = SchemaUtil.getTableName(SCHEMA3, generateUniqueName());
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
index 8decc8c..1f9e29b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
@@ -108,4 +108,9 @@ public class BaseMetaDataEndpointObserver implements MetaDataEndpointObserver{
             throws IOException {
 
     }
+
+    @Override
+    public void preCreateViewAddChildLink(
+            final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+            final String tableName) throws IOException {}
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ChildLinkMetaDataEndpoint.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ChildLinkMetaDataEndpoint.java
new file mode 100644
index 0000000..873a1e3
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/ChildLinkMetaDataEndpoint.java
@@ -0,0 +1,121 @@
+/*
+ * 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.protobuf.RpcCallback;
+import com.google.protobuf.RpcController;
+import com.google.protobuf.Service;
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.client.Mutation;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorException;
+import org.apache.hadoop.hbase.coprocessor.CoprocessorService;
+import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment;
+import org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest;
+import org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.ChildLinkMetaDataService;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
+import org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse;
+import org.apache.phoenix.jdbc.PhoenixDatabaseMetaData;
+import org.apache.phoenix.protobuf.ProtobufUtil;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.query.QueryServicesOptions;
+import org.apache.phoenix.util.EnvironmentEdgeManager;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.phoenix.coprocessor.MetaDataEndpointImpl.mutateRowsWithLocks;
+
+/**
+ * Endpoint co-processor through which Phoenix metadata mutations for SYSTEM.CHILD_LINK flow.
+ * The parent->child links ({@link org.apache.phoenix.schema.PTable.LinkType#CHILD_TABLE})
+ * are stored in the SYSTEM.CHILD_LINK table.
+ */
+public class ChildLinkMetaDataEndpoint extends ChildLinkMetaDataService implements
+        CoprocessorService, Coprocessor {
+
+	private static final Logger LOGGER = LoggerFactory.getLogger(ChildLinkMetaDataEndpoint.class);
+    private RegionCoprocessorEnvironment env;
+    private PhoenixMetaDataCoprocessorHost phoenixAccessCoprocessorHost;
+    private boolean accessCheckEnabled;
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+        if (env instanceof RegionCoprocessorEnvironment) {
+            this.env = (RegionCoprocessorEnvironment) env;
+        } else {
+            throw new CoprocessorException("Must be loaded on a table region!");
+        }
+        this.phoenixAccessCoprocessorHost = new PhoenixMetaDataCoprocessorHost(this.env);
+        this.accessCheckEnabled = env.getConfiguration().getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+            QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+        // Do nothing
+    }
+
+    @Override
+    public Service getService() {
+        return this;
+    }
+
+    @Override
+    public void createViewAddChildLink(RpcController controller,
+            CreateViewAddChildLinkRequest request, RpcCallback<MetaDataResponse> done) {
+
+        MetaDataResponse.Builder builder = MetaDataResponse.newBuilder();
+        try {
+            List<Mutation> childLinkMutations = ProtobufUtil.getMutations(request);
+            if (childLinkMutations.isEmpty()) {
+                done.run(builder.build());
+                return;
+            }
+            byte[][] rowKeyMetaData = new byte[3][];
+            MetaDataUtil.getTenantIdAndSchemaAndTableName(childLinkMutations, rowKeyMetaData);
+            byte[] parentSchemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
+            byte[] parentTableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
+            String fullparentTableName = SchemaUtil.getTableName(parentSchemaName, parentTableName);
+
+            getCoprocessorHost().preCreateViewAddChildLink(fullparentTableName);
+
+            // From 4.15 the parent->child links are stored in a separate table SYSTEM.CHILD_LINK
+            mutateRowsWithLocks(this.accessCheckEnabled, this.env.getRegion(), childLinkMutations,
+                Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+
+        } catch (Throwable t) {
+            LOGGER.error("Unable to write mutations to " +
+                    PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME, t);
+            builder.setReturnCode(MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
+            builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+            done.run(builder.build());
+        }
+	}
+
+	private PhoenixMetaDataCoprocessorHost getCoprocessorHost() {
+		return phoenixAccessCoprocessorHost;
+	}
+
+}
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 679c6b0..6df5bf8 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
@@ -1912,23 +1912,13 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // 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
+                // (this is already done at this point, as a separate client-server RPC
+                // to the ChildLinkMetaDataEndpoint coprocessor)
                 // 2. Update the encoded column qualifier for the parent table if its on a
                 // different region server (for tables that use column qualifier encoding)
                 // if the next step fails we end up wasting a few col qualifiers
                 // 3. Finally write the mutations to create the table
 
-                // From 4.15 the parent->child links are stored in a separate table SYSTEM.CHILD_LINK
-                // TODO remove this after PHOENIX-4810 is implemented
-                List<Mutation> childLinkMutations = MetaDataUtil.removeChildLinks(tableMetadata);
-                MetaDataResponse response =
-                        processRemoteRegionMutations(
-                                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES,
-                                childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_CREATE_CHILD_LINK);
-                if (response != null) {
-                    done.run(response);
-                    return;
-                }
-
                 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
@@ -1947,7 +1937,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     // column qualifier counter on the parent table)
                     if (parentTable != null && tableType == PTableType.VIEW && parentTable
                             .getEncodingScheme() != QualifierEncodingScheme.NON_ENCODED_QUALIFIERS) {
-                        response =
+                        // TODO: Avoid doing server-server RPC when we have held row locks
+                        MetaDataResponse response =
                                 processRemoteRegionMutations(
                                         PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES,
                                         remoteMutations, MetaDataProtos.MutationCode.UNABLE_TO_UPDATE_PARENT_TABLE);
@@ -1977,7 +1968,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 // 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(this.accessCheckEnabled, 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
@@ -2184,8 +2176,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
 
                 // drop rows from catalog on this region
-                mutateRowsWithLocks(region, localMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
-                        HConstants.NO_NONCE);
+                mutateRowsWithLocks(this.accessCheckEnabled, region, localMutations, Collections.<byte[]>emptySet(),
+                    HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 for (ImmutableBytesPtr ckey : invalidateList) {
@@ -2196,18 +2188,17 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     metaDataCache.invalidate(parentCacheKey);
                 }
 
-                // after the view metadata is dropped drop parent->child link
+                // 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);
+                                childLinkMutations, MetaDataProtos.MutationCode.UNABLE_TO_DELETE_CHILD_LINK);
                 if (response != null) {
                     done.run(response);
                     return;
                 }
 
                 done.run(MetaDataMutationResult.toProto(result));
-                return;
             } finally {
                 releaseRowLocks(region, locks);
             }
@@ -2347,7 +2338,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     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
+                        // Populate the delete mutations for parent->child link for the child view in question,
+                        // which we issue to SYSTEM.CHILD_LINK later
                         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());
@@ -2604,7 +2596,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         throw new IllegalStateException(msg);
                     }
                 }
-                mutateRowsWithLocks(region, localMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(this.accessCheckEnabled, region, localMutations, Collections.<byte[]>emptySet(),
+                    HConstants.NO_NONCE, HConstants.NO_NONCE);
                 // Invalidate from cache
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
@@ -3282,8 +3275,8 @@ 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(this.accessCheckEnabled, region, tableMetadata, Collections.<byte[]>emptySet(),
+                        HConstants.NO_NONCE, HConstants.NO_NONCE);
                     // Invalidate from cache
                     Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
                             GlobalCache.getInstance(this.env).getMetaDataCache();
@@ -3533,7 +3526,8 @@ 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);
+                    mutateRowsWithLocks(this.accessCheckEnabled, region, functionMetaData,
+                        Collections.<byte[]>emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 }
 
                 // Invalidate the cache - the next getFunction call will add it
@@ -3586,7 +3580,8 @@ 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(this.accessCheckEnabled, region, functionMetaData, Collections.<byte[]>emptySet(),
+                    HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 long currentTime = MetaDataUtil.getClientTimeStamp(functionMetaData);
@@ -3696,8 +3691,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                 }
-                mutateRowsWithLocks(region, schemaMutations, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
-                        HConstants.NO_NONCE);
+                mutateRowsWithLocks(this.accessCheckEnabled, region, schemaMutations, Collections.<byte[]>emptySet(),
+                    HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getSchema call will add it
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache =
@@ -3746,8 +3741,8 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                mutateRowsWithLocks(region, schemaMetaData, Collections.<byte[]>emptySet(), HConstants.NO_NONCE,
-                        HConstants.NO_NONCE);
+                mutateRowsWithLocks(this.accessCheckEnabled, region, schemaMetaData, Collections.<byte[]>emptySet(),
+                    HConstants.NO_NONCE, HConstants.NO_NONCE);
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env)
                         .getMetaDataCache();
                 long currentTime = MetaDataUtil.getClientTimeStamp(schemaMetaData);
@@ -3809,10 +3804,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     }
 
-    private void mutateRowsWithLocks(final Region region, final List<Mutation> mutations, final Set<byte[]> rowsToLock,
-                                     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) {
+    /**
+     * Perform atomic mutations on rows within a region
+     *
+     * @param accessCheckEnabled Use the login user to mutate rows if enabled
+     * @param region Region containing rows to be mutated
+     * @param mutations List of mutations for rows that must be contained within the region
+     * @param rowsToLock Rows to lock
+     * @param nonceGroup Optional nonce group of the operation
+     * @param nonce Optional nonce of the operation
+     * @throws IOException
+     */
+    static void mutateRowsWithLocks(final boolean accessCheckEnabled, final Region region,
+            final List<Mutation> mutations, final Set<byte[]> rowsToLock, final long nonceGroup,
+            final long nonce) throws IOException {
+        // We need to mutate SYSTEM.CATALOG or SYSTEM.CHILD_LINK with HBase/login user
+        // if access is enabled.
+        if (accessCheckEnabled) {
             User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
                 @Override
                 public Void run() throws Exception {
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
index 86b8bf1..629f00b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
@@ -65,4 +65,7 @@ public interface MetaDataEndpointObserver extends Coprocessor {
     void preIndexUpdate(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
             String indexName, TableName physicalTableName, TableName parentPhysicalTableName, PIndexState newState) throws IOException;
 
+    void preCreateViewAddChildLink(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+            final String tableName) throws IOException;
+
 }
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
index 5b4db91..e217a3b 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataProtocol.java
@@ -173,6 +173,9 @@ public abstract class MetaDataProtocol extends MetaDataService {
         AUTO_PARTITION_SEQUENCE_NOT_FOUND,
         CANNOT_COERCE_AUTO_PARTITION_ID,
         TOO_MANY_INDEXES,
+        UNABLE_TO_CREATE_CHILD_LINK,
+        UNABLE_TO_UPDATE_PARENT_TABLE,
+        UNABLE_TO_DELETE_CHILD_LINK,
         NO_OP
     };
 
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
index e089308..e4ee557 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/PhoenixMetaDataCoprocessorHost.java
@@ -50,9 +50,10 @@ public class PhoenixMetaDataCoprocessorHost
     private UserProvider userProvider;
     public static final String PHOENIX_META_DATA_COPROCESSOR_CONF_KEY =
             "hbase.coprocessor.phoenix.classes";
-    public static final String DEFAULT_PHOENIX_META_DATA_COPROCESSOR_CONF_KEY="org.apache.phoenix.coprocessor.PhoenixAccessController";
+    private static final String DEFAULT_PHOENIX_META_DATA_COPROCESSOR_CONF_KEY =
+            "org.apache.phoenix.coprocessor.PhoenixAccessController";
 
-    public PhoenixMetaDataCoprocessorHost(RegionCoprocessorEnvironment env) {
+    PhoenixMetaDataCoprocessorHost(RegionCoprocessorEnvironment env) throws IOException {
         super(null);
         this.env = env;
         this.conf = env.getConfiguration();
@@ -72,7 +73,7 @@ public class PhoenixMetaDataCoprocessorHost
             super(user);
         }
 
-        public void postEnvCall(T env) {}
+        void postEnvCall(T env) {}
     }
 
     private boolean execOperation(
@@ -119,7 +120,7 @@ public class PhoenixMetaDataCoprocessorHost
 
         private RegionCoprocessorEnvironment env;
 
-        public PhoenixMetaDataControllerEnvironment(RegionCoprocessorEnvironment env, Coprocessor instance,
+        PhoenixMetaDataControllerEnvironment(RegionCoprocessorEnvironment env, Coprocessor instance,
                 int priority, int sequence, Configuration conf) {
             super(instance, priority, sequence, conf);
             this.env = env;
@@ -161,7 +162,7 @@ public class PhoenixMetaDataCoprocessorHost
         return new PhoenixMetaDataControllerEnvironment(env, instance, priority, sequence, conf);
     }
 
-    public void preGetTable(final String tenantId, final String tableName, final TableName physicalTableName)
+    void preGetTable(final String tenantId, final String tableName, final TableName physicalTableName)
             throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
             @Override
@@ -172,7 +173,7 @@ public class PhoenixMetaDataCoprocessorHost
         });
     }
 
-    public void preCreateTable(final String tenantId, final String tableName, final TableName physicalTableName,
+    void preCreateTable(final String tenantId, final String tableName, final TableName physicalTableName,
             final TableName parentPhysicalTableName, final PTableType tableType, final Set<byte[]> familySet, final Set<TableName> indexes)
             throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
@@ -185,7 +186,17 @@ public class PhoenixMetaDataCoprocessorHost
         });
     }
 
-    public void preDropTable(final String tenantId, final String tableName, final TableName physicalTableName,
+    void preCreateViewAddChildLink(final String tableName) throws IOException {
+        execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
+            @Override
+            public void call(MetaDataEndpointObserver observer,
+                    ObserverContext<PhoenixMetaDataControllerEnvironment> ctx) throws IOException {
+                observer.preCreateViewAddChildLink(this, tableName);
+            }
+        });
+    }
+
+    void preDropTable(final String tenantId, final String tableName, final TableName physicalTableName,
             final TableName parentPhysicalTableName, final PTableType tableType, final List<PTable> indexes) throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
             @Override
@@ -196,7 +207,7 @@ public class PhoenixMetaDataCoprocessorHost
         });
     }
 
-    public void preAlterTable(final String tenantId, final String tableName, final TableName physicalTableName,
+    void preAlterTable(final String tenantId, final String tableName, final TableName physicalTableName,
             final TableName parentPhysicalTableName, final PTableType type) throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
             @Override
@@ -207,7 +218,7 @@ public class PhoenixMetaDataCoprocessorHost
         });
     }
 
-    public void preGetSchema(final String schemaName) throws IOException {
+    void preGetSchema(final String schemaName) throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
             @Override
             public void call(MetaDataEndpointObserver observer,
@@ -228,7 +239,7 @@ public class PhoenixMetaDataCoprocessorHost
         });
     }
 
-    public void preDropSchema(final String schemaName) throws IOException {
+    void preDropSchema(final String schemaName) throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
             @Override
             public void call(MetaDataEndpointObserver observer,
@@ -238,7 +249,7 @@ public class PhoenixMetaDataCoprocessorHost
         });
     }
 
-    public void preIndexUpdate(final String tenantId, final String indexName, final TableName physicalTableName,
+    void preIndexUpdate(final String tenantId, final String indexName, final TableName physicalTableName,
             final TableName parentPhysicalTableName, final PIndexState newState) throws IOException {
         execOperation(new CoprocessorOperation<PhoenixMetaDataControllerEnvironment>(getActiveUser()) {
             @Override
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ChildLinkMetaDataProtos.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ChildLinkMetaDataProtos.java
new file mode 100644
index 0000000..d3bc1e1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/generated/ChildLinkMetaDataProtos.java
@@ -0,0 +1,786 @@
+// Generated by the protocol buffer compiler.  DO NOT EDIT!
+// source: ChildLinkMetaDataService.proto
+
+package org.apache.phoenix.coprocessor.generated;
+
+public final class ChildLinkMetaDataProtos {
+  private ChildLinkMetaDataProtos() {}
+  public static void registerAllExtensions(
+      com.google.protobuf.ExtensionRegistry registry) {
+  }
+  public interface CreateViewAddChildLinkRequestOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // repeated bytes tableMetadataMutations = 1;
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    java.util.List<com.google.protobuf.ByteString> getTableMetadataMutationsList();
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    int getTableMetadataMutationsCount();
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    com.google.protobuf.ByteString getTableMetadataMutations(int index);
+  }
+  /**
+   * Protobuf type {@code CreateViewAddChildLinkRequest}
+   */
+  public static final class CreateViewAddChildLinkRequest extends
+      com.google.protobuf.GeneratedMessage
+      implements CreateViewAddChildLinkRequestOrBuilder {
+    // Use CreateViewAddChildLinkRequest.newBuilder() to construct.
+    private CreateViewAddChildLinkRequest(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private CreateViewAddChildLinkRequest(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final CreateViewAddChildLinkRequest defaultInstance;
+    public static CreateViewAddChildLinkRequest getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public CreateViewAddChildLinkRequest getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private CreateViewAddChildLinkRequest(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              if (!((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+                tableMetadataMutations_ = new java.util.ArrayList<com.google.protobuf.ByteString>();
+                mutable_bitField0_ |= 0x00000001;
+              }
+              tableMetadataMutations_.add(input.readBytes());
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        if (((mutable_bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
+        }
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.internal_static_CreateViewAddChildLinkRequest_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.internal_static_CreateViewAddChildLinkRequest_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.class, org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<CreateViewAddChildLinkRequest> PARSER =
+        new com.google.protobuf.AbstractParser<CreateViewAddChildLinkRequest>() {
+      public CreateViewAddChildLinkRequest parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new CreateViewAddChildLinkRequest(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<CreateViewAddChildLinkRequest> getParserForType() {
+      return PARSER;
+    }
+
+    // repeated bytes tableMetadataMutations = 1;
+    public static final int TABLEMETADATAMUTATIONS_FIELD_NUMBER = 1;
+    private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_;
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    public java.util.List<com.google.protobuf.ByteString>
+        getTableMetadataMutationsList() {
+      return tableMetadataMutations_;
+    }
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    public int getTableMetadataMutationsCount() {
+      return tableMetadataMutations_.size();
+    }
+    /**
+     * <code>repeated bytes tableMetadataMutations = 1;</code>
+     */
+    public com.google.protobuf.ByteString getTableMetadataMutations(int index) {
+      return tableMetadataMutations_.get(index);
+    }
+
+    private void initFields() {
+      tableMetadataMutations_ = java.util.Collections.emptyList();
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      for (int i = 0; i < tableMetadataMutations_.size(); i++) {
+        output.writeBytes(1, tableMetadataMutations_.get(i));
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      {
+        int dataSize = 0;
+        for (int i = 0; i < tableMetadataMutations_.size(); i++) {
+          dataSize += com.google.protobuf.CodedOutputStream
+            .computeBytesSizeNoTag(tableMetadataMutations_.get(i));
+        }
+        size += dataSize;
+        size += 1 * getTableMetadataMutationsList().size();
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    @java.lang.Override
+    public boolean equals(final java.lang.Object obj) {
+      if (obj == this) {
+       return true;
+      }
+      if (!(obj instanceof org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest)) {
+        return super.equals(obj);
+      }
+      org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest other = (org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest) obj;
+
+      boolean result = true;
+      result = result && getTableMetadataMutationsList()
+          .equals(other.getTableMetadataMutationsList());
+      result = result &&
+          getUnknownFields().equals(other.getUnknownFields());
+      return result;
+    }
+
+    private int memoizedHashCode = 0;
+    @java.lang.Override
+    public int hashCode() {
+      if (memoizedHashCode != 0) {
+        return memoizedHashCode;
+      }
+      int hash = 41;
+      hash = (19 * hash) + getDescriptorForType().hashCode();
+      if (getTableMetadataMutationsCount() > 0) {
+        hash = (37 * hash) + TABLEMETADATAMUTATIONS_FIELD_NUMBER;
+        hash = (53 * hash) + getTableMetadataMutationsList().hashCode();
+      }
+      hash = (29 * hash) + getUnknownFields().hashCode();
+      memoizedHashCode = hash;
+      return hash;
+    }
+
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code CreateViewAddChildLinkRequest}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequestOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.internal_static_CreateViewAddChildLinkRequest_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.internal_static_CreateViewAddChildLinkRequest_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.class, org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.Builder.class);
+      }
+
+      // Construct using org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        tableMetadataMutations_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.internal_static_CreateViewAddChildLinkRequest_descriptor;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest getDefaultInstanceForType() {
+        return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.getDefaultInstance();
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest build() {
+        org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest buildPartial() {
+        org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest result = new org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest(this);
+        int from_bitField0_ = bitField0_;
+        if (((bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = java.util.Collections.unmodifiableList(tableMetadataMutations_);
+          bitField0_ = (bitField0_ & ~0x00000001);
+        }
+        result.tableMetadataMutations_ = tableMetadataMutations_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest) {
+          return mergeFrom((org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest other) {
+        if (other == org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.getDefaultInstance()) return this;
+        if (!other.tableMetadataMutations_.isEmpty()) {
+          if (tableMetadataMutations_.isEmpty()) {
+            tableMetadataMutations_ = other.tableMetadataMutations_;
+            bitField0_ = (bitField0_ & ~0x00000001);
+          } else {
+            ensureTableMetadataMutationsIsMutable();
+            tableMetadataMutations_.addAll(other.tableMetadataMutations_);
+          }
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // repeated bytes tableMetadataMutations = 1;
+      private java.util.List<com.google.protobuf.ByteString> tableMetadataMutations_ = java.util.Collections.emptyList();
+      private void ensureTableMetadataMutationsIsMutable() {
+        if (!((bitField0_ & 0x00000001) == 0x00000001)) {
+          tableMetadataMutations_ = new java.util.ArrayList<com.google.protobuf.ByteString>(tableMetadataMutations_);
+          bitField0_ |= 0x00000001;
+         }
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public java.util.List<com.google.protobuf.ByteString>
+          getTableMetadataMutationsList() {
+        return java.util.Collections.unmodifiableList(tableMetadataMutations_);
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public int getTableMetadataMutationsCount() {
+        return tableMetadataMutations_.size();
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public com.google.protobuf.ByteString getTableMetadataMutations(int index) {
+        return tableMetadataMutations_.get(index);
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder setTableMetadataMutations(
+          int index, com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTableMetadataMutationsIsMutable();
+        tableMetadataMutations_.set(index, value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder addTableMetadataMutations(com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  ensureTableMetadataMutationsIsMutable();
+        tableMetadataMutations_.add(value);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder addAllTableMetadataMutations(
+          java.lang.Iterable<? extends com.google.protobuf.ByteString> values) {
+        ensureTableMetadataMutationsIsMutable();
+        super.addAll(values, tableMetadataMutations_);
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>repeated bytes tableMetadataMutations = 1;</code>
+       */
+      public Builder clearTableMetadataMutations() {
+        tableMetadataMutations_ = java.util.Collections.emptyList();
+        bitField0_ = (bitField0_ & ~0x00000001);
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:CreateViewAddChildLinkRequest)
+    }
+
+    static {
+      defaultInstance = new CreateViewAddChildLinkRequest(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:CreateViewAddChildLinkRequest)
+  }
+
+  /**
+   * Protobuf service {@code ChildLinkMetaDataService}
+   */
+  public static abstract class ChildLinkMetaDataService
+      implements com.google.protobuf.Service {
+    protected ChildLinkMetaDataService() {}
+
+    public interface Interface {
+      /**
+       * <code>rpc createViewAddChildLink(.CreateViewAddChildLinkRequest) returns (.MetaDataResponse);</code>
+       */
+      public abstract void createViewAddChildLink(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse> done);
+
+    }
+
+    public static com.google.protobuf.Service newReflectiveService(
+        final Interface impl) {
+      return new ChildLinkMetaDataService() {
+        @java.lang.Override
+        public  void createViewAddChildLink(
+            com.google.protobuf.RpcController controller,
+            org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest request,
+            com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse> done) {
+          impl.createViewAddChildLink(controller, request, done);
+        }
+
+      };
+    }
+
+    public static com.google.protobuf.BlockingService
+        newReflectiveBlockingService(final BlockingInterface impl) {
+      return new com.google.protobuf.BlockingService() {
+        public final com.google.protobuf.Descriptors.ServiceDescriptor
+            getDescriptorForType() {
+          return getDescriptor();
+        }
+
+        public final com.google.protobuf.Message callBlockingMethod(
+            com.google.protobuf.Descriptors.MethodDescriptor method,
+            com.google.protobuf.RpcController controller,
+            com.google.protobuf.Message request)
+            throws com.google.protobuf.ServiceException {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.callBlockingMethod() given method descriptor for " +
+              "wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return impl.createViewAddChildLink(controller, (org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest)request);
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+        public final com.google.protobuf.Message
+            getRequestPrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getRequestPrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+        public final com.google.protobuf.Message
+            getResponsePrototype(
+            com.google.protobuf.Descriptors.MethodDescriptor method) {
+          if (method.getService() != getDescriptor()) {
+            throw new java.lang.IllegalArgumentException(
+              "Service.getResponsePrototype() given method " +
+              "descriptor for wrong service type.");
+          }
+          switch(method.getIndex()) {
+            case 0:
+              return org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse.getDefaultInstance();
+            default:
+              throw new java.lang.AssertionError("Can't get here.");
+          }
+        }
+
+      };
+    }
+
+    /**
+     * <code>rpc createViewAddChildLink(.CreateViewAddChildLinkRequest) returns (.MetaDataResponse);</code>
+     */
+    public abstract void createViewAddChildLink(
+        com.google.protobuf.RpcController controller,
+        org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest request,
+        com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse> done);
+
+    public static final
+        com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptor() {
+      return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.getDescriptor().getServices().get(0);
+    }
+    public final com.google.protobuf.Descriptors.ServiceDescriptor
+        getDescriptorForType() {
+      return getDescriptor();
+    }
+
+    public final void callMethod(
+        com.google.protobuf.Descriptors.MethodDescriptor method,
+        com.google.protobuf.RpcController controller,
+        com.google.protobuf.Message request,
+        com.google.protobuf.RpcCallback<
+          com.google.protobuf.Message> done) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.callMethod() given method descriptor for wrong " +
+          "service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          this.createViewAddChildLink(controller, (org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest)request,
+            com.google.protobuf.RpcUtil.<org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse>specializeCallback(
+              done));
+          return;
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public final com.google.protobuf.Message
+        getRequestPrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getRequestPrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public final com.google.protobuf.Message
+        getResponsePrototype(
+        com.google.protobuf.Descriptors.MethodDescriptor method) {
+      if (method.getService() != getDescriptor()) {
+        throw new java.lang.IllegalArgumentException(
+          "Service.getResponsePrototype() given method " +
+          "descriptor for wrong service type.");
+      }
+      switch(method.getIndex()) {
+        case 0:
+          return org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse.getDefaultInstance();
+        default:
+          throw new java.lang.AssertionError("Can't get here.");
+      }
+    }
+
+    public static Stub newStub(
+        com.google.protobuf.RpcChannel channel) {
+      return new Stub(channel);
+    }
+
+    public static final class Stub extends org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.ChildLinkMetaDataService implements Interface {
+      private Stub(com.google.protobuf.RpcChannel channel) {
+        this.channel = channel;
+      }
+
+      private final com.google.protobuf.RpcChannel channel;
+
+      public com.google.protobuf.RpcChannel getChannel() {
+        return channel;
+      }
+
+      public  void createViewAddChildLink(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest request,
+          com.google.protobuf.RpcCallback<org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse> done) {
+        channel.callMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse.getDefaultInstance(),
+          com.google.protobuf.RpcUtil.generalizeCallback(
+            done,
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse.class,
+            org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse.getDefaultInstance()));
+      }
+    }
+
+    public static BlockingInterface newBlockingStub(
+        com.google.protobuf.BlockingRpcChannel channel) {
+      return new BlockingStub(channel);
+    }
+
+    public interface BlockingInterface {
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse createViewAddChildLink(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest request)
+          throws com.google.protobuf.ServiceException;
+    }
+
+    private static final class BlockingStub implements BlockingInterface {
+      private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) {
+        this.channel = channel;
+      }
+
+      private final com.google.protobuf.BlockingRpcChannel channel;
+
+      public org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse createViewAddChildLink(
+          com.google.protobuf.RpcController controller,
+          org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest request)
+          throws com.google.protobuf.ServiceException {
+        return (org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse) channel.callBlockingMethod(
+          getDescriptor().getMethods().get(0),
+          controller,
+          request,
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.MetaDataResponse.getDefaultInstance());
+      }
+
+    }
+
+    // @@protoc_insertion_point(class_scope:ChildLinkMetaDataService)
+  }
+
+  private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_CreateViewAddChildLinkRequest_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_CreateViewAddChildLinkRequest_fieldAccessorTable;
+
+  public static com.google.protobuf.Descriptors.FileDescriptor
+      getDescriptor() {
+    return descriptor;
+  }
+  private static com.google.protobuf.Descriptors.FileDescriptor
+      descriptor;
+  static {
+    java.lang.String[] descriptorData = {
+      "\n\036ChildLinkMetaDataService.proto\032\025MetaDa" +
+      "taService.proto\032\014PTable.proto\"?\n\035CreateV" +
+      "iewAddChildLinkRequest\022\036\n\026tableMetadataM" +
+      "utations\030\001 \003(\0142g\n\030ChildLinkMetaDataServi" +
+      "ce\022K\n\026createViewAddChildLink\022\036.CreateVie" +
+      "wAddChildLinkRequest\032\021.MetaDataResponseB" +
+      "K\n(org.apache.phoenix.coprocessor.genera" +
+      "tedB\027ChildLinkMetaDataProtosH\001\210\001\001\240\001\001"
+    };
+    com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
+      new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
+        public com.google.protobuf.ExtensionRegistry assignDescriptors(
+            com.google.protobuf.Descriptors.FileDescriptor root) {
+          descriptor = root;
+          internal_static_CreateViewAddChildLinkRequest_descriptor =
+            getDescriptor().getMessageTypes().get(0);
+          internal_static_CreateViewAddChildLinkRequest_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_CreateViewAddChildLinkRequest_descriptor,
+              new java.lang.String[] { "TableMetadataMutations", });
+          return null;
+        }
+      };
+    com.google.protobuf.Descriptors.FileDescriptor
+      .internalBuildGeneratedFileFrom(descriptorData,
+        new com.google.protobuf.Descriptors.FileDescriptor[] {
+          org.apache.phoenix.coprocessor.generated.MetaDataProtos.getDescriptor(),
+          org.apache.phoenix.coprocessor.generated.PTableProtos.getDescriptor(),
+        }, assigner);
+  }
+
+  // @@protoc_insertion_point(outer_class_scope)
+}
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 ca8b20c..28b58c6 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
@@ -113,6 +113,10 @@ public final class MetaDataProtos {
      * <code>UNABLE_TO_UPDATE_PARENT_TABLE = 24;</code>
      */
     UNABLE_TO_UPDATE_PARENT_TABLE(24, 24),
+    /**
+     * <code>UNABLE_TO_DELETE_CHILD_LINK = 25;</code>
+     */
+    UNABLE_TO_DELETE_CHILD_LINK(25, 25),
     ;
 
     /**
@@ -215,6 +219,10 @@ public final class MetaDataProtos {
      * <code>UNABLE_TO_UPDATE_PARENT_TABLE = 24;</code>
      */
     public static final int UNABLE_TO_UPDATE_PARENT_TABLE_VALUE = 24;
+    /**
+     * <code>UNABLE_TO_DELETE_CHILD_LINK = 25;</code>
+     */
+    public static final int UNABLE_TO_DELETE_CHILD_LINK_VALUE = 25;
 
 
     public final int getNumber() { return value; }
@@ -246,6 +254,7 @@ public final class MetaDataProtos {
         case 22: return TOO_MANY_INDEXES;
         case 23: return UNABLE_TO_CREATE_CHILD_LINK;
         case 24: return UNABLE_TO_UPDATE_PARENT_TABLE;
+        case 25: return UNABLE_TO_DELETE_CHILD_LINK;
         default: return null;
       }
     }
@@ -18039,7 +18048,7 @@ public final class MetaDataProtos {
       "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" +
+      "learTableFromCacheResponse*\332\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_" +
@@ -18057,30 +18066,31 @@ public final class MetaDataProtos {
       "#\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"
+      "ABLE\020\030\022\037\n\033UNABLE_TO_DELETE_CHILD_LINK\020\0312" +
+      "\345\006\n\017MetaDataService\022/\n\010getTable\022\020.GetTab" +
+      "leRequest\032\021.MetaDataResponse\0227\n\014getFunct" +
+      "ions\022\024.GetFunctionsRequest\032\021.MetaDataRes" +
+      "ponse\0221\n\tgetSchema\022\021.GetSchemaRequest\032\021.",
+      "MetaDataResponse\0225\n\013createTable\022\023.Create" +
+      "TableRequest\032\021.MetaDataResponse\022;\n\016creat" +
+      "eFunction\022\026.CreateFunctionRequest\032\021.Meta" +
+      "DataResponse\0227\n\014createSchema\022\024.CreateSch" +
+      "emaRequest\032\021.MetaDataResponse\0221\n\tdropTab" +
+      "le\022\021.DropTableRequest\032\021.MetaDataResponse" +
+      "\0223\n\ndropSchema\022\022.DropSchemaRequest\032\021.Met" +
+      "aDataResponse\0227\n\014dropFunction\022\024.DropFunc" +
+      "tionRequest\032\021.MetaDataResponse\0221\n\taddCol" +
+      "umn\022\021.AddColumnRequest\032\021.MetaDataRespons",
+      "e\0223\n\ndropColumn\022\022.DropColumnRequest\032\021.Me" +
+      "taDataResponse\022?\n\020updateIndexState\022\030.Upd" +
+      "ateIndexStateRequest\032\021.MetaDataResponse\022" +
+      "5\n\nclearCache\022\022.ClearCacheRequest\032\023.Clea" +
+      "rCacheResponse\0225\n\ngetVersion\022\022.GetVersio" +
+      "nRequest\032\023.GetVersionResponse\022P\n\023clearTa" +
+      "bleFromCache\022\033.ClearTableFromCacheReques" +
+      "t\032\034.ClearTableFromCacheResponseBB\n(org.a" +
+      "pache.phoenix.coprocessor.generatedB\016Met" +
+      "aDataProtosH\001\210\001\001\240\001\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
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 af97d92..878f442 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
@@ -35,6 +35,7 @@ import org.apache.phoenix.schema.ConcurrentTableMutationException;
 import org.apache.phoenix.schema.FunctionAlreadyExistsException;
 import org.apache.phoenix.schema.FunctionNotFoundException;
 import org.apache.phoenix.schema.IndexNotFoundException;
+import org.apache.phoenix.schema.PTable;
 import org.apache.phoenix.schema.PTableType;
 import org.apache.phoenix.schema.ReadOnlyTableException;
 import org.apache.phoenix.schema.SchemaAlreadyExistsException;
@@ -410,6 +411,12 @@ public enum SQLExceptionCode {
     CANNOT_SET_GUIDE_POST_WIDTH(1139, "XCL39", "Guide post width can only be set on base data tables"),
     CANNOT_CREATE_VIEWS_ON_SYSTEM_TABLES(1141, "XCL41", "Cannot create views on tables of type" +
             PTableType.SYSTEM),
+    UNABLE_TO_CREATE_CHILD_LINK(1142, "XCL42", "Error creating parent-child link (Link type=" +
+            PTable.LinkType.CHILD_TABLE + ") for view"),
+    UNABLE_TO_UPDATE_PARENT_TABLE(1143, "XCL43", "Error Updating the parent table"),
+    UNABLE_TO_DELETE_CHILD_LINK(1144, "XCL44", "Error deleting parent-child link (Link type=" +
+            PTable.LinkType.CHILD_TABLE + ") for view"),
+
     /**
      * Implementation defined class. Phoenix internal error. (errorcode 20, sqlstate INT).
      */
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java b/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
index d15749d..45f43e1 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/protobuf/ProtobufUtil.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto;
 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.MutationProto.MutationType;
 import org.apache.hadoop.hbase.util.ByteStringer;
 import org.apache.hadoop.util.StringUtils;
+import org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.PTableProtos;
 import org.apache.phoenix.coprocessor.generated.ServerCachingProtos;
@@ -107,6 +108,11 @@ public class ProtobufUtil {
         return getMutations(request.getTableMetadataMutationsList());
     }
 
+	public static List<Mutation> getMutations(CreateViewAddChildLinkRequest request)
+	throws IOException {
+		return getMutations(request.getTableMetadataMutationsList());
+	}
+
     /**
      * Each ByteString entry is a byte array serialized from MutationProto instance
      * @param mutations
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 fd9092f..a932d59 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
@@ -127,6 +127,7 @@ import org.apache.hadoop.hbase.client.Increment;
 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.Table;
 import org.apache.hadoop.hbase.client.coprocessor.Batch;
 import org.apache.hadoop.hbase.coprocessor.MultiRowMutationEndpoint;
 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
@@ -146,6 +147,7 @@ import org.apache.hadoop.hbase.util.VersionInfo;
 import org.apache.hadoop.hbase.zookeeper.ZKConfig;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.phoenix.compile.MutationPlan;
+import org.apache.phoenix.coprocessor.ChildLinkMetaDataEndpoint;
 import org.apache.phoenix.coprocessor.GroupedAggregateRegionObserver;
 import org.apache.phoenix.coprocessor.MetaDataEndpointImpl;
 import org.apache.phoenix.coprocessor.MetaDataProtocol;
@@ -157,6 +159,8 @@ import org.apache.phoenix.coprocessor.SequenceRegionObserver;
 import org.apache.phoenix.coprocessor.ServerCachingEndpointImpl;
 import org.apache.phoenix.coprocessor.TaskRegionObserver;
 import org.apache.phoenix.coprocessor.UngroupedAggregateRegionObserver;
+import org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.CreateViewAddChildLinkRequest;
+import org.apache.phoenix.coprocessor.generated.ChildLinkMetaDataProtos.ChildLinkMetaDataService;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.AddColumnRequest;
 import org.apache.phoenix.coprocessor.generated.MetaDataProtos.ClearCacheRequest;
@@ -1018,8 +1022,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
             } else if (SchemaUtil.isTaskTable(tableName)) {
                 if(!descriptor.hasCoprocessor(TaskRegionObserver.class.getName())) {
                     descriptor.addCoprocessor(TaskRegionObserver.class.getName(), null, priority, null);
+                }
+            } else if (SchemaUtil.isChildLinkTable(tableName)) {
+                if (!descriptor.hasCoprocessor(ChildLinkMetaDataEndpoint.class.getName())) {
+                    descriptor.addCoprocessor(ChildLinkMetaDataEndpoint.class.getName(), null, priority, null);
+                }
             }
-        }
 
             if (isTransactional) {
                 Class<? extends RegionObserver> coprocessorClass = provider.getTransactionProvider().getCoprocessor();
@@ -1524,6 +1532,30 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
     }
 
     /**
+     * Invoke the SYSTEM.CHILD_LINK metadata coprocessor endpoint
+     * @param parentTableKey key corresponding to the parent of the view
+     * @param callable used to invoke the coprocessor endpoint to write links from a parent to its child view
+     * @return result of invoking the coprocessor endpoint
+     * @throws SQLException
+     */
+    private MetaDataMutationResult childLinkMetaDataCoprocessorExec(byte[] parentTableKey,
+            Batch.Call<ChildLinkMetaDataService, MetaDataResponse> callable) throws SQLException {
+        try (Table htable = this.getTable(SchemaUtil.getPhysicalName(
+                PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_NAME_BYTES, this.getProps()).getName()))
+        {
+            final Map<byte[], MetaDataResponse> results =
+                    htable.coprocessorService(ChildLinkMetaDataService.class, parentTableKey, parentTableKey, callable);
+            assert(results.size() == 1);
+            MetaDataResponse result = results.values().iterator().next();
+            return MetaDataMutationResult.constructFromProto(result);
+        } catch (IOException e) {
+            throw ServerUtil.parseServerException(e);
+        } catch (Throwable t) {
+            throw new SQLException(t);
+        }
+    }
+
+    /**
      * Invoke meta data coprocessor with one retry if the key was found to not be in the regions
      * (due to a table split)
      */
@@ -1660,6 +1692,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                                               byte[][] splits, boolean isNamespaceMapped,
                                               final boolean allocateIndexId, final boolean isDoNotUpgradePropSet,
                                               final PTable parentTable) throws SQLException {
+        final List<Mutation> childLinkMutations = MetaDataUtil.removeChildLinkMutations(tableMetaData);
         byte[][] rowKeyMetadata = new byte[3][];
         Mutation m = MetaDataUtil.getPutOnlyTableHeaderRow(tableMetaData);
         byte[] key = m.getRow();
@@ -1667,7 +1700,7 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
         byte[] tenantIdBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] schemaBytes = rowKeyMetadata[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
         byte[] tableBytes = rowKeyMetadata[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-        byte[] tableName = physicalTableName != null ? physicalTableName :
+        byte[] physicalTableNameBytes = physicalTableName != null ? physicalTableName :
             SchemaUtil.getPhysicalHBaseTableName(schemaBytes, tableBytes, isNamespaceMapped).getBytes();
         boolean localIndexTable = false;
         for(Pair<byte[], Map<String, Object>> family: families) {
@@ -1676,10 +1709,12 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 break;
             }
         }
-        if ((tableType == PTableType.VIEW && physicalTableName != null) || (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
+        if ((tableType == PTableType.VIEW && physicalTableName != null) ||
+                (tableType != PTableType.VIEW && (physicalTableName == null || localIndexTable))) {
             // For views this will ensure that metadata already exists
             // For tables and indexes, this will create the metadata if it doesn't already exist
-            ensureTableCreated(tableName, tableType, tableProps, families, splits, true, isNamespaceMapped, isDoNotUpgradePropSet);
+            ensureTableCreated(physicalTableNameBytes, tableType, tableProps, families, splits, true,
+                    isNamespaceMapped, isDoNotUpgradePropSet);
         }
         ImmutableBytesWritable ptr = new ImmutableBytesWritable();
         if (tableType == PTableType.INDEX) { // Index on view
@@ -1711,18 +1746,56 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 familiesPlusDefault.add(new Pair<byte[],Map<String,Object>>(defaultCF,Collections.<String,Object>emptyMap()));
             }
             ensureViewIndexTableCreated(
-                tableName, tableProps, familiesPlusDefault, MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
+                physicalTableNameBytes, tableProps, familiesPlusDefault,
+                    MetaDataUtil.isSalted(m, kvBuilder, ptr) ? splits : null,
                 MetaDataUtil.getClientTimeStamp(m), isNamespaceMapped);
         }
 
+        // Avoid the client-server RPC if this is not a view creation
+        if (!childLinkMutations.isEmpty()) {
+            // Send mutations for parent-child links to SYSTEM.CHILD_LINK
+            // We invoke this using the parent table's key since child links are keyed by parent
+            final MetaDataMutationResult result = childLinkMetaDataCoprocessorExec(SchemaUtil.getTableKey(parentTable),
+                    new Batch.Call<ChildLinkMetaDataService, MetaDataResponse>() {
+                        @Override
+                        public MetaDataResponse call(ChildLinkMetaDataService instance) throws IOException {
+                            ServerRpcController controller = new ServerRpcController();
+                            BlockingRpcCallback<MetaDataResponse> rpcCallback =
+                                    new BlockingRpcCallback<>();
+                            CreateViewAddChildLinkRequest.Builder builder =
+                                    CreateViewAddChildLinkRequest.newBuilder();
+                            for (Mutation m: childLinkMutations) {
+                                MutationProto mp = ProtobufUtil.toProto(m);
+                                builder.addTableMetadataMutations(mp.toByteString());
+                            }
+                            CreateViewAddChildLinkRequest build = builder.build();
+                            instance.createViewAddChildLink(controller, build, rpcCallback);
+                            if (controller.getFailedOn() != null) {
+                                throw controller.getFailedOn();
+                            }
+                            return rpcCallback.get();
+                        }
+                    } );
+
+            switch (result.getMutationCode()) {
+                case UNABLE_TO_CREATE_CHILD_LINK:
+                    throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNABLE_TO_CREATE_CHILD_LINK)
+                            .setSchemaName(Bytes.toString(schemaBytes))
+                            .setTableName(Bytes.toString(physicalTableNameBytes)).build().buildException();
+                default:
+                    break;
+            }
+        }
+
+        // Send the remaining metadata mutations to SYSTEM.CATALOG
         byte[] tableKey = SchemaUtil.getTableKey(tenantIdBytes, schemaBytes, tableBytes);
-        MetaDataMutationResult result = metaDataCoprocessorExec(tableKey,
+        return metaDataCoprocessorExec(tableKey,
                 new Batch.Call<MetaDataService, MetaDataResponse>() {
             @Override
             public MetaDataResponse call(MetaDataService instance) throws IOException {
                 ServerRpcController controller = new ServerRpcController();
                 BlockingRpcCallback<MetaDataResponse> rpcCallback =
-                        new BlockingRpcCallback<MetaDataResponse>();
+                        new BlockingRpcCallback<>();
                 CreateTableRequest.Builder builder = CreateTableRequest.newBuilder();
                 for (Mutation m : tableMetaData) {
                     MutationProto mp = ProtobufUtil.toProto(m);
@@ -1743,7 +1816,6 @@ public class ConnectionQueryServicesImpl extends DelegateQueryServices implement
                 return rpcCallback.get();
             }
         });
-        return result;
     }
 
     @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 5ae53bb..d290333 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
@@ -78,6 +78,7 @@ import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.STORE_NULLS;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYNC_INDEX_CREATED_DATE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_SCHEMA;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CATALOG_TABLE;
+import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_CHILD_LINK_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.SYSTEM_FUNCTION_TABLE;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_NAME;
 import static org.apache.phoenix.jdbc.PhoenixDatabaseMetaData.TABLE_SCHEM;
@@ -97,7 +98,6 @@ import static org.apache.phoenix.query.QueryConstants.BASE_TABLE_BASE_COLUMN_COU
 import static org.apache.phoenix.query.QueryConstants.DEFAULT_COLUMN_FAMILY;
 import static org.apache.phoenix.query.QueryConstants.ENCODED_CQ_COUNTER_INITIAL_VALUE;
 import static org.apache.phoenix.query.QueryServices.DROP_METADATA_ATTRIB;
-import static org.apache.phoenix.query.QueryServices.LONG_VIEW_INDEX_ENABLED_ATTRIB;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_DROP_METADATA;
 import static org.apache.phoenix.query.QueryServicesOptions.DEFAULT_RUN_UPDATE_STATS_ASYNC;
 import static org.apache.phoenix.schema.PTable.EncodedCQCounter.NULL_COUNTER;
@@ -2896,6 +2896,11 @@ public class MetaDataClient {
                         .setSchemaName(SchemaUtil.getSchemaNameFromFullName(parent.getPhysicalName().getString()))
                         .setTableName(SchemaUtil.getTableNameFromFullName(parent.getPhysicalName().getString())).build()
                         .buildException();
+            case UNABLE_TO_UPDATE_PARENT_TABLE:
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNABLE_TO_UPDATE_PARENT_TABLE)
+                        .setSchemaName(SchemaUtil.getSchemaNameFromFullName(parent.getPhysicalName().getString()))
+                        .setTableName(SchemaUtil.getTableNameFromFullName(parent.getPhysicalName().getString())).build()
+                        .buildException();
             default:
                 // If the parent table of the view has the auto partition sequence name attribute,
                 // set the view statement and relevant partition column attributes correctly
@@ -3141,8 +3146,10 @@ public class MetaDataClient {
                 throw new NewerTableAlreadyExistsException(schemaName, tableName, result.getTable());
             case UNALLOWED_TABLE_MUTATION:
                 throw new SQLExceptionInfo.Builder(SQLExceptionCode.CANNOT_MUTATE_TABLE)
-
-                .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
+            case UNABLE_TO_DELETE_CHILD_LINK:
+                throw new SQLExceptionInfo.Builder(SQLExceptionCode.UNABLE_TO_DELETE_CHILD_LINK)
+                        .setSchemaName(schemaName).setTableName(tableName).build().buildException();
             default:
                 connection.removeTable(tenantId, SchemaUtil.getTableName(schemaName, tableName), parentTableName, result.getMutationTime());
 
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 239018b..dd5ccda 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
@@ -906,10 +906,16 @@ public class MetaDataUtil {
         byte[] physicalTableName = Bytes.toBytes(SchemaUtil.getTableNameFromFullName(view.getPhysicalName().getString()));
         return SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, physicalTableSchemaName, physicalTableName);
     }
-    
-	public static List<Mutation> removeChildLinks(List<Mutation> catalogMutations) {
-		List<Mutation> childLinks = Lists.newArrayList();
-		Iterator<Mutation> iter = catalogMutations.iterator();
+
+    /**
+     * Extract mutations of link type {@link PTable.LinkType#CHILD_TABLE} from the list of mutations.
+     * The child link mutations will be sent to SYSTEM.CHILD_LINK and other mutations to SYSTEM.CATALOG
+     * @param metadataMutations total list of mutations
+     * @return list of mutations pertaining to parent-child links
+     */
+	public static List<Mutation> removeChildLinkMutations(List<Mutation> metadataMutations) {
+		List<Mutation> childLinkMutations = Lists.newArrayList();
+		Iterator<Mutation> iter = metadataMutations.iterator();
 		while (iter.hasNext()) {
 			Mutation m = iter.next();
 			for (KeyValue kv : m.getFamilyMap().get(PhoenixDatabaseMetaData.TABLE_FAMILY_BYTES)) {
@@ -920,12 +926,12 @@ public class MetaDataUtil {
 						&& ((Bytes.compareTo(kv.getValueArray(), kv.getValueOffset(), kv.getValueLength(),
 								LinkType.CHILD_TABLE.getSerializedValueAsByteArray(), 0,
 								LinkType.CHILD_TABLE.getSerializedValueAsByteArray().length) == 0))) {
-					childLinks.add(m);
+					childLinkMutations.add(m);
 					iter.remove();
 				}
 			}
 		}
-		return childLinks;
+		return childLinkMutations;
 	}
 
 	public static IndexType getIndexType(List<Mutation> tableMetaData, KeyValueBuilder builder,
diff --git a/phoenix-protocol/src/main/ChildLinkMetaDataService.proto b/phoenix-protocol/src/main/ChildLinkMetaDataService.proto
new file mode 100644
index 0000000..046397c
--- /dev/null
+++ b/phoenix-protocol/src/main/ChildLinkMetaDataService.proto
@@ -0,0 +1,36 @@
+/*
+ * 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.
+ */
+
+option java_package = "org.apache.phoenix.coprocessor.generated";
+option java_outer_classname = "ChildLinkMetaDataProtos";
+option java_generic_services = true;
+option java_generate_equals_and_hash = true;
+option optimize_for = SPEED;
+
+import "MetaDataService.proto";
+import "PTable.proto";
+
+message CreateViewAddChildLinkRequest {
+    repeated bytes tableMetadataMutations = 1;
+}
+
+service ChildLinkMetaDataService {
+
+    rpc createViewAddChildLink(CreateViewAddChildLinkRequest)
+        returns (MetaDataResponse);
+}
\ No newline at end of file
diff --git a/phoenix-protocol/src/main/MetaDataService.proto b/phoenix-protocol/src/main/MetaDataService.proto
index 1227ad6..a30fad7 100644
--- a/phoenix-protocol/src/main/MetaDataService.proto
+++ b/phoenix-protocol/src/main/MetaDataService.proto
@@ -52,6 +52,7 @@ enum MutationCode {
   TOO_MANY_INDEXES = 22;
   UNABLE_TO_CREATE_CHILD_LINK = 23;
   UNABLE_TO_UPDATE_PARENT_TABLE = 24;
+  UNABLE_TO_DELETE_CHILD_LINK = 25;
 };
 
 message SharedTableState {