You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by th...@apache.org on 2015/05/29 22:14:25 UTC

hive git commit: HS2 metadata api calls should use HiveAuthorizer interface for authorization (Thejas Nair, reviewed by Vaibhav Gumashta)

Repository: hive
Updated Branches:
  refs/heads/master db14cdf04 -> 317612b59


HS2 metadata api calls should use HiveAuthorizer interface for authorization (Thejas Nair, reviewed by Vaibhav Gumashta)


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/317612b5
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/317612b5
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/317612b5

Branch: refs/heads/master
Commit: 317612b59a8e03182526977f27c5e6e22aa4d56d
Parents: db14cdf
Author: Thejas Nair <th...@hortonworks.com>
Authored: Fri May 29 13:14:19 2015 -0700
Committer: Thejas Nair <th...@hortonworks.com>
Committed: Fri May 29 13:14:19 2015 -0700

----------------------------------------------------------------------
 .../plugin/TestHiveAuthorizerShowFilters.java   |   2 +-
 .../authorization/TestJdbcMetadataApiAuth.java  | 260 +++++++++++++++++++
 .../TestJdbcWithSQLAuthorization.java           |  18 ++
 .../AuthorizationMetaStoreFilterHook.java       |   9 +-
 .../authorization/plugin/HiveOperationType.java |  19 +-
 .../plugin/HivePrivilegeObjectUtils.java        |  51 ++++
 .../plugin/sqlstd/Operation2Privilege.java      |  17 +-
 .../cli/operation/GetCatalogsOperation.java     |  12 +-
 .../cli/operation/GetColumnsOperation.java      |  34 +++
 .../cli/operation/GetFunctionsOperation.java    |  25 ++
 .../cli/operation/GetSchemasOperation.java      |  13 +
 .../cli/operation/GetTableTypesOperation.java   |   4 +
 .../cli/operation/GetTablesOperation.java       |  10 +
 .../cli/operation/GetTypeInfoOperation.java     |   4 +
 .../cli/operation/MetadataOperation.java        |  35 +++
 15 files changed, 498 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
