You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@phoenix.apache.org by an...@apache.org on 2017/11/09 05:57:29 UTC

[2/2] phoenix git commit: PHOENIX-4198 Remove the need for users to have access to the Phoenix SYSTEM tables to create tables

PHOENIX-4198 Remove the need for users to have access to the Phoenix SYSTEM tables to create tables


Project: http://git-wip-us.apache.org/repos/asf/phoenix/repo
Commit: http://git-wip-us.apache.org/repos/asf/phoenix/commit/217867c7
Tree: http://git-wip-us.apache.org/repos/asf/phoenix/tree/217867c7
Diff: http://git-wip-us.apache.org/repos/asf/phoenix/diff/217867c7

Branch: refs/heads/master
Commit: 217867c78108b29d991794726c01c1eefb49b828
Parents: 4a1f0df
Author: Ankit Singhal <an...@gmail.com>
Authored: Thu Nov 9 11:27:11 2017 +0530
Committer: Ankit Singhal <an...@gmail.com>
Committed: Thu Nov 9 11:27:11 2017 +0530

----------------------------------------------------------------------
 .../phoenix/end2end/TableDDLPermissionsIT.java  | 692 +++++++++++++++++++
 .../org/apache/hadoop/hbase/ipc/RpcUtil.java    |  32 +
 .../BaseMetaDataEndpointObserver.java           | 111 +++
 .../coprocessor/MetaDataEndpointImpl.java       | 339 +++++++--
 .../coprocessor/MetaDataEndpointObserver.java   |  68 ++
 .../coprocessor/MetaDataRegionObserver.java     |  17 +-
 .../coprocessor/PhoenixAccessController.java    | 628 +++++++++++++++++
 .../PhoenixMetaDataCoprocessorHost.java         | 236 +++++++
 .../index/PhoenixIndexFailurePolicy.java        | 109 +--
 .../query/ConnectionQueryServicesImpl.java      |  15 +-
 .../org/apache/phoenix/query/QueryServices.java |   4 +
 .../phoenix/query/QueryServicesOptions.java     |  14 +-
 .../phoenix/schema/stats/StatisticsWriter.java  |  42 +-
 .../org/apache/phoenix/util/MetaDataUtil.java   |  18 +
 .../org/apache/phoenix/util/SchemaUtil.java     |  12 +
 15 files changed, 2196 insertions(+), 141 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
new file mode 100644
index 0000000..971383b
--- /dev/null
+++ b/phoenix-core/src/it/java/org/apache/phoenix/end2end/TableDDLPermissionsIT.java
@@ -0,0 +1,692 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.phoenix.end2end;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.IOException;
+import java.lang.reflect.UndeclaredThrowableException;
+import java.security.PrivilegedExceptionAction;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.AuthUtil;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.NamespaceDescriptor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.security.AccessDeniedException;
+import org.apache.hadoop.hbase.security.access.AccessControlClient;
+import org.apache.hadoop.hbase.security.access.Permission.Action;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.phoenix.exception.PhoenixIOException;
+import org.apache.phoenix.query.QueryServices;
+import org.apache.phoenix.util.MetaDataUtil;
+import org.apache.phoenix.util.SchemaUtil;
+import org.junit.After;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import com.google.common.collect.Maps;
+
+/**
+ * Test that verifies a user can read Phoenix tables with a minimal set of permissions.
+ */
+@Category(NeedsOwnMiniClusterTest.class)
+@RunWith(Parameterized.class)
+public class TableDDLPermissionsIT{
+    private static String SUPERUSER;
+
+    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"));
+    // PHOENIX-XXXX SYSTEM.MUTEX isn't being created in the SYSTEM namespace as it should be.
+    private static final Set<String> PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES = new HashSet<>(
+            Arrays.asList("SYSTEM:CATALOG", "SYSTEM:SEQUENCE", "SYSTEM:STATS", "SYSTEM:FUNCTION",
+                "SYSTEM.MUTEX"));
+    private static final String GROUP_SYSTEM_ACCESS = "group_system_access";
+    final UserGroupInformation superUser = UserGroupInformation.createUserForTesting(SUPERUSER, new String[0]);
+    final UserGroupInformation superUser2 = UserGroupInformation.createUserForTesting("superuser", new String[0]);
+    final UserGroupInformation regularUser = UserGroupInformation.createUserForTesting("user",  new String[0]);
+    final UserGroupInformation groupUser = UserGroupInformation.createUserForTesting("user2", new String[] { GROUP_SYSTEM_ACCESS });
+    final UserGroupInformation unprivilegedUser = UserGroupInformation.createUserForTesting("unprivilegedUser",
+            new String[0]);
+
+
+    private static final int NUM_RECORDS = 5;
+
+    private boolean isNamespaceMapped;
+
+    public TableDDLPermissionsIT(final boolean isNamespaceMapped) throws Exception {
+        this.isNamespaceMapped = isNamespaceMapped;
+        Map<String, String> clientProps = Maps.newHashMapWithExpectedSize(1);
+        clientProps.put(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, "true");
+    }
+
+    private void startNewMiniCluster(Configuration overrideConf) throws Exception{
+        if (null != testUtil) {
+            testUtil.shutdownMiniCluster();
+            testUtil = null;
+        }
+        testUtil = new HBaseTestingUtility();
+
+        Configuration config = testUtil.getConfiguration();
+        
+        config.set("hbase.coprocessor.master.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.coprocessor.region.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.coprocessor.regionserver.classes",
+                "org.apache.hadoop.hbase.security.access.AccessController");
+        config.set("hbase.security.exec.permission.checks", "true");
+        config.set("hbase.security.authorization", "true");
+        config.set("hbase.superuser", SUPERUSER+","+superUser2.getShortUserName());
+        config.set("hbase.regionserver.wal.codec", "org.apache.hadoop.hbase.regionserver.wal.IndexedWALEditCodec");
+        config.set(QueryServices.PHOENIX_ACLS_ENABLED,"true");
+        config.set(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        // Avoid multiple clusters trying to bind the master's info port (16010)
+        config.setInt(HConstants.MASTER_INFO_PORT, -1);
+        
+        if (overrideConf != null) {
+            config.addResource(overrideConf);
+        }
+        testUtil.startMiniCluster(1);
+    }
+    
+    private void grantSystemTableAccess() throws Exception{
+        try (Connection conn = getConnection()) {
+            if (isNamespaceMapped) {
+                grantPermissions(regularUser.getShortUserName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES, Action.READ,
+                        Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
+                        Action.READ, Action.EXEC);
+                grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), PHOENIX_NAMESPACE_MAPPED_SYSTEM_TABLES,
+                        Action.READ, Action.EXEC);
+                // Local Index requires WRITE permission on SYSTEM.SEQUENCE TABLE.
+                grantPermissions(regularUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+                
+            } else {
+                grantPermissions(regularUser.getShortUserName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), PHOENIX_SYSTEM_TABLES, Action.READ, Action.EXEC);
+                // Local Index requires WRITE permission on SYSTEM.SEQUENCE TABLE.
+                grantPermissions(regularUser.getShortUserName(), Collections.singleton("SYSTEM.SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+                grantPermissions(unprivilegedUser.getShortUserName(), Collections.singleton("SYSTEM:SEQUENCE"), Action.WRITE,
+                        Action.READ, Action.EXEC);
+            }
+        } catch (Throwable e) {
+            if (e instanceof Exception) {
+                throw (Exception)e;
+            } else {
+                throw new Exception(e);
+            }
+        }
+    }
+
+    @Parameters(name = "isNamespaceMapped={0}") // name is used by failsafe as file name in reports
+    public static Collection<Boolean> data() {
+        return Arrays.asList(true, false);
+    }
+
+    @BeforeClass
+    public static void doSetup() throws Exception {
+        SUPERUSER = System.getProperty("user.name");
+        //setUpTestDriver(new ReadOnlyProps(serverProps.entrySet().iterator()), new ReadOnlyProps(clientProps.entrySet().iterator()));
+    }
+
+    protected static String getUrl() {
+        return "jdbc:phoenix:localhost:" + testUtil.getZkCluster().getClientPort() + ":/hbase";
+    }
+
+    public Connection getConnection() throws SQLException{
+        Properties props = new Properties();
+        props.setProperty(QueryServices.IS_NAMESPACE_MAPPING_ENABLED, Boolean.toString(isNamespaceMapped));
+        return DriverManager.getConnection(getUrl(),props);
+    }
+
+    @Test
+    public void testSchemaPermissions() throws Throwable{
+
+        if (!isNamespaceMapped) { return; }
+        try {
+            startNewMiniCluster(null);
+            grantSystemTableAccess();
+            final String schemaName = "TEST_SCHEMA_PERMISSION";
+            superUser.doAs(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    try {
+                        AccessControlClient.grant(getUtility().getConnection(), regularUser.getShortUserName(),
+                                Action.ADMIN);
+                    } catch (Throwable e) {
+                        if (e instanceof Exception) {
+                            throw (Exception)e;
+                        } else {
+                            throw new Exception(e);
+                        }
+                    }
+                    return null;
+                }
+            });
+            verifyAllowed(createSchema(schemaName), regularUser);
+            // Unprivileged user cannot drop a schema
+            verifyDenied(dropSchema(schemaName), unprivilegedUser);
+            verifyDenied(createSchema(schemaName), unprivilegedUser);
+
+            verifyAllowed(dropSchema(schemaName), regularUser);
+        } finally {
+            revokeAll();
+        }
+    }
+
+    @Test
+    public void testAutomaticGrantDisabled() throws Throwable{
+        testIndexAndView(false);
+    }
+    
+    public void testIndexAndView(boolean isAutomaticGrant) throws Throwable {
+        Configuration conf = new Configuration();
+        conf.set(QueryServices.PHOENIX_AUTOMATIC_GRANT_ENABLED, Boolean.toString(isAutomaticGrant));
+        startNewMiniCluster(conf);
+        final String schema = "TEST_INDEX_VIEW";
+        final String tableName = "TABLE_DDL_PERMISSION_IT";
+        final String phoenixTableName = schema + "." + tableName;
+        final String indexName1 = tableName + "_IDX1";
+        final String indexName2 = tableName + "_IDX2";
+        final String lIndexName1 = tableName + "_LIDX1";
+        final String viewName1 = schema+"."+tableName + "_V1";
+        final String viewName2 = schema+"."+tableName + "_V2";
+        final String viewName3 = schema+"."+tableName + "_V3";
+        final String viewName4 = schema+"."+tableName + "_V4";
+        final String viewIndexName1 = tableName + "_VIDX1";
+        final String viewIndexName2 = tableName + "_VIDX2";
+        grantSystemTableAccess();
+        try {
+            superUser.doAs(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    try {
+                        verifyAllowed(createSchema(schema), superUser);
+                        if (isNamespaceMapped) {
+                            grantPermissions(regularUser.getShortUserName(), schema, Action.CREATE);
+                            grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS), schema, Action.CREATE);
+
+                        } else {
+                            grantPermissions(regularUser.getShortUserName(),
+                                    NamespaceDescriptor.DEFAULT_NAMESPACE.getName(), Action.CREATE);
+                            grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),
+                                    NamespaceDescriptor.DEFAULT_NAMESPACE.getName(), Action.CREATE);
+
+                        }
+                    } catch (Throwable e) {
+                        if (e instanceof Exception) {
+                            throw (Exception)e;
+                        } else {
+                            throw new Exception(e);
+                        }
+                    }
+                    return null;
+                }
+            });
+
+            verifyAllowed(createTable(phoenixTableName), regularUser);
+            verifyAllowed(createIndex(indexName1, phoenixTableName), regularUser);
+            verifyAllowed(createView(viewName1, phoenixTableName), regularUser);
+            verifyAllowed(createLocalIndex(lIndexName1, phoenixTableName), regularUser);
+            verifyAllowed(createIndex(viewIndexName1, viewName1), regularUser);
+            verifyAllowed(createIndex(viewIndexName2, viewName1), regularUser);
+            verifyAllowed(createView(viewName4, viewName1), regularUser);
+            verifyAllowed(readTable(phoenixTableName), regularUser);
+
+            verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+            verifyDenied(createView(viewName2, phoenixTableName), unprivilegedUser);
+            verifyDenied(createView(viewName3, viewName1), unprivilegedUser);
+            verifyDenied(dropView(viewName1), unprivilegedUser);
+            
+            verifyDenied(dropIndex(indexName1, phoenixTableName), unprivilegedUser);
+            verifyDenied(dropTable(phoenixTableName), unprivilegedUser);
+            verifyDenied(rebuildIndex(indexName1, phoenixTableName), unprivilegedUser);
+            verifyDenied(addColumn(phoenixTableName, "val1"), unprivilegedUser);
+            verifyDenied(dropColumn(phoenixTableName, "val"), unprivilegedUser);
+            verifyDenied(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), unprivilegedUser);
+
+            // Granting read permission to unprivileged user, now he should be able to create view but not index
+            grantPermissions(unprivilegedUser.getShortUserName(),
+                    Collections.singleton(
+                            SchemaUtil.getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getString()),
+                    Action.READ, Action.EXEC);
+            grantPermissions(AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),
+                    Collections.singleton(
+                            SchemaUtil.getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getString()),
+                    Action.READ, Action.EXEC);
+            verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+            if (!isAutomaticGrant) {
+                // Automatic grant will read access for all indexes
+                verifyDenied(createView(viewName2, phoenixTableName), unprivilegedUser);
+
+                // Granting read permission to unprivileged user on index so that a new view can read a index as well,
+                // now
+                // he should be able to create view but not index
+                grantPermissions(unprivilegedUser.getShortUserName(),
+                        Collections.singleton(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, indexName1, isNamespaceMapped).getString()),
+                        Action.READ, Action.EXEC);
+                verifyDenied(createView(viewName3, viewName1), unprivilegedUser);
+            }
+            
+            verifyAllowed(createView(viewName2, phoenixTableName), unprivilegedUser);
+            
+            if (!isAutomaticGrant) {
+                // Grant access to view index for parent view
+                grantPermissions(unprivilegedUser.getShortUserName(),
+                        Collections.singleton(Bytes.toString(MetaDataUtil.getViewIndexPhysicalName(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, tableName, isNamespaceMapped).getBytes()))),
+                        Action.READ, Action.EXEC);
+            }
+            verifyAllowed(createView(viewName3, viewName1), unprivilegedUser);
+            
+            // Grant create permission in namespace
+            if (isNamespaceMapped) {
+                grantPermissions(unprivilegedUser.getShortUserName(), schema, Action.CREATE);
+            } else {
+                grantPermissions(unprivilegedUser.getShortUserName(), NamespaceDescriptor.DEFAULT_NAMESPACE.getName(),
+                        Action.CREATE);
+            }
+            if (!isAutomaticGrant) {
+                verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+                // Give user of data table access to index table which will be created by unprivilegedUser
+                grantPermissions(regularUser.getShortUserName(),
+                        Collections.singleton(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, indexName2, isNamespaceMapped).getString()),
+                        Action.WRITE);
+                verifyDenied(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+                grantPermissions(regularUser.getShortUserName(),
+                        Collections.singleton(SchemaUtil
+                                .getPhysicalHBaseTableName(schema, indexName2, isNamespaceMapped).getString()),
+                        Action.WRITE, Action.READ, Action.CREATE, Action.EXEC, Action.ADMIN);
+            }
+            // we should be able to read the data from another index as well to which we have not given any access to
+            // this user
+            verifyAllowed(createIndex(indexName2, phoenixTableName), unprivilegedUser);
+            verifyAllowed(readTable(phoenixTableName, indexName1), unprivilegedUser);
+            verifyAllowed(readTable(phoenixTableName, indexName2), unprivilegedUser);
+            verifyAllowed(rebuildIndex(indexName2, phoenixTableName), unprivilegedUser);
+
+            // data table user should be able to read new index
+            verifyAllowed(rebuildIndex(indexName2, phoenixTableName), regularUser);
+            verifyAllowed(readTable(phoenixTableName, indexName2), regularUser);
+
+            verifyAllowed(readTable(phoenixTableName), regularUser);
+            verifyAllowed(rebuildIndex(indexName1, phoenixTableName), regularUser);
+            verifyAllowed(addColumn(phoenixTableName, "val1"), regularUser);
+            verifyAllowed(addProperties(phoenixTableName, "GUIDE_POSTS_WIDTH", "100"), regularUser);
+            verifyAllowed(dropView(viewName1), regularUser);
+            verifyAllowed(dropView(viewName2), regularUser);
+            verifyAllowed(dropColumn(phoenixTableName, "val1"), regularUser);
+            verifyAllowed(dropIndex(indexName2, phoenixTableName), regularUser);
+            verifyAllowed(dropIndex(indexName1, phoenixTableName), regularUser);
+            verifyAllowed(dropTable(phoenixTableName), regularUser);
+
+            // check again with super users
+            verifyAllowed(createTable(phoenixTableName), superUser2);
+            verifyAllowed(createIndex(indexName1, phoenixTableName), superUser2);
+            verifyAllowed(createView(viewName1, phoenixTableName), superUser2);
+            verifyAllowed(readTable(phoenixTableName), superUser2);
+            verifyAllowed(dropView(viewName1), superUser2);
+            verifyAllowed(dropTable(phoenixTableName), superUser2);
+
+        } finally {
+            revokeAll();
+        }
+    }
+    
+    
+    @Test
+    public void testAutomaticGrantEnabled() throws Throwable{
+        testIndexAndView(true);
+    }
+
+    private void revokeAll() throws IOException, Throwable {
+        AccessControlClient.revoke(getUtility().getConnection(), AuthUtil.toGroupEntry(GROUP_SYSTEM_ACCESS),Action.values() );
+        AccessControlClient.revoke(getUtility().getConnection(), regularUser.getShortUserName(),Action.values() );
+        AccessControlClient.revoke(getUtility().getConnection(), unprivilegedUser.getShortUserName(),Action.values() );
+        
+    }
+
+    protected void grantPermissions(String groupEntry, Action... actions) throws IOException, Throwable {
+        AccessControlClient.grant(getUtility().getConnection(), groupEntry, actions);
+    }
+
+    private AccessTestAction dropTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP TABLE IF EXISTS " + tableName));
+                }
+                return null;
+            }
+        };
+
+    }
+
+    private AccessTestAction createTable(final String tableName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+        try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+            assertFalse(stmt.execute("CREATE TABLE " + tableName + "(pk INTEGER not null primary key, data VARCHAR,val integer)"));
+            try (PreparedStatement pstmt = conn.prepareStatement("UPSERT INTO " + tableName + " values(?, ?, ?)")) {
+                for (int i = 0; i < NUM_RECORDS; i++) {
+                    pstmt.setInt(1, i);
+                    pstmt.setString(2, Integer.toString(i));
+                    pstmt.setInt(3, i);
+                    assertEquals(1, pstmt.executeUpdate());
+                }
+            }
+            conn.commit();
+        }
+        return null;
+            }
+        };
+    }
+
+    private AccessTestAction readTable(final String tableName) throws SQLException {
+        return readTable(tableName,null);
+    }
+    private AccessTestAction readTable(final String tableName, final String indexName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement()) {
+                    ResultSet rs = stmt.executeQuery("SELECT "+(indexName!=null?"/*+ INDEX("+tableName+" "+indexName+")*/":"")+" pk, data,val FROM " + tableName +" where data>='0'");
+                    assertNotNull(rs);
+                    int i = 0;
+                    while (rs.next()) {
+                        assertEquals(i, rs.getInt(1));
+                        assertEquals(Integer.toString(i), rs.getString(2));
+                        assertEquals(i, rs.getInt(3));
+                        i++;
+                    }
+                    assertEquals(NUM_RECORDS, i);
+                }
+                return null;
+                }
+            };
+    }
+
+    public static HBaseTestingUtility getUtility(){
+        return testUtil;
+    }
+
+    private void grantPermissions(String toUser, Set<String> tablesToGrant, Action... actions) throws Throwable {
+        for (String table : tablesToGrant) {
+            AccessControlClient.grant(getUtility().getConnection(), TableName.valueOf(table), toUser, null, null,
+                    actions);
+        }
+    }
+
+    private void grantPermissions(String toUser, String namespace, Action... actions) throws Throwable {
+        AccessControlClient.grant(getUtility().getConnection(), namespace, toUser, actions);
+    }
+    
+
+    private AccessTestAction dropColumn(final String tableName, final String columnName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " DROP COLUMN "+columnName));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction addColumn(final String tableName, final String columnName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " ADD "+columnName+" varchar"));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction addProperties(final String tableName, final String property, final String value)
+            throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER TABLE " + tableName + " SET " + property + "=" + value));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction dropView(final String viewName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP VIEW " + viewName));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction createView(final String viewName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE VIEW " + viewName + " AS SELECT * FROM " + dataTable));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction createIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE INDEX " + indexName + " on " + dataTable + "(data)"));
+                }
+                return null;
+            }
+        };
+    }
+    
+    private AccessTestAction createLocalIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("CREATE LOCAL INDEX " + indexName + " on " + dataTable + "(data)"));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction dropIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("DROP INDEX " + indexName + " on " + dataTable));
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction createSchema(final String schemaName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                if (isNamespaceMapped) {
+                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                        assertFalse(stmt.execute("CREATE SCHEMA " + schemaName));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction dropSchema(final String schemaName) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                if (isNamespaceMapped) {
+                    try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                        assertFalse(stmt.execute("DROP SCHEMA " + schemaName));
+                    }
+                }
+                return null;
+            }
+        };
+    }
+
+    private AccessTestAction rebuildIndex(final String indexName, final String dataTable) throws SQLException {
+        return new AccessTestAction() {
+            @Override
+            public Object run() throws Exception {
+                try (Connection conn = getConnection(); Statement stmt = conn.createStatement();) {
+                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " DISABLE"));
+                    assertFalse(stmt.execute("ALTER INDEX " + indexName + " on " + dataTable + " REBUILD"));
+                }
+                return null;
+            }
+        };
+    }
+
+    static interface AccessTestAction extends PrivilegedExceptionAction<Object> { }
+
+    @After
+    public void cleanup() throws Exception {
+        if (null != testUtil) {
+          testUtil.shutdownMiniCluster();
+          testUtil = null;
+        }
+    }
+
+    /** This fails only in case of ADE or empty list for any of the users. */
+    private void verifyAllowed(AccessTestAction action, UserGroupInformation... users) throws Exception {
+      for (UserGroupInformation user : users) {
+        verifyAllowed(user, action);
+      }
+    }
+
+    /** This passes only in case of ADE for all users. */
+    private void verifyDenied(AccessTestAction action, UserGroupInformation... users) throws Exception {
+      for (UserGroupInformation user : users) {
+        verifyDenied(user, action);
+      }
+    }
+
+    /** This fails only in case of ADE or empty list for any of the actions. */
+    private void verifyAllowed(UserGroupInformation user, AccessTestAction... actions) throws Exception {
+      for (AccessTestAction action : actions) {
+        try {
+          Object obj = user.doAs(action);
+          if (obj != null && obj instanceof List<?>) {
+            List<?> results = (List<?>) obj;
+            if (results != null && results.isEmpty()) {
+              fail("Empty non null results from action for user '" + user.getShortUserName() + "'");
+            }
+          }
+        } catch (AccessDeniedException ade) {
+          fail("Expected action to pass for user '" + user.getShortUserName() + "' but was denied");
+        }
+      }
+    }
+
+    /** This passes only in case of ADE for all actions. */
+    private void verifyDenied(UserGroupInformation user, AccessTestAction... actions) throws Exception {
+        for (AccessTestAction action : actions) {
+            try {
+                user.doAs(action);
+                fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
+            } catch (IOException e) {
+                fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
+            } catch (UndeclaredThrowableException ute) {
+                Throwable ex = ute.getUndeclaredThrowable();
+
+                if (ex instanceof PhoenixIOException) {
+                    if (ex.getCause() instanceof AccessDeniedException) {
+                        // expected result
+                        validateAccessDeniedException((AccessDeniedException) ex.getCause());
+                        return;
+                    }
+                }
+            }catch(RuntimeException ex){
+                // This can occur while accessing tabledescriptors from client by the unprivileged user
+                if (ex.getCause() instanceof AccessDeniedException) {
+                    // expected result
+                    validateAccessDeniedException((AccessDeniedException) ex.getCause());
+                    return;
+                }
+            }
+            fail("Expected exception was not thrown for user '" + user.getShortUserName() + "'");
+        }
+    }
+
+    private void validateAccessDeniedException(AccessDeniedException ade) {
+        String msg = ade.getMessage();
+        assertTrue("Exception contained unexpected message: '" + msg + "'",
+            !msg.contains("is not the scanner owner"));
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java
new file mode 100644
index 0000000..ac281f1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/hadoop/hbase/ipc/RpcUtil.java
@@ -0,0 +1,32 @@
+/*
+ * 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.hadoop.hbase.ipc;
+
+import org.apache.hadoop.hbase.ipc.RpcServer.Call;
+
+public class RpcUtil {
+
+    public static Call getRpcContext() {
+        return RpcServer.CurCall.get();
+    }
+    
+    public static void setRpcContext(Call c){
+        RpcServer.CurCall.set(c);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..8decc8c
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/BaseMetaDataEndpointObserver.java
@@ -0,0 +1,111 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.CoprocessorEnvironment;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+
+public class BaseMetaDataEndpointObserver implements MetaDataEndpointObserver{
+
+    @Override
+    public void start(CoprocessorEnvironment env) throws IOException {
+
+    }
+
+    @Override
+    public void stop(CoprocessorEnvironment env) throws IOException {
+
+    }
+
+    @Override
+    public void preGetTable(
+            org.apache.hadoop.hbase.coprocessor.ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,
+            String tenantId, String tableName, TableName physicalTableName) throws IOException {
+
+    }
+
+    
+    @Override
+    public void preCreateTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+            Set<byte[]> familySet, Set<TableName> indexes) throws IOException {
+
+    }
+
+    @Override
+    public void preDropTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType,
+            List<PTable> indexes) throws IOException {
+
+    }
+
+    @Override
+    public void preAlterTable(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String tableName, TableName physicalTableName, TableName parentPhysicalTableName, PTableType type) throws IOException {
+
+    }
+
+    @Override
+    public void preGetSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+
+    }
+
+    @Override
+    public void preCreateSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName)
+            throws IOException {
+
+    }
+
+    @Override
+    public void preDropSchema(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String schemaName) throws IOException {
+
+    }
+
+    @Override
+    public void preCreateFunction(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String functionName) throws IOException {
+
+    }
+
+    @Override
+    public void preDropFunction(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId, String functionName)
+            throws IOException {}
+
+    @Override
+    public void preGetFunctions(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId, String functionName)
+            throws IOException {
+
+    }
+
+    @Override
+    public void preIndexUpdate(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String indexName, TableName physicalTableName, TableName parentPhysicalTableName, PIndexState newState)
+            throws IOException {
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointImpl.java
----------------------------------------------------------------------
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 d05ab79..afbd63f 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
@@ -84,6 +84,7 @@ import static org.apache.phoenix.util.SchemaUtil.getVarCharLength;
 import static org.apache.phoenix.util.SchemaUtil.getVarChars;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
 import java.sql.Statement;
@@ -91,10 +92,12 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
+import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hbase.Cell;
@@ -105,6 +108,7 @@ import org.apache.hadoop.hbase.DoNotRetryIOException;
 import org.apache.hadoop.hbase.HConstants;
 import org.apache.hadoop.hbase.KeyValue;
 import org.apache.hadoop.hbase.KeyValue.Type;
+import org.apache.hadoop.hbase.TableName;
 import org.apache.hadoop.hbase.client.Delete;
 import org.apache.hadoop.hbase.client.Get;
 import org.apache.hadoop.hbase.client.HTableInterface;
@@ -121,9 +125,12 @@ import org.apache.hadoop.hbase.filter.FilterList;
 import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
+import org.apache.hadoop.hbase.ipc.RpcServer.Call;
+import org.apache.hadoop.hbase.ipc.RpcUtil;
 import org.apache.hadoop.hbase.regionserver.Region;
 import org.apache.hadoop.hbase.regionserver.Region.RowLock;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.hadoop.hbase.util.VersionInfo;
@@ -452,7 +459,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
     private static final int DEFAULT_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(DEFAULT_VALUE_KV);
     private static final int MIN_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MIN_VALUE_KV);
     private static final int MAX_VALUE_INDEX = FUNCTION_ARG_KV_COLUMNS.indexOf(MAX_VALUE_KV);
-    
+
     private static PName newPName(byte[] keyBuffer, int keyOffset, int keyLength) {
         if (keyLength <= 0) {
             return null;
@@ -463,6 +470,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
 
     private RegionCoprocessorEnvironment env;
 
+    private PhoenixMetaDataCoprocessorHost phoenixAccessCoprocessorHost;
+    private boolean accessCheckEnabled;
+
     /**
      * Stores a reference to the coprocessor environment provided by the
      * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this
@@ -480,6 +490,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         } else {
             throw new CoprocessorException("Must be loaded on a table region!");
         }
+        
+        phoenixAccessCoprocessorHost = new PhoenixMetaDataCoprocessorHost(this.env);
+        this.accessCheckEnabled = env.getConfiguration().getBoolean(QueryServices.PHOENIX_ACLS_ENABLED,
+                QueryServicesOptions.DEFAULT_PHOENIX_ACLS_ENABLED);
         logger.info("Starting Tracing-Metrics Systems");
         // Start the phoenix trace collection
         Tracing.addTraceMetricsSource();
@@ -523,6 +537,9 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             }
+            getCoprocessorHost().preGetTable(Bytes.toString(tenantId), SchemaUtil.getTableName(schemaName, tableName),
+                    TableName.valueOf(table.getPhysicalName().getBytes()));
+
             builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_ALREADY_EXISTS);
             long disableIndexTimestamp = table.getIndexDisableTimestamp();
             long minNonZerodisableIndexTimestamp = disableIndexTimestamp > 0 ? disableIndexTimestamp : Long.MAX_VALUE;
@@ -554,6 +571,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         }
     }
 
+    private PhoenixMetaDataCoprocessorHost getCoprocessorHost() {
+        return phoenixAccessCoprocessorHost;
+    }
+
     private PTable buildTable(byte[] key, ImmutableBytesPtr cacheKey, Region region,
             long clientTimeStamp, int clientVersion) throws IOException, SQLException {
         Scan scan = MetaDataUtil.newTableRowsScan(key, MIN_TABLE_TIMESTAMP, clientTimeStamp);
@@ -1317,12 +1338,14 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
      * @return null if the physical table row information is not present.
      * 
      */
-    private static Mutation getPhysicalTableForView(List<Mutation> tableMetadata, byte[][] parentSchemaTableNames) {
+    private static Mutation getPhysicalTableRowForView(List<Mutation> tableMetadata, byte[][] parentTenantSchemaTableNames, byte[][] physicalSchemaTableNames) {
         int size = tableMetadata.size();
         byte[][] rowKeyMetaData = new byte[3][];
         MetaDataUtil.getTenantIdAndSchemaAndTableName(tableMetadata, rowKeyMetaData);
         Mutation physicalTableRow = null;
+        Mutation parentTableRow = null;
         boolean physicalTableLinkFound = false;
+        boolean parentTableLinkFound = false;
         if (size >= 2) {
             int i = size - 1;
             while (i >= 1) {
@@ -1332,28 +1355,51 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (linkType == LinkType.PHYSICAL_TABLE) {
                         physicalTableRow = m;
                         physicalTableLinkFound = true;
-                        break;
                     }
+                    if (linkType == LinkType.PARENT_TABLE) {
+                        parentTableRow=m;
+                        parentTableLinkFound = true;
+                    }
+                }
+                if(physicalTableLinkFound && parentTableLinkFound){
+                    break;
                 }
                 i--;
             }
         }
+        if (!parentTableLinkFound) {
+            parentTenantSchemaTableNames[0] = null;
+            parentTenantSchemaTableNames[1] = null;
+            parentTenantSchemaTableNames[2] = null;
+            
+        }
         if (!physicalTableLinkFound) {
-            parentSchemaTableNames[0] = null;
-            parentSchemaTableNames[1] = null;
-            return null;
+            physicalSchemaTableNames[0] = null;
+            physicalSchemaTableNames[1] = null;
+            physicalSchemaTableNames[2] = null;
+        }
+        if (physicalTableLinkFound) {
+            getSchemaTableNames(physicalTableRow,physicalSchemaTableNames);
+        }
+        if (parentTableLinkFound) {
+            getSchemaTableNames(parentTableRow,parentTenantSchemaTableNames);   
         }
-        rowKeyMetaData = new byte[5][];
-        getVarChars(physicalTableRow.getRow(), 5, rowKeyMetaData);
+        return physicalTableRow;
+    }
+    
+    private static void getSchemaTableNames(Mutation row, byte[][] schemaTableNames) {
+        byte[][] rowKeyMetaData = new byte[5][];
+        getVarChars(row.getRow(), 5, rowKeyMetaData);
+        byte[] tenantId = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
         byte[] colBytes = rowKeyMetaData[PhoenixDatabaseMetaData.COLUMN_NAME_INDEX];
         byte[] famBytes = rowKeyMetaData[PhoenixDatabaseMetaData.FAMILY_NAME_INDEX];
         if ((colBytes == null || colBytes.length == 0) && (famBytes != null && famBytes.length > 0)) {
             byte[] sName = SchemaUtil.getSchemaNameFromFullName(famBytes).getBytes();
             byte[] tName = SchemaUtil.getTableNameFromFullName(famBytes).getBytes();
-            parentSchemaTableNames[0] = sName;
-            parentSchemaTableNames[1] = tName;
+            schemaTableNames[0]= tenantId;
+            schemaTableNames[1] = sName;
+            schemaTableNames[2] = tName;
         }
-        return physicalTableRow;
     }
     
     @Override
@@ -1370,25 +1416,76 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             byte[] tenantIdBytes = rowKeyMetaData[PhoenixDatabaseMetaData.TENANT_ID_INDEX];
             schemaName = rowKeyMetaData[PhoenixDatabaseMetaData.SCHEMA_NAME_INDEX];
             tableName = rowKeyMetaData[PhoenixDatabaseMetaData.TABLE_NAME_INDEX];
-
+            boolean isNamespaceMapped = MetaDataUtil.isNameSpaceMapped(tableMetadata, GenericKeyValueBuilder.INSTANCE,
+                    new ImmutableBytesWritable());
+            final IndexType indexType = MetaDataUtil.getIndexType(tableMetadata, GenericKeyValueBuilder.INSTANCE,
+                    new ImmutableBytesWritable());
             byte[] parentSchemaName = null;
             byte[] parentTableName = null;
             PTableType tableType = MetaDataUtil.getTableType(tableMetadata, GenericKeyValueBuilder.INSTANCE, new ImmutableBytesWritable());
             byte[] parentTableKey = null;
             Mutation viewPhysicalTableRow = null;
+            Set<TableName> indexes = new HashSet<TableName>();;
+            byte[] cPhysicalName = SchemaUtil.getPhysicalHBaseTableName(schemaName, tableName, isNamespaceMapped)
+                    .getBytes();
+            byte[] cParentPhysicalName=null;
             if (tableType == PTableType.VIEW) {
-                byte[][] parentSchemaTableNames = new byte[2][];
+                byte[][] parentSchemaTableNames = new byte[3][];
+                byte[][] parentPhysicalSchemaTableNames = new byte[3][];
                 /*
                  * For a view, we lock the base physical table row. For a mapped view, there is 
                  * no link present to the physical table. So the viewPhysicalTableRow is null
                  * in that case.
                  */
-                viewPhysicalTableRow = getPhysicalTableForView(tableMetadata, parentSchemaTableNames);
-                parentSchemaName = parentSchemaTableNames[0];
-                parentTableName = parentSchemaTableNames[1];
-                if (parentTableName != null) {
-                    parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY, parentSchemaName, parentTableName);
+                
+                viewPhysicalTableRow = getPhysicalTableRowForView(tableMetadata, parentSchemaTableNames,parentPhysicalSchemaTableNames);
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                if (parentPhysicalSchemaTableNames[2] != null) {
+                    
+                    parentTableKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
+                            parentPhysicalSchemaTableNames[1], parentPhysicalSchemaTableNames[2]);
+                    PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                            clientTimeStamp, clientTimeStamp, clientVersion);
+                    if (parentTable == null) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    cParentPhysicalName = parentTable.getPhysicalName().getBytes();
+                    if (parentSchemaTableNames[2] != null
+                            && Bytes.compareTo(parentSchemaTableNames[2], parentPhysicalSchemaTableNames[2]) != 0) {
+                        // if view is created on view
+                        byte[] parentKey = SchemaUtil.getTableKey(
+                                parentSchemaTableNames[0] == null ? ByteUtil.EMPTY_BYTE_ARRAY : parentSchemaTableNames[0],
+                                parentSchemaTableNames[1], parentSchemaTableNames[2]);
+                        parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                                clientTimeStamp, clientTimeStamp, clientVersion);
+                        if (parentTable == null) {
+                            // it could be a global view
+                            parentKey = SchemaUtil.getTableKey(ByteUtil.EMPTY_BYTE_ARRAY,
+                                    parentSchemaTableNames[1], parentSchemaTableNames[2]);
+                            parentTable = loadTable(env, parentKey, new ImmutableBytesPtr(parentKey),
+                                    clientTimeStamp, clientTimeStamp, clientVersion);
+                        }
+                    }
+                    if (parentTable == null) {
+                        builder.setReturnCode(MetaDataProtos.MutationCode.PARENT_TABLE_NOT_FOUND);
+                        builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                        done.run(builder.build());
+                        return;
+                    }
+                    for (PTable index : parentTable.getIndexes()) {
+                        indexes.add(TableName.valueOf(index.getPhysicalName().getBytes()));
+                    }
+
+                } else {
+                    // Mapped View
+                    cParentPhysicalName = SchemaUtil.getTableNameAsBytes(schemaName, tableName);
                 }