index ddb4730..d87f74f 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/authorization/plugin/TestHiveAuthorizerShowFilters.java
@@ -43,7 +43,7 @@ import org.junit.Test;
 import org.mockito.Mockito;
 
 /**
- * Test HiveAuthorizer api invocation
+ * Test HiveAuthorizer api invocation for filtering objects
  */
 public class TestHiveAuthorizerShowFilters {
   protected static HiveConf conf;

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcMetadataApiAuth.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcMetadataApiAuth.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcMetadataApiAuth.java
new file mode 100644
index 0000000..19b311d
--- /dev/null
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcMetadataApiAuth.java
@@ -0,0 +1,260 @@
+/**
+ * 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.hive.jdbc.authorization;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.List;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.SessionStateUserAuthenticator;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizer;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthorizerImpl;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzSessionContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveMetastoreClientFactory;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAccessControllerWrapper;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.sqlstd.SQLStdHiveAuthorizationValidator;
+import org.apache.hive.jdbc.miniHS2.MiniHS2;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * Verify validation of jdbc metadata methods is happening
+ */
+public class TestJdbcMetadataApiAuth {
+  private static MiniHS2 miniHS2 = null;
+
+  /**
+   * HiveAuthorizationValidator that allows/disallows actions based on
+   * allowActions boolean value
+   */
+  public static class TestAuthValidator extends SQLStdHiveAuthorizationValidator {
+
+    public static boolean allowActions;
+    public static final String DENIED_ERR = "Actions not allowed because of allowActions=false";
+
+    public TestAuthValidator(HiveMetastoreClientFactory metastoreClientFactory, HiveConf conf,
+        HiveAuthenticationProvider authenticator,
+        SQLStdHiveAccessControllerWrapper privilegeManager, HiveAuthzSessionContext ctx)
+        throws HiveAuthzPluginException {
+      super(metastoreClientFactory, conf, authenticator, privilegeManager, ctx);
+    }
+
+    @Override
+    public void checkPrivileges(HiveOperationType hiveOpType, List<HivePrivilegeObject> inputHObjs,
+        List<HivePrivilegeObject> outputHObjs, HiveAuthzContext context)
+        throws HiveAuthzPluginException, HiveAccessControlException {
+      if (!allowActions) {
+        throw new HiveAccessControlException(DENIED_ERR);
+      }
+    }
+  }
+
+  /**
+   * Factory that uses TestAuthValidator
+   */
+  public static class TestAuthorizerFactory implements HiveAuthorizerFactory {
+    @Override
+    public HiveAuthorizer createHiveAuthorizer(HiveMetastoreClientFactory metastoreClientFactory,
+        HiveConf conf, HiveAuthenticationProvider authenticator, HiveAuthzSessionContext ctx)
+        throws HiveAuthzPluginException {
+      SQLStdHiveAccessControllerWrapper privilegeManager = new SQLStdHiveAccessControllerWrapper(
+          metastoreClientFactory, conf, authenticator, ctx);
+      return new HiveAuthorizerImpl(privilegeManager, new TestAuthValidator(metastoreClientFactory,
+          conf, authenticator, privilegeManager, ctx));
+    }
+  }
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    Class.forName(MiniHS2.getJdbcDriverName());
+    HiveConf conf = new HiveConf();
+    conf.setVar(ConfVars.HIVE_AUTHORIZATION_MANAGER, TestAuthorizerFactory.class.getName());
+    conf.setVar(ConfVars.HIVE_AUTHENTICATOR_MANAGER, SessionStateUserAuthenticator.class.getName());
+    conf.setBoolVar(ConfVars.HIVE_AUTHORIZATION_ENABLED, true);
+    conf.setBoolVar(ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
+    conf.setBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS, false);
+
+    miniHS2 = new MiniHS2(conf);
+    miniHS2.start(new HashMap<String, String>());
+
+    TestAuthValidator.allowActions = true;
+    // set up a db and table
+    String tableName1 = TestJdbcMetadataApiAuth.class.getSimpleName() + "_tab";
+    String dbName1 = TestJdbcMetadataApiAuth.class.getSimpleName() + "_db";
+    // create connection as user1
+    Connection hs2Conn = getConnection("user1");
+    Statement stmt = hs2Conn.createStatement();
+
+    // create table, db
+    stmt.execute("create table " + tableName1 + "(i int) ");
+    stmt.execute("create database " + dbName1);
+    stmt.close();
+    hs2Conn.close();
+  }
+
+  @AfterClass
+  public static void afterTest() throws Exception {
+    if (miniHS2.isStarted()) {
+      miniHS2.stop();
+    }
+  }
+
+  /**
+   * Call the HS2 metadata api's with authorizer allowing those calls
+   * @throws Exception
+   */
+  @Test
+  public void testMetaApiAllowed() throws Exception {
+    TestAuthValidator.allowActions = true;
+
+    Connection hs2Conn = getConnection("user1");
+    DatabaseMetaData dbmetadata = hs2Conn.getMetaData();
+    ResultSet res;
+
+    res = dbmetadata.getCatalogs();
+    assertFalse(res.next());
+
+    res = dbmetadata.getSchemas();
+    assertTrue(res.next());
+    assertTrue(res.next());
+
+    res = dbmetadata.getTypeInfo();
+    assertTrue(res.next());
+
+    res = dbmetadata.getTables(null, "default", "t%", null);
+    assertTrue(res.next());
+
+    res = dbmetadata.getTableTypes();
+    assertTrue(res.next());
+
+    res = dbmetadata.getColumns(null, "default", "nosuchtable", null);
+    assertFalse(res.next());
+
+    res = dbmetadata.getFunctions(null, null, "trim");
+    assertTrue(res.next());
+
+  }
+
+  /**
+   * Call the HS2 metadata api's with authorizer disallowing those calls
+   * @throws Exception
+   */
+
+  @Test
+  public void testMetaApiDisAllowed() throws Exception {
+    TestAuthValidator.allowActions = false;
+
+    Connection hs2Conn = getConnection("user1");
+    DatabaseMetaData dbmetadata = hs2Conn.getMetaData();
+
+    try {
+      dbmetadata.getCatalogs();
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+    try {
+      dbmetadata.getSchemas();
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+    try {
+      dbmetadata.getTypeInfo();
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+    try {
+      dbmetadata.getTables(null, "default", "t%", null);
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+    try {
+      dbmetadata.getTableTypes();
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+    try {
+      dbmetadata.getColumns(null, "default", "nosuchtable", null);
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+    try {
+      dbmetadata.getFunctions(null, null, "trim");
+      fail("HiveAccessControlException expected");
+    } catch (SQLException e) {
+      assertErrorContains(e, TestAuthValidator.DENIED_ERR);
+    } catch (Exception e) {
+      fail("HiveAccessControlException expected");
+    }
+
+  }
+
+  private void assertErrorContains(SQLException e, String deniedErr) {
+    if(!e.getMessage().contains(deniedErr)) {
+      fail("Exception message [" + e.getMessage() + "] does not contain [" + deniedErr + "]");
+    }
+  }
+
+  private static Connection getConnection(String userName) throws SQLException {
+    return DriverManager.getConnection(miniHS2.getJdbcURL(), userName, "bar");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
----------------------------------------------------------------------
diff --git a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
index bd0ba0e..dacde45 100644
--- a/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
+++ b/itests/hive-unit/src/test/java/org/apache/hive/jdbc/authorization/TestJdbcWithSQLAuthorization.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 
 import java.sql.Connection;
 import java.sql.DriverManager;
+import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.util.HashMap;
@@ -90,6 +91,23 @@ public class TestJdbcWithSQLAuthorization {
     }
 
     {
+      // try using jdbc metadata api to get column list as user2 - should fail
+      Connection hs2Conn = getConnection("user2");
+      try {
+        hs2Conn.getMetaData().getColumns(null, "default", tableName2, null);
+        fail("Exception due to authorization failure is expected");
+      } catch (SQLException e) {
+        String msg = e.getMessage();
+        // check parts of the error, not the whole string so as not to tightly
+        // couple the error message with test
+        System.err.println("Got SQLException with message " + msg);
+        assertTrue("Checking permission denied error", msg.contains("user2"));
+        assertTrue("Checking permission denied error", msg.contains(tableName2));
+        assertTrue("Checking permission denied error", msg.contains("SELECT"));
+      }
+    }
+
+    {
       // try dropping table as user2 - should fail
       Connection hs2Conn = getConnection("user2");
       try {

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
index 0989e20..24322d0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/AuthorizationMetaStoreFilterHook.java
@@ -50,17 +50,10 @@ public class AuthorizationMetaStoreFilterHook extends DefaultMetaStoreFilterHook
 
   @Override
   public List<String> filterDatabases(List<String> dbList) throws MetaException {
-    List<HivePrivilegeObject> listObjs = getHivePrivObjects(dbList);
+    List<HivePrivilegeObject> listObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(dbList);
     return getDbNames(getFilteredObjects(listObjs));
   }
 
-  private List<HivePrivilegeObject> getHivePrivObjects(List<String> dbList) {
-    List<HivePrivilegeObject> objs = new ArrayList<HivePrivilegeObject>();
-    for(String dbname : dbList) {
-      objs.add(new HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbname, dbname));
-    }
-    return objs;
-  }
 
   private List<String> getDbNames(List<HivePrivilegeObject> filteredObjects) {
     List<String> tnames = new ArrayList<String>();

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index b79c080..b974b59 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -118,11 +118,28 @@ public enum HiveOperationType {
   ALTERTABLE_COMPACT,
   SHOW_COMPACTIONS,
   SHOW_TRANSACTIONS,
+  // ==== Hive command operation types starts here ==== //
   SET,
   RESET,
   DFS,
   ADD,
   DELETE,
-  COMPILE
+  COMPILE,
+  // ==== Hive command operations ends here ==== //
+
+  // ==== HiveServer2 metadata api types start here ==== //
+  // these corresponds to various java.sql.DatabaseMetaData calls.
+  GET_CATALOGS, // DatabaseMetaData.getCatalogs()  catalogs are actually not supported in
+                // hive, so this is a no-op
+
+  GET_COLUMNS, // getColumns(String catalog, String schemaPattern, String
+               // tableNamePattern, String columnNamePattern)
+
+  GET_FUNCTIONS, // getFunctions(String catalog, String schemaPattern, String functionNamePattern)
+  GET_SCHEMAS, // getSchemas()
+  GET_TABLES, // getTables(String catalog, String schemaPattern, String tableNamePattern, String[] types)
+  GET_TABLETYPES,// getTableTypes()
+  GET_TYPEINFO // getTypeInfo()
+  // ==== HiveServer2 metadata api types ends here ==== //
 
 }

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObjectUtils.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObjectUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObjectUtils.java
new file mode 100644
index 0000000..371c33e
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HivePrivilegeObjectUtils.java
@@ -0,0 +1,51 @@
+/**
+ * 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.hive.ql.security.authorization.plugin;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceStability.Evolving;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.LimitedPrivate;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
+
+/**
+ * Utility functions for working with HivePrivilegeObject
+ */
+@LimitedPrivate(value = { "Apache Argus (incubating)" })
+@Evolving
+public class HivePrivilegeObjectUtils {
+
+  /**
+   * Convert list of dbnames into list of HivePrivilegeObject
+   * @param dbList
+   * @return
+   */
+  public static List<HivePrivilegeObject> getHivePrivDbObjects(List<String> dbList) {
+    List<HivePrivilegeObject> objs = new ArrayList<HivePrivilegeObject>();
+    for (String dbname : dbList) {
+      objs.add(new HivePrivilegeObject(HivePrivilegeObjectType.DATABASE, dbname, dbname));
+    }
+    return objs;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index d43eee4..a6226b6 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -29,6 +29,8 @@ import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObje
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivObjectActionType;
 import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
 
+import com.google.common.base.Preconditions;
+
 /**
  * Mapping of operation to its required input and output privileges
  */
@@ -387,10 +389,16 @@ public class Operation2Privilege {
 (null, null));
     op2Priv.put(HiveOperationType.SHOW_ROLE_GRANT, PrivRequirement.newIOPrivRequirement
 (null, null));
-    op2Priv.put(HiveOperationType.SHOW_ROLE_PRINCIPALS, PrivRequirement.newIOPrivRequirement
-(null, null));
-
-
+    op2Priv.put(HiveOperationType.SHOW_ROLE_PRINCIPALS,
+        PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_CATALOGS, PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_SCHEMAS, PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_TABLES, PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_FUNCTIONS, PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_TABLETYPES, PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_TYPEINFO, PrivRequirement.newIOPrivRequirement(null, null));
+    op2Priv.put(HiveOperationType.GET_COLUMNS,
+        PrivRequirement.newIOPrivRequirement(SEL_NOGRANT_AR, null));
 
   }
 
@@ -427,6 +435,7 @@ public class Operation2Privilege {
   public static RequiredPrivileges getRequiredPrivs(HiveOperationType hiveOpType,
       HivePrivilegeObject hObj, IOType ioType) {
     List<PrivRequirement> opPrivs = op2Priv.get(hiveOpType);
+    Preconditions.checkNotNull(opPrivs, "Privileges for " + hiveOpType + " are null");
     RequiredPrivileges reqPrivs = new RequiredPrivileges();
 
     // Find the PrivRequirements that match on IOType, ActionType, and HivePrivilegeObjectType add

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
index dec8757..8868ec1 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hive.service.cli.operation;
 
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -45,7 +46,16 @@ public class GetCatalogsOperation extends MetadataOperation {
   @Override
   public void runInternal() throws HiveSQLException {
     setState(OperationState.RUNNING);
-    setState(OperationState.FINISHED);
+    try {
+      if (isAuthV2Enabled()) {
+        authorizeMetaGets(HiveOperationType.GET_CATALOGS, null);
+      }
+      setState(OperationState.FINISHED);
+    } catch (HiveSQLException e) {
+      setState(OperationState.ERROR);
+      throw e;
+    }
+
   }
 
   /* (non-Javadoc)

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
index 95edc6a..309f10f 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java
@@ -19,13 +19,21 @@
 package org.apache.hive.service.cli.operation;
 
 import java.sql.DatabaseMetaData;
+import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
 import java.util.regex.Pattern;
 
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType;
 import org.apache.hive.service.cli.ColumnDescriptor;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
@@ -130,9 +138,24 @@ public class GetColumnsOperation extends MetadataOperation {
 
       List<String> dbNames = metastoreClient.getDatabases(schemaPattern);
       Collections.sort(dbNames);
+      Map<String, List<String>> db2Tabs = new HashMap<>();
+
       for (String dbName : dbNames) {
         List<String> tableNames = metastoreClient.getTables(dbName, tablePattern);
         Collections.sort(tableNames);
+        db2Tabs.put(dbName, tableNames);
+      }
+
+      if (isAuthV2Enabled()) {
+        List<HivePrivilegeObject> privObjs = getPrivObjs(db2Tabs);
+        String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName
+            + ", tablePattern : " + tableName;
+        authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr);
+      }
+
+      for (Entry<String, List<String>> dbTabs : db2Tabs.entrySet()) {
+        String dbName = dbTabs.getKey();
+        List<String> tableNames = dbTabs.getValue();
         for (Table table : metastoreClient.getTableObjectsByName(dbName, tableNames)) {
           TableSchema schema = new TableSchema(metastoreClient.getSchema(dbName, table.getTableName()));
           for (ColumnDescriptor column : schema.getColumnDescriptors()) {
@@ -177,6 +200,17 @@ public class GetColumnsOperation extends MetadataOperation {
   }
 
 
+  private List<HivePrivilegeObject> getPrivObjs(Map<String, List<String>> db2Tabs) {
+    List<HivePrivilegeObject> privObjs = new ArrayList<>();
+    for (Entry<String, List<String>> dbTabs : db2Tabs.entrySet()) {
+      for (String tabName : dbTabs.getValue()) {
+        privObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbTabs.getKey(),
+            tabName));
+      }
+    }
+    return privObjs;
+  }
+
   /* (non-Javadoc)
    * @see org.apache.hive.service.cli.Operation#getResultSetSchema()
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
index c54d6e3..6df1e8a 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java
@@ -19,10 +19,16 @@
 package org.apache.hive.service.cli.operation;
 
 import java.sql.DatabaseMetaData;
+import java.util.List;
 import java.util.Set;
 
+import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.exec.FunctionInfo;
 import org.apache.hadoop.hive.ql.exec.FunctionRegistry;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils;
 import org.apache.hive.service.cli.CLIServiceUtils;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
@@ -33,6 +39,7 @@ import org.apache.hive.service.cli.RowSetFactory;
 import org.apache.hive.service.cli.TableSchema;
 import org.apache.hive.service.cli.Type;
 import org.apache.hive.service.cli.session.HiveSession;
+import org.apache.thrift.TException;
 
 /**
  * GetFunctionsOperation.
@@ -71,6 +78,24 @@ public class GetFunctionsOperation extends MetadataOperation {
   @Override
   public void runInternal() throws HiveSQLException {
     setState(OperationState.RUNNING);
+    if (isAuthV2Enabled()) {
+      // get databases for schema pattern
+      IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient();
+      String schemaPattern = convertSchemaPattern(schemaName);
+      List<String> matchingDbs;
+      try {
+        matchingDbs = metastoreClient.getDatabases(schemaPattern);
+      } catch (TException e) {
+        setState(OperationState.ERROR);
+        throw new HiveSQLException(e);
+      }
+      // authorize this call on the schema objects
+      List<HivePrivilegeObject> privObjs = HivePrivilegeObjectUtils
+          .getHivePrivDbObjects(matchingDbs);
+      String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName;
+      authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr);
+    }
+
     try {
       if ((null == catalogName || "".equals(catalogName))
           && (null == schemaName || "".equals(schemaName))) {

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
index f0e22b8..e56686a 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java
@@ -18,7 +18,16 @@
 
 package org.apache.hive.service.cli.operation;
 
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -53,6 +62,10 @@ public class GetSchemasOperation extends MetadataOperation {
   @Override
   public void runInternal() throws HiveSQLException {
     setState(OperationState.RUNNING);
+    if (isAuthV2Enabled()) {
+      String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName;
+      authorizeMetaGets(HiveOperationType.GET_SCHEMAS, null, cmdStr);
+    }
     try {
       IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient();
       String schemaPattern = convertSchemaPattern(schemaName);

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
index 486e50e..a09b39a 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java
@@ -20,6 +20,7 @@ package org.apache.hive.service.cli.operation;
 
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -53,6 +54,9 @@ public class GetTableTypesOperation extends MetadataOperation {
   @Override
   public void runInternal() throws HiveSQLException {
     setState(OperationState.RUNNING);
+    if (isAuthV2Enabled()) {
+      authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null);
+    }
     try {
       for (TableType type : TableType.values()) {
         rowSet.addRow(new String[] {tableTypeMapping.mapToClientType(type.toString())});

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
index 823dc4d..0e2fdc6 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTablesOperation.java
@@ -24,6 +24,9 @@ import java.util.List;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.IMetaStoreClient;
 import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -77,6 +80,13 @@ public class GetTablesOperation extends MetadataOperation {
     try {
       IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient();
       String schemaPattern = convertSchemaPattern(schemaName);
+      List<String> matchingDbs = metastoreClient.getDatabases(schemaPattern);
+      if(isAuthV2Enabled()){
+        List<HivePrivilegeObject> privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs);
+        String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName;
+        authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr);
+      }
+
       String tablePattern = convertIdentifierPattern(tableName, true);
       for (String dbName : metastoreClient.getDatabases(schemaPattern)) {
         List<String> tableNames = metastoreClient.getTables(dbName, tablePattern);

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
index d251c9b..2a0fec2 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java
@@ -18,6 +18,7 @@
 
 package org.apache.hive.service.cli.operation;
 
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
 import org.apache.hive.service.cli.FetchOrientation;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
@@ -82,6 +83,9 @@ public class GetTypeInfoOperation extends MetadataOperation {
   @Override
   public void runInternal() throws HiveSQLException {
     setState(OperationState.RUNNING);
+    if (isAuthV2Enabled()) {
+      authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null);
+    }
     try {
       for (Type type : Type.values()) {
         Object[] rowData = new Object[] {

http://git-wip-us.apache.org/repos/asf/hive/blob/317612b5/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
----------------------------------------------------------------------
diff --git a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
index 3a1e2a0..4595ef5 100644
--- a/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
+++ b/service/src/java/org/apache/hive/service/cli/operation/MetadataOperation.java
@@ -18,6 +18,16 @@
 
 package org.apache.hive.service.cli.operation;
 
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType;
+import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject;
+import org.apache.hadoop.hive.ql.session.SessionState;
 import org.apache.hive.service.cli.HiveSQLException;
 import org.apache.hive.service.cli.OperationState;
 import org.apache.hive.service.cli.OperationType;
@@ -97,4 +107,29 @@ public abstract class MetadataOperation extends Operation {
         .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", ".");
   }
 
+  protected boolean isAuthV2Enabled(){
+    SessionState ss = SessionState.get();
+    return (ss.isAuthorizationModeV2() &&
+        HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED));
+  }
+
+  protected void authorizeMetaGets(HiveOperationType opType, List<HivePrivilegeObject> inpObjs)
+      throws HiveSQLException {
+    authorizeMetaGets(opType, inpObjs, null);
+  }
+
+  protected void authorizeMetaGets(HiveOperationType opType, List<HivePrivilegeObject> inpObjs,
+      String cmdString) throws HiveSQLException {
+    SessionState ss = SessionState.get();
+    HiveAuthzContext.Builder ctxBuilder = new HiveAuthzContext.Builder();
+    ctxBuilder.setUserIpAddress(ss.getUserIpAddress());
+    ctxBuilder.setCommandString(cmdString);
+    try {
+      ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null,
+          ctxBuilder.build());
+    } catch (HiveAuthzPluginException | HiveAccessControlException e) {
+      throw new HiveSQLException(e.getMessage(), e);
+    }
+  }
+
 }