+                parentSchemaName = parentPhysicalSchemaTableNames[1];
+                parentTableName = parentPhysicalSchemaTableNames[2];
+                    
             } else if (tableType == PTableType.INDEX) {
                 parentSchemaName = schemaName;
                 /* 
@@ -1398,7 +1495,27 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                  */ 
                 parentTableName = MetaDataUtil.getParentTableName(tableMetadata);
                 parentTableKey = SchemaUtil.getTableKey(tenantIdBytes, parentSchemaName, parentTableName);
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                PTable parentTable = loadTable(env, parentTableKey, new ImmutableBytesPtr(parentTableKey),
+                        clientTimeStamp, clientTimeStamp, clientVersion);
+                if (IndexType.LOCAL == indexType) {
+                    cPhysicalName = parentTable.getPhysicalName().getBytes();
+                    cParentPhysicalName=parentTable.getPhysicalName().getBytes();
+                } else if (parentTable.getType() == PTableType.VIEW) {
+                    cPhysicalName = MetaDataUtil.getViewIndexPhysicalName(parentTable.getPhysicalName().getBytes());
+                    cParentPhysicalName = parentTable.getPhysicalName().getBytes();
+                }else{
+                    cParentPhysicalName = SchemaUtil
+                            .getPhysicalHBaseTableName(parentSchemaName, parentTableName, isNamespaceMapped).getBytes();
+                }
             }
+            
+            getCoprocessorHost().preCreateTable(Bytes.toString(tenantIdBytes),
+                    SchemaUtil.getTableName(schemaName, tableName),
+                    (tableType == PTableType.VIEW) ? null : TableName.valueOf(cPhysicalName),
+                    cParentPhysicalName == null ? null : TableName.valueOf(cParentPhysicalName), tableType,
+                    /* TODO: During inital create we may not need the family map */
+                    Collections.<byte[]> emptySet(), indexes);
 
             Region region = env.getRegion();
             List<RowLock> locks = Lists.newArrayList();
@@ -1613,7 +1730,7 @@ 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.
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, tableMetadata, 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
@@ -1632,7 +1749,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             logger.error("createTable failed", t);
@@ -1648,16 +1765,6 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 QueryServicesOptions.DEFAULT_MAX_INDEXES_PER_TABLE);
     }
 
-    private static RowLock acquireLock(Region region, byte[] key, List<RowLock> locks)
-        throws IOException {
-        RowLock rowLock = region.getRowLock(key, false);
-        if (rowLock == null) {
-            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
-        }
-        locks.add(rowLock);
-        return rowLock;
-    }
-
     private static final byte[] CHILD_TABLE_BYTES = new byte[] {PTable.LinkType.CHILD_TABLE.getSerializedValue()};
 
     
@@ -1846,6 +1953,23 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     parentTableName == null ? lockKey : SchemaUtil.getTableKey(tenantIdBytes,
                         schemaName, tableName);
 
+            
+            PTableType ptableType=PTableType.fromSerializedValue(tableType);
+            long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+            byte[] cKey = SchemaUtil.getTableKey(tenantIdBytes, schemaName, tableName);
+            PTable loadedTable = loadTable(env, cKey, new ImmutableBytesPtr(cKey), clientTimeStamp, clientTimeStamp,
+                    request.getClientVersion());
+            if (loadedTable == null) {
+                builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+                builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                done.run(builder.build());
+                return;
+            }
+            getCoprocessorHost().preDropTable(Bytes.toString(tenantIdBytes),
+                    SchemaUtil.getTableName(schemaName, tableName),
+                    TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
+                    getParentPhysicalTableName(loadedTable), ptableType,loadedTable.getIndexes());
+
             Region region = env.getRegion();
             MetaDataMutationResult result = checkTableKeyInRegion(key, region);
             if (result != null) {
@@ -1870,7 +1994,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 // Commit the list of deletion.
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                     HConstants.NO_NONCE);
                 long currentTime = MetaDataUtil.getClientTimeStamp(tableMetadata);
                 for (ImmutableBytesPtr ckey : invalidateList) {
@@ -1883,7 +2007,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
           logger.error("dropTable failed", t);
@@ -1891,6 +2015,24 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 ServerUtil.createIOException(SchemaUtil.getTableName(schemaName, tableName), t));
         }
     }
+    
+    protected void releaseRowLocks(Region region, List<RowLock> locks) {
+        if (locks != null) {
+            region.releaseRowLocks(locks);
+        }
+    }
+
+    private RowLock acquireLock(Region region, byte[] lockKey, List<RowLock> locks) throws IOException {
+        //LockManager.RowLock rowLock = lockManager.lockRow(lockKey, rowLockWaitDuration);
+        RowLock rowLock = region.getRowLock(lockKey, false);
+        if (rowLock == null) {
+            throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(lockKey));
+        }
+        if (locks != null) {
+            locks.add(rowLock);
+        }
+        return rowLock;
+    }
 
     private MetaDataMutationResult doDropTable(byte[] key, byte[] tenantId, byte[] schemaName,
         byte[] tableName, byte[] parentTableName, PTableType tableType, List<Mutation> rowsToDelete,
@@ -2093,18 +2235,15 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                             EnvironmentEdgeManager.currentTimeMillis(), null);
                 }
                 if (table.getTimeStamp() >= clientTimeStamp) {
-                    logger.info("Found newer table as of " + table.getTimeStamp() + " versus client timestamp of " + clientTimeStamp);
+                    logger.info("Found newer table as of " + table.getTimeStamp() + " versus client timestamp of "
+                            + clientTimeStamp);
                     return new MetaDataMutationResult(MutationCode.NEWER_TABLE_FOUND,
                             EnvironmentEdgeManager.currentTimeMillis(), table);
-                } else if (isTableDeleted(table)) {
-                    return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
-                            EnvironmentEdgeManager.currentTimeMillis(), null);
-                }
-
-                long expectedSeqNum = MetaDataUtil.getSequenceNumber(tableMetadata) - 1; // lookup
-                                                                                         // TABLE_SEQ_NUM
-                                                                                         // in
+                } else if (isTableDeleted(table)) { return new MetaDataMutationResult(MutationCode.TABLE_NOT_FOUND,
+                        EnvironmentEdgeManager.currentTimeMillis(), null); }
+                long expectedSeqNum = MetaDataUtil.getSequenceNumber(tableMetadata) - 1; // lookup TABLE_SEQ_NUM in
                                                                                          // tableMetaData
+
                 if (logger.isDebugEnabled()) {
                     logger.debug("For table " + Bytes.toStringBinary(key) + " expecting seqNum "
                             + expectedSeqNum + " and found seqNum " + table.getSequenceNumber()
@@ -2139,7 +2278,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 if (result != null && result.getMutationCode()!=MutationCode.TABLE_ALREADY_EXISTS) {
                     return result;
                 }
-                region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 // Invalidate from cache
                 for (ImmutableBytesPtr invalidateKey : invalidateList) {
                     metaDataCache.invalidate(invalidateKey);
@@ -2155,7 +2294,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     return new MetaDataMutationResult(MutationCode.TABLE_ALREADY_EXISTS, currentTime, table);
                 }
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             ServerUtil.throwIOException(SchemaUtil.getTableName(schemaName, tableName), t);
@@ -2971,6 +3110,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     PTableType type = table.getType();
                     byte[] tableHeaderRowKey = SchemaUtil.getTableKey(tenantId,
                             schemaName, tableName);
+                    byte[] cPhysicalTableName=table.getPhysicalName().getBytes();
+                    getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
+                            SchemaUtil.getTableName(schemaName, tableName), TableName.valueOf(cPhysicalTableName),
+                            getParentPhysicalTableName(table),type);
+
                     // Size for worst case - all new columns are PK column
                     List<Mutation> mutationsForAddingColumnsToViews = Lists.newArrayListWithExpectedSize(tableMetaData.size() * ( 1 + table.getIndexes().size()));
                     if (type == PTableType.TABLE || type == PTableType.SYSTEM) {
@@ -3124,10 +3268,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         boolean blockWriteRebuildIndex = env.getConfiguration().getBoolean(QueryServices.INDEX_FAILURE_BLOCK_WRITE, 
                 QueryServicesOptions.DEFAULT_INDEX_FAILURE_BLOCK_WRITE);
         if (!wasLocked) {
-            rowLock = region.getRowLock(key, false);
-            if (rowLock == null) {
-                throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
-            }
+            rowLock = acquireLock(region, key, null);
         }
         try {
             PTable table = (PTable)metaDataCache.getIfPresent(cacheKey);
@@ -3184,16 +3325,10 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
          * Lock directly on key, though it may be an index table. This will just prevent a table
          * from getting rebuilt too often.
          */
-        List<RowLock> rowLocks = new ArrayList<Region.RowLock>(keys.size());;
+        List<RowLock> rowLocks = new ArrayList<RowLock>(keys.size());;
         try {
-            rowLocks = new ArrayList<Region.RowLock>(keys.size());
             for (int i = 0; i < keys.size(); i++) {
-                Region.RowLock rowLock = region.getRowLock(keys.get(i), false);
-                if (rowLock == null) {
-                    throw new IOException("Failed to acquire lock on "
-                            + Bytes.toStringBinary(keys.get(i)));
-                }
-                rowLocks.add(rowLock);
+                acquireLock(region, keys.get(i), rowLocks);
             }
 
             List<PFunction> functionsAvailable = new ArrayList<PFunction>(keys.size());
@@ -3223,10 +3358,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             if(functionsAvailable.size() == numFunctions) return functionsAvailable;
             return null;
         } finally {
-            for (Region.RowLock lock : rowLocks) {
-                lock.release();
-            }
-            rowLocks.clear();
+            releaseRowLocks(region,rowLocks);
         }
     }
 
@@ -3248,6 +3380,11 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     byte[] schemaName = rowKeyMetaData[SCHEMA_NAME_INDEX];
                     byte[] tableName = rowKeyMetaData[TABLE_NAME_INDEX];
                     boolean deletePKColumn = false;
+                    getCoprocessorHost().preAlterTable(Bytes.toString(tenantId),
+                            SchemaUtil.getTableName(schemaName, tableName),
+                            TableName.valueOf(table.getPhysicalName().getBytes()),
+                            getParentPhysicalTableName(table),table.getType());
+
                     List<Mutation> additionalTableMetaData = Lists.newArrayList();
                     
                     PTableType type = table.getType();
@@ -3480,7 +3617,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             }
             PIndexState newState =
                     PIndexState.fromSerializedValue(newKV.getValueArray()[newKV.getValueOffset()]);
-            RowLock rowLock = region.getRowLock(key, false);
+            RowLock rowLock = acquireLock(region, key, null);
             if (rowLock == null) {
                 throw new IOException("Failed to acquire lock on " + Bytes.toStringBinary(key));
             }
@@ -3502,6 +3639,22 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 Cell currentDisableTimeStamp = currentResult.getColumnLatestCell(TABLE_FAMILY_BYTES, INDEX_DISABLE_TIMESTAMP_BYTES);
                 boolean rowKeyOrderOptimizable = currentResult.getColumnLatestCell(TABLE_FAMILY_BYTES, ROW_KEY_ORDER_OPTIMIZABLE_BYTES) != null;
 
+                //check permission on data table
+                long clientTimeStamp = MetaDataUtil.getClientTimeStamp(tableMetadata);
+                PTable loadedTable = loadTable(env, key, new ImmutableBytesPtr(key), clientTimeStamp, clientTimeStamp,
+                        request.getClientVersion());
+                if (loadedTable == null) {
+                    builder.setReturnCode(MetaDataProtos.MutationCode.TABLE_NOT_FOUND);
+                    builder.setMutationTime(EnvironmentEdgeManager.currentTimeMillis());
+                    done.run(builder.build());
+                    return;
+                }
+                getCoprocessorHost().preIndexUpdate(Bytes.toString(tenantId),
+                        SchemaUtil.getTableName(schemaName, tableName),
+                        TableName.valueOf(loadedTable.getPhysicalName().getBytes()),
+                        getParentPhysicalTableName(loadedTable),
+                        newState);
+
                 PIndexState currentState =
                         PIndexState.fromSerializedValue(currentStateKV.getValueArray()[currentStateKV
                                 .getValueOffset()]);
@@ -3611,7 +3764,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     if (setRowKeyOrderOptimizableCell) {
                         UpgradeUtil.addRowKeyOrderOptimizableCell(tableMetadata, key, timeStamp);
                     }
-                    region.mutateRowsWithLocks(tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                    mutateRowsWithLocks(region, tableMetadata, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
                     // Invalidate from cache
                     Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
@@ -3772,6 +3925,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         long clientTimeStamp = request.getClientTimestamp();
         List<RowLock> locks = Lists.newArrayList();
         try {
+            getCoprocessorHost().preGetSchema(schemaName);
             acquireLock(region, lockKey, locks);
             // Get as of latest timestamp so we can detect if we have a
             // newer schema that already
@@ -3802,7 +3956,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
             done.run(builder.build());
             return;
         } finally {
-            region.releaseRowLocks(locks);
+            releaseRowLocks(region,locks);
         }
     }
 
@@ -3905,7 +4059,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 }
                 // Don't store function info for temporary functions.
                 if(!temporaryFunction) {
-                    region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                    mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
                 }
 
                 // Invalidate the cache - the next getFunction call will add it
@@ -3919,7 +4073,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
           logger.error("createFunction failed", t);
@@ -3958,7 +4112,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                region.mutateRowsWithLocks(functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
+                mutateRowsWithLocks(region, functionMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE, HConstants.NO_NONCE);
 
                 Cache<ImmutableBytesPtr,PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env).getMetaDataCache();
                 long currentTime = MetaDataUtil.getClientTimeStamp(functionMetaData);
@@ -3971,7 +4125,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
           logger.error("dropFunction failed", t);
@@ -4068,7 +4222,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                         return;
                     }
                 }
-                region.mutateRowsWithLocks(schemaMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, schemaMutations, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
 
                 // Invalidate the cache - the next getSchema call will add it
@@ -4086,7 +4240,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(builder.build());
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             logger.error("Creating the schema" + schemaName + "failed", t);
@@ -4100,6 +4254,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
         try {
             List<Mutation> schemaMetaData = ProtobufUtil.getMutations(request);
             schemaName = request.getSchemaName();
+            getCoprocessorHost().preDropSchema(schemaName);
             byte[] lockKey = SchemaUtil.getSchemaKey(schemaName);
             Region region = env.getRegion();
             MetaDataMutationResult result = checkSchemaKeyInRegion(lockKey, region);
@@ -4117,7 +4272,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                     done.run(MetaDataMutationResult.toProto(result));
                     return;
                 }
-                region.mutateRowsWithLocks(schemaMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
+                mutateRowsWithLocks(region, schemaMetaData, Collections.<byte[]> emptySet(), HConstants.NO_NONCE,
                         HConstants.NO_NONCE);
                 Cache<ImmutableBytesPtr, PMetaDataEntity> metaDataCache = GlobalCache.getInstance(this.env)
                         .getMetaDataCache();
@@ -4129,7 +4284,7 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 done.run(MetaDataMutationResult.toProto(result));
                 return;
             } finally {
-                region.releaseRowLocks(locks);
+                releaseRowLocks(region,locks);
             }
         } catch (Throwable t) {
             logger.error("drop schema failed:", t);
@@ -4175,4 +4330,48 @@ public class MetaDataEndpointImpl extends MetaDataProtocol implements Coprocesso
                 null);
 
     }
+    
+    private void mutateRowsWithLocks(final Region region, final List<Mutation> mutations, final Set<byte[]> rowsToLock,
+            final long nonceGroup, final long nonce) throws IOException {
+        // we need to mutate SYSTEM.CATALOG with HBase/login user if access is enabled.
+        if (this.accessCheckEnabled) {
+            User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+                @Override
+                public Void run() throws Exception {
+                    final Call rpcContext = RpcUtil.getRpcContext();
+                    // Setting RPC context as null so that user can be resetted
+                    try {
+                        RpcUtil.setRpcContext(null);
+                        region.mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce);
+                    } catch (Throwable e) {
+                        throw new IOException(e);
+                    } finally {
+                        // Setting RPC context back to original context of the RPC
+                        RpcUtil.setRpcContext(rpcContext);
+                    }
+                    return null;
+                }
+            });
+        } else {
+            region.mutateRowsWithLocks(mutations, rowsToLock, nonceGroup, nonce);
+        }
+    }
+    
+    private TableName getParentPhysicalTableName(PTable table) {
+        return table
+                .getType() == PTableType.VIEW
+                        ? TableName.valueOf(table.getPhysicalName().getBytes())
+                        : table.getType() == PTableType.INDEX
+                                ? TableName
+                                        .valueOf(SchemaUtil
+                                                .getPhysicalHBaseTableName(table.getParentSchemaName(),
+                                                        table.getParentTableName(), table.isNamespaceMapped())
+                                                .getBytes())
+                                : TableName
+                                        .valueOf(
+                                                SchemaUtil
+                                                        .getPhysicalHBaseTableName(table.getSchemaName(),
+                                                                table.getTableName(), table.isNamespaceMapped())
+                                                        .getBytes());
+    }
 }

http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
----------------------------------------------------------------------
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
new file mode 100644
index 0000000..86b8bf1
--- /dev/null
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataEndpointObserver.java
@@ -0,0 +1,68 @@
+/*
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hbase.Coprocessor;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.coprocessor.ObserverContext;
+import org.apache.phoenix.coprocessor.PhoenixMetaDataCoprocessorHost.PhoenixMetaDataControllerEnvironment;
+import org.apache.phoenix.schema.PIndexState;
+import org.apache.phoenix.schema.PTable;
+import org.apache.phoenix.schema.PTableType;
+
+public interface MetaDataEndpointObserver extends Coprocessor {
+
+    void preGetTable( ObserverContext<PhoenixMetaDataControllerEnvironment> ctx,  String tenantId,String tableName,
+             TableName physicalTableName) throws IOException;
+
+    void preCreateTable(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            String tableName, TableName physicalTableName, final TableName parentPhysicalTableName,
+            PTableType tableType, final Set<byte[]> familySet, Set<TableName> indexes) throws IOException;
+
+    void preDropTable(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String tableName,TableName physicalTableName, TableName parentPhysicalTableName, PTableType tableType, List<PTable> indexes) throws IOException;
+
+    void preAlterTable(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,final String tableName,
+            final TableName physicalTableName,final TableName parentPhysicalTableName, PTableType type) throws IOException;
+
+    void preGetSchema(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String schemaName)
+            throws IOException;
+
+    void preCreateSchema(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String schemaName)
+            throws IOException;
+
+    void preDropSchema(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String schemaName)
+            throws IOException;
+
+    void preCreateFunction(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String functionName) throws IOException;
+
+    void preDropFunction(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String functionName) throws IOException;
+
+    void preGetFunctions(final ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, final String tenantId,
+            final String functionName) throws IOException;
+
+    void preIndexUpdate(ObserverContext<PhoenixMetaDataControllerEnvironment> ctx, String tenantId,
+            String indexName, TableName physicalTableName, TableName parentPhysicalTableName, PIndexState newState) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/phoenix/blob/217867c7/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
----------------------------------------------------------------------
diff --git a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
index c816549..af06235 100644
--- a/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
+++ b/phoenix-core/src/main/java/org/apache/phoenix/coprocessor/MetaDataRegionObserver.java
@@ -20,6 +20,7 @@ package org.apache.phoenix.coprocessor;
 import static org.apache.phoenix.schema.types.PDataType.TRUE_BYTES;
 
 import java.io.IOException;
+import java.security.PrivilegedExceptionAction;
 import java.sql.SQLException;
 import java.util.ArrayList;
 import java.util.Collections;
@@ -50,6 +51,7 @@ import org.apache.hadoop.hbase.filter.CompareFilter;
 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
 import org.apache.hadoop.hbase.io.ImmutableBytesWritable;
 import org.apache.hadoop.hbase.regionserver.RegionScanner;
+import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Pair;
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
@@ -163,9 +165,18 @@ public class MetaDataRegionObserver extends BaseRegionObserver {
                             SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_CATALOG_NAME_BYTES, props));
                     statsTable = env.getTable(
                             SchemaUtil.getPhysicalName(PhoenixDatabaseMetaData.SYSTEM_STATS_NAME_BYTES, props));
-                    if (UpgradeUtil.truncateStats(metaTable, statsTable)) {
-                        LOG.info("Stats are successfully truncated for upgrade 4.7!!");
-                    }
+                    final HTableInterface mTable=metaTable;
+                    final HTableInterface sTable=statsTable;
+                    User.runAsLoginUser(new PrivilegedExceptionAction<Void>() {
+                        @Override
+                        public Void run() throws Exception {
+                            if (UpgradeUtil.truncateStats(mTable, sTable)) {
+                                LOG.info("Stats are successfully truncated for upgrade 4.7!!");
+                            }
+                            return null;
+                        }
+                    });
+
                 } catch (Exception exception) {
                     LOG.warn("Exception while truncate stats..,"
                             + " please check and delete stats manually inorder to get proper result with old client!!");