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 2014/10/01 00:09:39 UTC

svn commit: r1628562 - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/ itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/ itests/hive-unit/src/test/j...

Author: thejas
Date: Tue Sep 30 22:09:39 2014
New Revision: 1628562

URL: http://svn.apache.org/r1628562
Log:
HIVE-8221 : authorize additional metadata read operations in metastore storage based authorization  (Thejas Nair, reviewed by Sushanth Sowmyan)

Added:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Sep 30 22:09:39 2014
@@ -1370,6 +1370,8 @@ public class HiveConf extends Configurat
         "authorization manager class name to be used in the metastore for authorization.\n" +
         "The user defined authorization class should implement interface \n" +
         "org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider. "),
+    HIVE_METASTORE_AUTHORIZATION_AUTH_READS("hive.security.metastore.authorization.auth.reads", true,
+        "If this is true, metastore authorizer authorizes read actions on database, table"),
     HIVE_METASTORE_AUTHENTICATOR_MANAGER("hive.security.metastore.authenticator.manager",
         "org.apache.hadoop.hive.ql.security.HadoopDefaultMetastoreAuthenticator",
         "authenticator manager class name to be used in the metastore for authentication. \n" +

Modified: hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java (original)
+++ hive/trunk/hcatalog/server-extensions/src/main/java/org/apache/hive/hcatalog/listener/NotificationListener.java Tue Sep 30 22:09:39 2014
@@ -214,7 +214,7 @@ public class NotificationListener extend
       HiveConf conf = handler.getHiveConf();
       Table newTbl;
       try {
-        newTbl = handler.get_table(tbl.getDbName(), tbl.getTableName())
+        newTbl = handler.get_table_core(tbl.getDbName(), tbl.getTableName())
           .deepCopy();
         newTbl.getParameters().put(
           HCatConstants.HCAT_MSGBUS_TOPIC_NAME,

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreEventListener.java Tue Sep 30 22:09:39 2014
@@ -193,43 +193,39 @@ public class TestMetaStoreEventListener 
 
     driver.run("create database " + dbName);
     listSize++;
+    PreCreateDatabaseEvent preDbEvent = (PreCreateDatabaseEvent)(preNotifyList.get(preNotifyList.size() - 1));
     Database db = msc.getDatabase(dbName);
     assertEquals(listSize, notifyList.size());
-    assertEquals(listSize, preNotifyList.size());
+    assertEquals(listSize + 1, preNotifyList.size());
+    validateCreateDb(db, preDbEvent.getDatabase());
 
     CreateDatabaseEvent dbEvent = (CreateDatabaseEvent)(notifyList.get(listSize - 1));
     assert dbEvent.getStatus();
     validateCreateDb(db, dbEvent.getDatabase());
 
-    PreCreateDatabaseEvent preDbEvent = (PreCreateDatabaseEvent)(preNotifyList.get(listSize - 1));
-    validateCreateDb(db, preDbEvent.getDatabase());
 
     driver.run("use " + dbName);
     driver.run(String.format("create table %s (a string) partitioned by (b string)", tblName));
+    PreCreateTableEvent preTblEvent = (PreCreateTableEvent)(preNotifyList.get(preNotifyList.size() - 1));
     listSize++;
     Table tbl = msc.getTable(dbName, tblName);
+    validateCreateTable(tbl, preTblEvent.getTable());
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
 
     CreateTableEvent tblEvent = (CreateTableEvent)(notifyList.get(listSize - 1));
     assert tblEvent.getStatus();
     validateCreateTable(tbl, tblEvent.getTable());
 
-    PreCreateTableEvent preTblEvent = (PreCreateTableEvent)(preNotifyList.get(listSize - 1));
-    validateCreateTable(tbl, preTblEvent.getTable());
-
     driver.run("alter table tmptbl add partition (b='2011')");
     listSize++;
-    Partition part = msc.getPartition("hive2038", "tmptbl", "b=2011");
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    PreAddPartitionEvent prePartEvent = (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
 
     AddPartitionEvent partEvent = (AddPartitionEvent)(notifyList.get(listSize-1));
     assert partEvent.getStatus();
+    Partition part = msc.getPartition("hive2038", "tmptbl", "b=2011");
     validateAddPartition(part, partEvent.getPartitions().get(0));
     validateTableInAddPartition(tbl, partEvent.getTable());
-
-    PreAddPartitionEvent prePartEvent = (PreAddPartitionEvent)(preNotifyList.get(listSize-1));
     validateAddPartition(part, prePartEvent.getPartitions().get(0));
 
     // Test adding multiple partitions in a single partition-set, atomically.
@@ -254,7 +250,8 @@ public class TestMetaStoreEventListener 
     driver.run(String.format("alter table %s touch partition (%s)", tblName, "b='2011'"));
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    PreAlterPartitionEvent preAlterPartEvent =
+        (PreAlterPartitionEvent)preNotifyList.get(preNotifyList.size() - 1);
 
     //the partition did not change,
     // so the new partition should be similar to the original partition
@@ -266,40 +263,39 @@ public class TestMetaStoreEventListener 
         alterPartEvent.getOldPartition().getTableName(),
         alterPartEvent.getOldPartition().getValues(), alterPartEvent.getNewPartition());
 
-    PreAlterPartitionEvent preAlterPartEvent =
-        (PreAlterPartitionEvent)preNotifyList.get(listSize - 1);
+
     validateAlterPartition(origP, origP, preAlterPartEvent.getDbName(),
         preAlterPartEvent.getTableName(), preAlterPartEvent.getNewPartition().getValues(),
         preAlterPartEvent.getNewPartition());
 
     List<String> part_vals = new ArrayList<String>();
     part_vals.add("c=2012");
+    int preEventListSize;
+    preEventListSize = preNotifyList.size() + 1;
     Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
 
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    assertEquals(preNotifyList.size(), preEventListSize);
 
     AddPartitionEvent appendPartEvent =
         (AddPartitionEvent)(notifyList.get(listSize-1));
     validateAddPartition(newPart, appendPartEvent.getPartitions().get(0));
 
     PreAddPartitionEvent preAppendPartEvent =
-        (PreAddPartitionEvent)(preNotifyList.get(listSize-1));
+        (PreAddPartitionEvent)(preNotifyList.get(preNotifyList.size() - 1));
     validateAddPartition(newPart, preAppendPartEvent.getPartitions().get(0));
 
     driver.run(String.format("alter table %s rename to %s", tblName, renamed));
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    PreAlterTableEvent preAlterTableE = (PreAlterTableEvent) preNotifyList.get(preNotifyList.size() - 1);
 
     Table renamedTable = msc.getTable(dbName, renamed);
 
     AlterTableEvent alterTableE = (AlterTableEvent) notifyList.get(listSize-1);
     assert alterTableE.getStatus();
     validateAlterTable(tbl, renamedTable, alterTableE.getOldTable(), alterTableE.getNewTable());
-
-    PreAlterTableEvent preAlterTableE = (PreAlterTableEvent) preNotifyList.get(listSize-1);
     validateAlterTable(tbl, renamedTable, preAlterTableE.getOldTable(),
         preAlterTableE.getNewTable());
 
@@ -307,20 +303,17 @@ public class TestMetaStoreEventListener 
     driver.run(String.format("alter table %s rename to %s", renamed, tblName));
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
 
     driver.run(String.format("alter table %s ADD COLUMNS (c int)", tblName));
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    preAlterTableE = (PreAlterTableEvent) preNotifyList.get(preNotifyList.size() - 1);
 
     Table altTable = msc.getTable(dbName, tblName);
 
     alterTableE = (AlterTableEvent) notifyList.get(listSize-1);
     assert alterTableE.getStatus();
     validateAlterTableColumns(tbl, altTable, alterTableE.getOldTable(), alterTableE.getNewTable());
-
-    preAlterTableE = (PreAlterTableEvent) preNotifyList.get(listSize-1);
     validateAlterTableColumns(tbl, altTable, preAlterTableE.getOldTable(),
         preAlterTableE.getNewTable());
 
@@ -329,7 +322,6 @@ public class TestMetaStoreEventListener 
     msc.markPartitionForEvent("hive2038", "tmptbl", kvs, PartitionEventType.LOAD_DONE);
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
 
     LoadPartitionDoneEvent partMarkEvent = (LoadPartitionDoneEvent)notifyList.get(listSize - 1);
     assert partMarkEvent.getStatus();
@@ -337,46 +329,42 @@ public class TestMetaStoreEventListener 
         partMarkEvent.getPartitionName());
 
     PreLoadPartitionDoneEvent prePartMarkEvent =
-        (PreLoadPartitionDoneEvent)preNotifyList.get(listSize - 1);
+        (PreLoadPartitionDoneEvent)preNotifyList.get(preNotifyList.size() - 1);
     validateLoadPartitionDone("tmptbl", kvs, prePartMarkEvent.getTableName(),
         prePartMarkEvent.getPartitionName());
 
     driver.run(String.format("alter table %s drop partition (b='2011')", tblName));
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    PreDropPartitionEvent preDropPart = (PreDropPartitionEvent) preNotifyList.get(preNotifyList
+        .size() - 1);
 
     DropPartitionEvent dropPart = (DropPartitionEvent)notifyList.get(listSize - 1);
     assert dropPart.getStatus();
     validateDropPartition(part, dropPart.getPartition());
     validateTableInDropPartition(tbl, dropPart.getTable());
 
-    PreDropPartitionEvent preDropPart = (PreDropPartitionEvent)preNotifyList.get(listSize - 1);
     validateDropPartition(part, preDropPart.getPartition());
     validateTableInDropPartition(tbl, preDropPart.getTable());
 
     driver.run("drop table " + tblName);
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    PreDropTableEvent preDropTbl = (PreDropTableEvent)preNotifyList.get(preNotifyList.size() - 1);
 
     DropTableEvent dropTbl = (DropTableEvent)notifyList.get(listSize-1);
     assert dropTbl.getStatus();
     validateDropTable(tbl, dropTbl.getTable());
-
-    PreDropTableEvent preDropTbl = (PreDropTableEvent)preNotifyList.get(listSize-1);
     validateDropTable(tbl, preDropTbl.getTable());
 
     driver.run("drop database " + dbName);
     listSize++;
     assertEquals(notifyList.size(), listSize);
-    assertEquals(preNotifyList.size(), listSize);
+    PreDropDatabaseEvent preDropDB = (PreDropDatabaseEvent)preNotifyList.get(preNotifyList.size() - 1);
 
     DropDatabaseEvent dropDB = (DropDatabaseEvent)notifyList.get(listSize-1);
     assert dropDB.getStatus();
     validateDropDb(db, dropDB.getDatabase());
-
-    PreDropDatabaseEvent preDropDB = (PreDropDatabaseEvent)preNotifyList.get(listSize-1);
     validateDropDb(db, preDropDB.getDatabase());
 
     SetProcessor.setVariable("metaconf:hive.metastore.try.direct.sql", "false");

Added: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java?rev=1628562&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java (added)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/StorageBasedMetastoreTestBase.java Tue Sep 30 22:09:39 2014
@@ -0,0 +1,134 @@
+/**
+ * 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;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.cli.CliSessionState;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
+import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+
+/**
+ * Base class for some storage based authorization test classes
+ */
+public class StorageBasedMetastoreTestBase {
+  protected HiveConf clientHiveConf;
+  protected HiveMetaStoreClient msc;
+  protected Driver driver;
+  protected UserGroupInformation ugi;
+  private static int objNum = 0;
+
+  protected String getAuthorizationProvider(){
+    return StorageBasedAuthorizationProvider.class.getName();
+  }
+
+  protected HiveConf createHiveConf() throws Exception {
+    return new HiveConf(this.getClass());
+  }
+
+  @Before
+  public void setUp() throws Exception {
+
+    int port = MetaStoreUtils.findFreePort();
+
+    // Turn on metastore-side authorization
+    System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
+        AuthorizationPreEventListener.class.getName());
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER.varname,
+        getAuthorizationProvider());
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHENTICATOR_MANAGER.varname,
+        InjectableDummyAuthenticator.class.getName());
+
+    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
+
+    clientHiveConf = createHiveConf();
+
+    // Turn off client-side authorization
+    clientHiveConf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED,false);
+
+    clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
+    clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
+    clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+
+    clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
+    clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
+
+    ugi = ShimLoader.getHadoopShims().getUGIForConf(clientHiveConf);
+
+    SessionState.start(new CliSessionState(clientHiveConf));
+    msc = new HiveMetaStoreClient(clientHiveConf, null);
+    driver = new Driver(clientHiveConf);
+
+    setupFakeUser();
+    InjectableDummyAuthenticator.injectMode(false);
+  }
+
+  protected void setupFakeUser() {
+    String fakeUser = "mal";
+    List<String> fakeGroupNames = new ArrayList<String>();
+    fakeGroupNames.add("groupygroup");
+
+    InjectableDummyAuthenticator.injectUserName(fakeUser);
+    InjectableDummyAuthenticator.injectGroupNames(fakeGroupNames);
+  }
+
+  protected String setupUser() {
+    return ugi.getUserName();
+  }
+
+  protected String getTestTableName() {
+    return this.getClass().getSimpleName() + "tab" + ++objNum;
+  }
+
+  protected String getTestDbName() {
+    return this.getClass().getSimpleName() + "db" + ++objNum;
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    InjectableDummyAuthenticator.injectMode(false);
+  }
+
+  protected void setPermissions(String locn, String permissions) throws Exception {
+    FileSystem fs = FileSystem.get(new URI(locn), clientHiveConf);
+    fs.setPermission(new Path(locn), FsPermission.valueOf(permissions));
+  }
+
+  protected void validateCreateDb(Database expectedDb, String dbName) {
+    Assert.assertEquals(expectedDb.getName().toLowerCase(), dbName.toLowerCase());
+  }
+
+
+}

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java Tue Sep 30 22:09:39 2014
@@ -171,41 +171,36 @@ public class TestAuthorizationPreEventLi
 
     driver.run("create database " + dbName);
     listSize++;
-    Database db = msc.getDatabase(dbName);
-
     Database dbFromEvent = (Database)assertAndExtractSingleObjectFromEvent(listSize, authCalls,
         DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.DB);
+    Database db = msc.getDatabase(dbName);
     validateCreateDb(db,dbFromEvent);
 
     driver.run("use " + dbName);
     driver.run(String.format("create table %s (a string) partitioned by (b string)", tblName));
-    listSize++;
-    Table tbl = msc.getTable(dbName, tblName);
+    listSize = authCalls.size();
 
     Table tblFromEvent = (
         (org.apache.hadoop.hive.ql.metadata.Table)
         assertAndExtractSingleObjectFromEvent(listSize, authCalls,
             DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
             .getTTable();
+    Table tbl = msc.getTable(dbName, tblName);
     validateCreateTable(tbl, tblFromEvent);
 
     driver.run("alter table tmptbl add partition (b='2011')");
-    listSize++;
-    Partition part = msc.getPartition("hive3705", "tmptbl", "b=2011");
+    listSize = authCalls.size();
 
     Partition ptnFromEvent = (
         (org.apache.hadoop.hive.ql.metadata.Partition)
         assertAndExtractSingleObjectFromEvent(listSize, authCalls,
             DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.PARTITION))
             .getTPartition();
+    Partition part = msc.getPartition("hive3705", "tmptbl", "b=2011");
     validateAddPartition(part,ptnFromEvent);
 
     driver.run(String.format("alter table %s touch partition (%s)", tblName, "b='2011'"));
-    listSize++;
-
-    //the partition did not change,
-    // so the new partition should be similar to the original partition
-    Partition modifiedP = msc.getPartition(dbName, tblName, "b=2011");
+    listSize = authCalls.size();
 
     Partition ptnFromEventAfterAlter = (
         (org.apache.hadoop.hive.ql.metadata.Partition)
@@ -213,6 +208,9 @@ public class TestAuthorizationPreEventLi
             DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.PARTITION))
             .getTPartition();
 
+    //the partition did not change,
+    // so the new partition should be similar to the original partition
+    Partition modifiedP = msc.getPartition(dbName, tblName, "b=2011");
     validateAlterPartition(part, modifiedP, ptnFromEventAfterAlter.getDbName(),
         ptnFromEventAfterAlter.getTableName(), ptnFromEventAfterAlter.getValues(),
         ptnFromEventAfterAlter);
@@ -220,8 +218,9 @@ public class TestAuthorizationPreEventLi
 
     List<String> part_vals = new ArrayList<String>();
     part_vals.add("c=2012");
-    Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
 
+    listSize = authCalls.size();
+    Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
     listSize++;
 
     Partition newPtnFromEvent = (
@@ -233,25 +232,23 @@ public class TestAuthorizationPreEventLi
 
 
     driver.run(String.format("alter table %s rename to %s", tblName, renamed));
-    listSize++;
+    listSize = authCalls.size();
 
-    Table renamedTable = msc.getTable(dbName, renamed);
     Table renamedTableFromEvent = (
         (org.apache.hadoop.hive.ql.metadata.Table)
         assertAndExtractSingleObjectFromEvent(listSize, authCalls,
             DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
             .getTTable();
 
+    Table renamedTable = msc.getTable(dbName, renamed);
     validateAlterTable(tbl, renamedTable, renamedTableFromEvent,
         renamedTable);
     assertFalse(tbl.getTableName().equals(renamedTable.getTableName()));
 
     //change the table name back
     driver.run(String.format("alter table %s rename to %s", renamed, tblName));
-    listSize++;
-
     driver.run(String.format("alter table %s drop partition (b='2011')", tblName));
-    listSize++;
+    listSize = authCalls.size();
 
     Partition ptnFromDropPartition = (
         (org.apache.hadoop.hive.ql.metadata.Partition)
@@ -262,7 +259,7 @@ public class TestAuthorizationPreEventLi
     validateDropPartition(modifiedP, ptnFromDropPartition);
 
     driver.run("drop table " + tblName);
-    listSize++;
+    listSize = authCalls.size();
     Table tableFromDropTableEvent = (
         (org.apache.hadoop.hive.ql.metadata.Table)
         assertAndExtractSingleObjectFromEvent(listSize, authCalls,
@@ -290,16 +287,16 @@ public class TestAuthorizationPreEventLi
     }
 
     tCustom.setTableName(tbl.getTableName() + "_custom");
+    listSize = authCalls.size();
     msc.createTable(tCustom);
     listSize++;
 
-    Table customCreatedTable = msc.getTable(tCustom.getDbName(), tCustom.getTableName());
     Table customCreatedTableFromEvent = (
         (org.apache.hadoop.hive.ql.metadata.Table)
             assertAndExtractSingleObjectFromEvent(listSize, authCalls,
                 DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
         .getTTable();
-
+    Table customCreatedTable = msc.getTable(tCustom.getDbName(), tCustom.getTableName());
     validateCreateTable(tCustom,customCreatedTable);
     validateCreateTable(tCustom,customCreatedTableFromEvent);
 
@@ -316,8 +313,10 @@ public class TestAuthorizationPreEventLi
     assertEquals(tCustom.getSd().getSerdeInfo().getSerializationLib(),
         customCreatedTableFromEvent.getSd().getSerdeInfo().getSerializationLib());
 
-    msc.dropTable(tCustom.getDbName(),tCustom.getTableName());
-    listSize++;
+    listSize = authCalls.size();
+    msc.dropTable(tCustom.getDbName(), tCustom.getTableName());
+    listSize += 2;
+
     Table table2FromDropTableEvent = (
         (org.apache.hadoop.hive.ql.metadata.Table)
             assertAndExtractSingleObjectFromEvent(listSize, authCalls,
@@ -327,7 +326,7 @@ public class TestAuthorizationPreEventLi
     validateDropTable(tCustom, table2FromDropTableEvent);
 
     driver.run("drop database " + dbName);
-    listSize++;
+    listSize = authCalls.size();
     Database dbFromDropDatabaseEvent =
         (Database)assertAndExtractSingleObjectFromEvent(listSize, authCalls,
         DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.DB);

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java Tue Sep 30 22:09:39 2014
@@ -89,6 +89,7 @@ public class TestMetastoreAuthorizationP
         AuthorizationPreEventListener.class.getName());
     System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER.varname,
         getAuthorizationProvider());
+    setupMetaStoreReadAuthorization();
     System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHENTICATOR_MANAGER.varname,
         InjectableDummyAuthenticator.class.getName());
     System.setProperty(HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS.varname, "");
@@ -115,6 +116,13 @@ public class TestMetastoreAuthorizationP
     driver = new Driver(clientHiveConf);
   }
 
+  protected void setupMetaStoreReadAuthorization() {
+    // read authorization does not work with default/legacy authorization mode
+    // It is a chicken and egg problem granting select privilege to database, as the
+    // grant statement would invoke get_database which needs select privilege
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname, "false");
+  }
+
   @Override
   protected void tearDown() throws Exception {
     super.tearDown();

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestMultiAuthorizationPreEventListener.java Tue Sep 30 22:09:39 2014
@@ -88,6 +88,7 @@ public class TestMultiAuthorizationPreEv
 
     // verify that the actual action also went through
     Database db = msc.getDatabase(dbName);
+    listSize += 2; // 1 read database auth calls for each authorization provider
     Database dbFromEvent = (Database)assertAndExtractSingleObjectFromEvent(listSize, authCalls,
         DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.DB);
     validateCreateDb(db,dbFromEvent);

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationDrops.java Tue Sep 30 22:09:39 2014
@@ -18,88 +18,19 @@
 
 package org.apache.hadoop.hive.ql.security;
 
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-
-import junit.framework.TestCase;
-
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hive.cli.CliSessionState;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
-import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
-import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
-import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
-import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
-import org.apache.hadoop.hive.ql.session.SessionState;
-import org.apache.hadoop.hive.shims.ShimLoader;
-import org.apache.hadoop.security.UserGroupInformation;
+import org.junit.Assert;
+import org.junit.Test;
 
 /**
  * Test cases focusing on drop table permission checks
  */
-public class TestStorageBasedMetastoreAuthorizationDrops extends TestCase{
-  protected HiveConf clientHiveConf;
-  protected HiveMetaStoreClient msc;
-  protected Driver driver;
-  protected UserGroupInformation ugi;
-  private static int objNum = 0;
-
-  protected String getAuthorizationProvider(){
-    return StorageBasedAuthorizationProvider.class.getName();
-  }
-
-  protected HiveConf createHiveConf() throws Exception {
-    return new HiveConf(this.getClass());
-  }
-
-  @Override
-  protected void setUp() throws Exception {
-
-    super.setUp();
-
-    int port = MetaStoreUtils.findFreePort();
-
-    // Turn on metastore-side authorization
-    System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
-        AuthorizationPreEventListener.class.getName());
-    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER.varname,
-        getAuthorizationProvider());
-    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHENTICATOR_MANAGER.varname,
-        InjectableDummyAuthenticator.class.getName());
-
-    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
-
-    clientHiveConf = createHiveConf();
-
-    // Turn off client-side authorization
-    clientHiveConf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED,false);
-
-    clientHiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
-    clientHiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    clientHiveConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-
-    clientHiveConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
-    clientHiveConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-
-    ugi = ShimLoader.getHadoopShims().getUGIForConf(clientHiveConf);
-
-    SessionState.start(new CliSessionState(clientHiveConf));
-    msc = new HiveMetaStoreClient(clientHiveConf, null);
-    driver = new Driver(clientHiveConf);
-
-    setupFakeUser();
-    InjectableDummyAuthenticator.injectMode(false);
-  }
-
+public class TestStorageBasedMetastoreAuthorizationDrops extends StorageBasedMetastoreTestBase {
 
+  @Test
   public void testDropDatabase() throws Exception {
     dropDatabaseByOtherUser("-rwxrwxrwx", 0);
     dropDatabaseByOtherUser("-rwxrwxrwt", 1);
@@ -111,12 +42,12 @@ public class TestStorageBasedMetastoreAu
    * @param expectedRet - expected return code for drop by other user
    * @throws Exception
    */
-  private void dropDatabaseByOtherUser(String perm, int expectedRet) throws Exception {
+  public void dropDatabaseByOtherUser(String perm, int expectedRet) throws Exception {
     String dbName = getTestDbName();
     setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), perm);
 
     CommandProcessorResponse resp = driver.run("create database " + dbName);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
     Database db = msc.getDatabase(dbName);
     validateCreateDb(db, dbName);
 
@@ -124,10 +55,11 @@ public class TestStorageBasedMetastoreAu
 
 
     resp = driver.run("drop database " + dbName);
-    assertEquals(expectedRet, resp.getResponseCode());
+    Assert.assertEquals(expectedRet, resp.getResponseCode());
 
   }
 
+  @Test
   public void testDropTable() throws Exception {
     dropTableByOtherUser("-rwxrwxrwx", 0);
     dropTableByOtherUser("-rwxrwxrwt", 1);
@@ -138,13 +70,13 @@ public class TestStorageBasedMetastoreAu
    * @param expectedRet expected return code on drop table
    * @throws Exception
    */
-  private void dropTableByOtherUser(String perm, int expectedRet) throws Exception {
+  public void dropTableByOtherUser(String perm, int expectedRet) throws Exception {
     String dbName = getTestDbName();
     String tblName = getTestTableName();
     setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
 
     CommandProcessorResponse resp = driver.run("create database " + dbName);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
     Database db = msc.getDatabase(dbName);
     validateCreateDb(db, dbName);
 
@@ -152,18 +84,19 @@ public class TestStorageBasedMetastoreAu
 
     String dbDotTable = dbName + "." + tblName;
     resp = driver.run("create table " + dbDotTable + "(i int)");
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
 
 
     InjectableDummyAuthenticator.injectMode(true);
     resp = driver.run("drop table " + dbDotTable);
-    assertEquals(expectedRet, resp.getResponseCode());
+    Assert.assertEquals(expectedRet, resp.getResponseCode());
   }
 
   /**
    * Drop view should not be blocked by SBA. View will not have any location to drop.
    * @throws Exception
    */
+  @Test
   public void testDropView() throws Exception {
     String dbName = getTestDbName();
     String tblName = getTestTableName();
@@ -171,7 +104,7 @@ public class TestStorageBasedMetastoreAu
     setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
 
     CommandProcessorResponse resp = driver.run("create database " + dbName);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
     Database db = msc.getDatabase(dbName);
     validateCreateDb(db, dbName);
 
@@ -179,20 +112,20 @@ public class TestStorageBasedMetastoreAu
 
     String dbDotTable = dbName + "." + tblName;
     resp = driver.run("create table " + dbDotTable + "(i int)");
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
 
     String dbDotView = dbName + "." + viewName;
     resp = driver.run("create view " + dbDotView + " as select * from " +  dbDotTable);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
 
     resp = driver.run("drop view " + dbDotView);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
 
     resp = driver.run("drop table " + dbDotTable);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
   }
 
-
+  @Test
   public void testDropPartition() throws Exception {
     dropPartitionByOtherUser("-rwxrwxrwx", 0);
     dropPartitionByOtherUser("-rwxrwxrwt", 1);
@@ -203,70 +136,29 @@ public class TestStorageBasedMetastoreAu
    * @param expectedRet expected return code
    * @throws Exception
    */
-  private void dropPartitionByOtherUser(String perm, int expectedRet) throws Exception {
+  public void dropPartitionByOtherUser(String perm, int expectedRet) throws Exception {
     String dbName = getTestDbName();
     String tblName = getTestTableName();
     setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
 
     CommandProcessorResponse resp = driver.run("create database " + dbName);
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
     Database db = msc.getDatabase(dbName);
     validateCreateDb(db, dbName);
     setPermissions(db.getLocationUri(), "-rwxrwxrwx");
 
     String dbDotTable = dbName + "." + tblName;
     resp = driver.run("create table " + dbDotTable + "(i int) partitioned by (b string)");
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
     Table tab = msc.getTable(dbName, tblName);
     setPermissions(tab.getSd().getLocation(), perm);
 
     resp = driver.run("alter table " + dbDotTable + " add partition (b='2011')");
-    assertEquals(0, resp.getResponseCode());
+    Assert.assertEquals(0, resp.getResponseCode());
 
     InjectableDummyAuthenticator.injectMode(true);
     resp = driver.run("alter table " + dbDotTable + " drop partition (b='2011')");
-    assertEquals(expectedRet, resp.getResponseCode());
+    Assert.assertEquals(expectedRet, resp.getResponseCode());
   }
 
-  private void setupFakeUser() {
-    String fakeUser = "mal";
-    List<String> fakeGroupNames = new ArrayList<String>();
-    fakeGroupNames.add("groupygroup");
-
-    InjectableDummyAuthenticator.injectUserName(fakeUser);
-    InjectableDummyAuthenticator.injectGroupNames(fakeGroupNames);
-  }
-
-  private String setupUser() {
-    return ugi.getUserName();
-  }
-
-  private String getTestTableName() {
-    return this.getClass().getSimpleName() + "tab" + ++objNum;
-  }
-
-  private String getTestDbName() {
-    return this.getClass().getSimpleName() + "db" + ++objNum;
-  }
-
-  @Override
-  protected void tearDown() throws Exception {
-    super.tearDown();
-    InjectableDummyAuthenticator.injectMode(false);
-  }
-
-  protected void setPermissions(String locn, String permissions) throws Exception {
-    FileSystem fs = FileSystem.get(new URI(locn), clientHiveConf);
-    fs.setPermission(new Path(locn), FsPermission.valueOf(permissions));
-  }
-
-  private void validateCreateDb(Database expectedDb, String dbName) {
-    assertEquals(expectedDb.getName().toLowerCase(), dbName.toLowerCase());
-  }
-
-  private void validateCreateTable(Table expectedTable, String tblName, String dbName) {
-    assertNotNull(expectedTable);
-    assertEquals(expectedTable.getTableName().toLowerCase(),tblName.toLowerCase());
-    assertEquals(expectedTable.getDbName().toLowerCase(),dbName.toLowerCase());
-  }
 }

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java Tue Sep 30 22:09:39 2014
@@ -23,6 +23,7 @@ import java.net.URI;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
 
@@ -102,4 +103,10 @@ public class TestStorageBasedMetastoreAu
     return super.getTestTableName() + "_SBAP";
   }
 
+  @Override
+  protected void setupMetaStoreReadAuthorization() {
+    // enable read authorization in metastore
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname, "true");
+  }
+
 }

Added: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java?rev=1628562&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java (added)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationReads.java Tue Sep 30 22:09:39 2014
@@ -0,0 +1,122 @@
+/**
+ * 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;
+
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.CommandNeedRetryException;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.junit.Assert;
+import org.junit.Test;
+
+/**
+ * Test cases focusing on drop table permission checks
+ */
+public class TestStorageBasedMetastoreAuthorizationReads extends StorageBasedMetastoreTestBase {
+
+  @Test
+  public void testReadTableSuccess() throws Exception {
+    readTableByOtherUser("-rwxrwxrwx", true);
+  }
+
+  @Test
+  public void testReadTableFailure() throws Exception {
+    readTableByOtherUser("-rwxrwx---", false);
+  }
+
+  /**
+   * @param perm dir permission for table dir
+   * @param isSuccess if command was successful
+   * @throws Exception
+   */
+  private void readTableByOtherUser(String perm, boolean isSuccess) throws Exception {
+    String dbName = getTestDbName();
+    String tblName = getTestTableName();
+    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), "-rwxrwxrwx");
+
+    CommandProcessorResponse resp = driver.run("create database " + dbName);
+    Assert.assertEquals(0, resp.getResponseCode());
+    Database db = msc.getDatabase(dbName);
+    validateCreateDb(db, dbName);
+
+    setPermissions(db.getLocationUri(), "-rwxrwxrwx");
+
+    String dbDotTable = dbName + "." + tblName;
+    resp = driver.run("create table " + dbDotTable + "(i int) partitioned by (date string)");
+    Assert.assertEquals(0, resp.getResponseCode());
+    Table tab = msc.getTable(dbName, tblName);
+    setPermissions(tab.getSd().getLocation(), perm);
+
+    InjectableDummyAuthenticator.injectMode(true);
+
+    testCmd(driver, "DESCRIBE  " + dbDotTable, isSuccess);
+    testCmd(driver, "DESCRIBE EXTENDED  " + dbDotTable, isSuccess);
+    testCmd(driver, "SHOW PARTITIONS  " + dbDotTable, isSuccess);
+    testCmd(driver, "SHOW COLUMNS IN " + tblName + " IN " + dbName, isSuccess);
+    testCmd(driver, "use " + dbName, true);
+    testCmd(driver, "SHOW TABLE EXTENDED LIKE " + tblName, isSuccess);
+
+  }
+
+  @Test
+  public void testReadDbSuccess() throws Exception {
+    readDbByOtherUser("-rwxrwxrwx", true);
+  }
+
+  @Test
+  public void testReadDbFailure() throws Exception {
+    readDbByOtherUser("-rwxrwx---", false);
+  }
+
+
+  /**
+   * @param perm dir permission for database dir
+   * @param isSuccess if command was successful
+   * @throws Exception
+   */
+  private void readDbByOtherUser(String perm, boolean isSuccess) throws Exception {
+    String dbName = getTestDbName();
+    setPermissions(clientHiveConf.getVar(ConfVars.METASTOREWAREHOUSE), perm);
+
+    CommandProcessorResponse resp = driver.run("create database " + dbName);
+    Assert.assertEquals(0, resp.getResponseCode());
+    Database db = msc.getDatabase(dbName);
+    validateCreateDb(db, dbName);
+    setPermissions(db.getLocationUri(), perm);
+
+    InjectableDummyAuthenticator.injectMode(true);
+
+    testCmd(driver, "DESCRIBE DATABASE " + dbName, isSuccess);
+    testCmd(driver, "DESCRIBE DATABASE EXTENDED " + dbName, isSuccess);
+    testCmd(driver, "SHOW TABLES IN " + dbName, isSuccess);
+    driver.run("use " + dbName);
+    testCmd(driver, "SHOW TABLES ", isSuccess);
+
+  }
+
+  private void testCmd(Driver driver, String cmd, boolean isSuccess)
+      throws CommandNeedRetryException {
+    CommandProcessorResponse resp = driver.run(cmd);
+    Assert.assertEquals(isSuccess, resp.getResponseCode() == 0);
+  }
+
+
+}

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Tue Sep 30 22:09:39 2014
@@ -48,9 +48,6 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
 import java.util.regex.Pattern;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableListMultimap;
-import com.google.common.collect.Multimaps;
 import org.apache.commons.cli.OptionBuilder;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -171,6 +168,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
 import org.apache.hadoop.hive.metastore.events.PreLoadPartitionDoneEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
 import org.apache.hadoop.hive.metastore.model.MDBPrivilege;
 import org.apache.hadoop.hive.metastore.model.MGlobalPrivilege;
 import org.apache.hadoop.hive.metastore.model.MPartitionColumnPrivilege;
@@ -203,7 +202,10 @@ import org.apache.thrift.transport.TTran
 import com.facebook.fb303.FacebookBase;
 import com.facebook.fb303.fb_status;
 import com.google.common.base.Splitter;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableListMultimap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Multimaps;
 
 /**
  * TODO:pc remove application logic to a separate interface.
@@ -803,7 +805,7 @@ public class HiveMetaStore extends Thrif
       Exception ex = null;
       try {
         try {
-          if (null != get_database(db.getName())) {
+          if (null != get_database_core(db.getName())) {
             throw new AlreadyExistsException("Database " + db.getName() + " already exists");
           }
         } catch (NoSuchObjectException e) {
@@ -829,25 +831,45 @@ public class HiveMetaStore extends Thrif
     }
 
     @Override
-    public Database get_database(final String name) throws NoSuchObjectException,
-        MetaException {
+    public Database get_database(final String name) throws NoSuchObjectException, MetaException {
       startFunction("get_database", ": " + name);
       Database db = null;
       Exception ex = null;
       try {
-        db = getMS().getDatabase(name);
+        db = get_database_core(name);
+        firePreEvent(new PreReadDatabaseEvent(db, this));
       } catch (MetaException e) {
         ex = e;
         throw e;
       } catch (NoSuchObjectException e) {
         ex = e;
         throw e;
+      } finally {
+        endFunction("get_database", db != null, ex);
+      }
+      return db;
+    }
+
+    /**
+     * Equivalent to get_database, but does not write to audit logs, or fire pre-event listners.
+     * Meant to be used for internal hive classes that don't use the thrift interface.
+     * @param name
+     * @return
+     * @throws NoSuchObjectException
+     * @throws MetaException
+     */
+    public Database get_database_core(final String name) throws NoSuchObjectException,
+        MetaException {
+      Database db = null;
+      try {
+        db = getMS().getDatabase(name);
+      } catch (MetaException e) {
+        throw e;
+      } catch (NoSuchObjectException e) {
+        throw e;
       } catch (Exception e) {
-        ex = e;
         assert (e instanceof RuntimeException);
         throw (RuntimeException) e;
-      } finally {
-        endFunction("get_database", db != null, ex);
       }
       return db;
     }
@@ -1373,7 +1395,7 @@ public class HiveMetaStore extends Thrif
       try {
         ms.openTransaction();
         // drop any partitions
-        tbl = get_table(dbname, name);
+        tbl = get_table_core(dbname, name);
         if (tbl == null) {
           throw new NoSuchObjectException(name + " doesn't exist");
         }
@@ -1625,13 +1647,40 @@ public class HiveMetaStore extends Thrif
       startTableFunction("get_table", dbname, name);
       Exception ex = null;
       try {
+        t = get_table_core(dbname, name);
+        firePreEvent(new PreReadTableEvent(t, this));
+      } catch (MetaException e) {
+        ex = e;
+        throw e;
+      } catch (NoSuchObjectException e) {
+        ex = e;
+        throw e;
+      } finally {
+        endFunction("get_table", t != null, ex, name);
+      }
+      return t;
+    }
+
+    /**
+     * Equivalent of get_table, but does not log audits and fire pre-event listener.
+     * Meant to be used for calls made by other hive classes, that are not using the
+     * thrift interface.
+     * @param dbname
+     * @param name
+     * @return Table object
+     * @throws MetaException
+     * @throws NoSuchObjectException
+     */
+    public Table get_table_core(final String dbname, final String name) throws MetaException,
+        NoSuchObjectException {
+      Table t;
+      try {
         t = getMS().getTable(dbname, name);
         if (t == null) {
           throw new NoSuchObjectException(dbname + "." + name
               + " table not found");
         }
       } catch (Exception e) {
-        ex = e;
         if (e instanceof MetaException) {
           throw (MetaException) e;
         } else if (e instanceof NoSuchObjectException) {
@@ -1639,8 +1688,6 @@ public class HiveMetaStore extends Thrif
         } else {
           throw newMetaException(e);
         }
-      } finally {
-        endFunction("get_table", t != null, ex, name);
       }
       return t;
     }
@@ -2418,7 +2465,7 @@ public class HiveMetaStore extends Thrif
       try {
         ms.openTransaction();
         part = ms.getPartition(db_name, tbl_name, part_vals);
-        tbl = get_table(db_name, tbl_name);
+        tbl = get_table_core(db_name, tbl_name);
         firePreEvent(new PreDropPartitionEvent(tbl, part, deleteData, this));
 
         if (part == null) {
@@ -2512,7 +2559,7 @@ public class HiveMetaStore extends Thrif
       try {
         // We need Partition-s for firing events and for result; DN needs MPartition-s to drop.
         // Great... Maybe we could bypass fetching MPartitions by issuing direct SQL deletes.
-        tbl = get_table(dbName, tblName);
+        tbl = get_table_core(dbName, tblName);
         int minCount = 0;
         RequestPartsSpec spec = request.getParts();
         List<String> partNames = null;
@@ -2671,6 +2718,7 @@ public class HiveMetaStore extends Thrif
       Partition ret = null;
       Exception ex = null;
       try {
+        fireReadTablePreEvent(db_name, tbl_name);
         ret = getMS().getPartition(db_name, tbl_name, part_vals);
       } catch (Exception e) {
         ex = e;
@@ -2687,6 +2735,28 @@ public class HiveMetaStore extends Thrif
       return ret;
     }
 
+    /**
+     * Fire a pre-event for read table operation, if there are any
+     * pre-event listeners registered
+     *
+     * @param db_name
+     * @param tbl_name
+     * @throws MetaException
+     * @throws NoSuchObjectException
+     */
+    private void fireReadTablePreEvent(String dbName, String tblName) throws MetaException, NoSuchObjectException {
+      if(preListeners.size() > 0) {
+        // do this only if there is a pre event listener registered (avoid unnecessary
+        // metastore api call)
+        Table t = getMS().getTable(dbName, tblName);
+        if (t == null) {
+          throw new NoSuchObjectException(dbName + "." + tblName
+              + " table not found");
+        }
+        firePreEvent(new PreReadTableEvent(t, this));
+      }
+    }
+
     @Override
     public Partition get_partition_with_auth(final String db_name,
         final String tbl_name, final List<String> part_vals,
@@ -2694,7 +2764,7 @@ public class HiveMetaStore extends Thrif
         throws MetaException, NoSuchObjectException, TException {
       startPartitionFunction("get_partition_with_auth", db_name, tbl_name,
           part_vals);
-
+      fireReadTablePreEvent(db_name, tbl_name);
       Partition ret = null;
       Exception ex = null;
       try {
@@ -2716,7 +2786,7 @@ public class HiveMetaStore extends Thrif
     public List<Partition> get_partitions(final String db_name, final String tbl_name,
         final short max_parts) throws NoSuchObjectException, MetaException {
       startTableFunction("get_partitions", db_name, tbl_name);
-
+      fireReadTablePreEvent(db_name, tbl_name);
       List<Partition> ret = null;
       Exception ex = null;
       try {
@@ -2773,7 +2843,7 @@ public class HiveMetaStore extends Thrif
 
       List<PartitionSpec> partitionSpecs = null;
       try {
-        Table table = get_table(dbName, tableName);
+        Table table = get_table_core(dbName, tableName);
         List<Partition> partitions = get_partitions(dbName, tableName, (short) max_parts);
 
         if (is_partition_spec_grouping_enabled(table)) {
@@ -2797,7 +2867,7 @@ public class HiveMetaStore extends Thrif
 
     private static class StorageDescriptorKey {
 
-      private StorageDescriptor sd;
+      private final StorageDescriptor sd;
 
       StorageDescriptorKey(StorageDescriptor sd) { this.sd = sd; }
 
@@ -2919,9 +2989,9 @@ public class HiveMetaStore extends Thrif
 
     @Override
     public List<String> get_partition_names(final String db_name, final String tbl_name,
-        final short max_parts) throws MetaException {
+        final short max_parts) throws MetaException, NoSuchObjectException {
       startTableFunction("get_partition_names", db_name, tbl_name);
-
+      fireReadTablePreEvent(db_name, tbl_name);
       List<String> ret = null;
       Exception ex = null;
       try {
@@ -3038,14 +3108,7 @@ public class HiveMetaStore extends Thrif
       Exception ex = null;
       try {
         for (Partition tmpPart : new_parts) {
-          try {
-            for (MetaStorePreEventListener listener : preListeners) {
-              listener.onEvent(
-                  new PreAlterPartitionEvent(db_name, tbl_name, null, tmpPart, this));
-            }
-          } catch (NoSuchObjectException e) {
-            throw new MetaException(e.getMessage());
-          }
+          firePreEvent(new PreAlterPartitionEvent(db_name, tbl_name, null, tmpPart, this));
         }
         oldParts = alterHandler.alterPartitions(getMS(), wh, db_name, tbl_name, new_parts);
 
@@ -3150,7 +3213,7 @@ public class HiveMetaStore extends Thrif
       boolean success = false;
       Exception ex = null;
       try {
-        Table oldt = get_table(dbname, name);
+        Table oldt = get_table_core(dbname, name);
         firePreEvent(new PreAlterTableEvent(oldt, newTable, this));
         alterHandler.alterTable(getMS(), wh, dbname, name, newTable);
         success = true;
@@ -3234,7 +3297,7 @@ public class HiveMetaStore extends Thrif
       Exception ex = null;
       try {
         try {
-          tbl = get_table(db, base_table_name);
+          tbl = get_table_core(db, base_table_name);
         } catch (NoSuchObjectException e) {
           throw new UnknownTableException(e.getMessage());
         }
@@ -3294,7 +3357,7 @@ public class HiveMetaStore extends Thrif
 
         Table tbl;
         try {
-          tbl = get_table(db, base_table_name);
+          tbl = get_table_core(db, base_table_name);
         } catch (NoSuchObjectException e) {
           throw new UnknownTableException(e.getMessage());
         }
@@ -3413,6 +3476,7 @@ public class HiveMetaStore extends Thrif
     private Partition get_partition_by_name_core(final RawStore ms, final String db_name,
         final String tbl_name, final String part_name)
         throws MetaException, NoSuchObjectException, TException {
+      fireReadTablePreEvent(db_name, tbl_name);
       List<String> partVals = null;
       try {
         partVals = getPartValsFromName(ms, db_name, tbl_name, part_name);
@@ -3434,7 +3498,6 @@ public class HiveMetaStore extends Thrif
 
       startFunction("get_partition_by_name", ": db=" + db_name + " tbl="
           + tbl_name + " part=" + part_name);
-
       Partition ret = null;
       Exception ex = null;
       try {
@@ -3564,6 +3627,7 @@ public class HiveMetaStore extends Thrif
         final List<String> groupNames) throws MetaException, TException, NoSuchObjectException {
       startPartitionFunction("get_partitions_ps_with_auth", db_name, tbl_name,
           part_vals);
+      fireReadTablePreEvent(db_name, tbl_name);
       List<Partition> ret = null;
       Exception ex = null;
       try {
@@ -3586,6 +3650,7 @@ public class HiveMetaStore extends Thrif
         final String tbl_name, final List<String> part_vals, final short max_parts)
         throws MetaException, TException, NoSuchObjectException {
       startPartitionFunction("get_partitions_names_ps", db_name, tbl_name, part_vals);
+      fireReadTablePreEvent(db_name, tbl_name);
       List<String> ret = null;
       Exception ex = null;
       try {
@@ -3754,7 +3819,7 @@ public class HiveMetaStore extends Thrif
         String idxTblName = index.getIndexTableName();
         if (idxTblName != null) {
           String[] qualified = MetaStoreUtils.getQualifiedName(index.getDbName(), idxTblName);
-          Table tbl = get_table(qualified[0], qualified[1]);
+          Table tbl = get_table_core(qualified[0], qualified[1]);
           if (tbl.getSd() == null) {
             throw new MetaException("Table metadata is corrupted");
           }
@@ -4056,7 +4121,7 @@ public class HiveMetaStore extends Thrif
       } finally {
         endFunction("write_partition_column_statistics: ", ret != false, null, tableName);
       }
-    } 
+    }
 
     @Override
     public boolean delete_partition_column_statistics(String dbName, String tableName,
@@ -4111,7 +4176,7 @@ public class HiveMetaStore extends Thrif
         final String tblName, final String filter, final short maxParts)
         throws MetaException, NoSuchObjectException, TException {
       startTableFunction("get_partitions_by_filter", dbName, tblName);
-
+      fireReadTablePreEvent(dbName, tblName);
       List<Partition> ret = null;
       Exception ex = null;
       try {
@@ -4134,7 +4199,7 @@ public class HiveMetaStore extends Thrif
 
       List<PartitionSpec> partitionSpecs = null;
       try {
-        Table table = get_table(dbName, tblName);
+        Table table = get_table_core(dbName, tblName);
         List<Partition> partitions = get_partitions_by_filter(dbName, tblName, filter, (short) maxParts);
 
         if (is_partition_spec_grouping_enabled(table)) {
@@ -4161,6 +4226,7 @@ public class HiveMetaStore extends Thrif
         PartitionsByExprRequest req) throws TException {
       String dbName = req.getDbName(), tblName = req.getTblName();
       startTableFunction("get_partitions_by_expr", dbName, tblName);
+      fireReadTablePreEvent(dbName, tblName);
       PartitionsByExprResult ret = null;
       Exception ex = null;
       try {
@@ -4197,7 +4263,7 @@ public class HiveMetaStore extends Thrif
         throws MetaException, NoSuchObjectException, TException {
 
       startTableFunction("get_partitions_by_names", dbName, tblName);
-
+      fireReadTablePreEvent(dbName, tblName);
       List<Partition> ret = null;
       Exception ex = null;
       try {
@@ -4242,7 +4308,7 @@ public class HiveMetaStore extends Thrif
       List<String> partValue = hiveObject.getPartValues();
       if (partValue != null && partValue.size() > 0) {
         try {
-          Table table = get_table(hiveObject.getDbName(), hiveObject
+          Table table = get_table_core(hiveObject.getDbName(), hiveObject
               .getObjectName());
           partName = Warehouse
               .makePartName(table.getPartitionKeys(), partValue);
@@ -4686,7 +4752,7 @@ public class HiveMetaStore extends Thrif
         if (dbName == null) {
           return getMS().listPrincipalPartitionColumnGrantsAll(principalName, principalType);
         }
-        Table tbl = get_table(dbName, tableName);
+        Table tbl = get_table_core(dbName, tableName);
         String partName = Warehouse.makePartName(tbl.getPartitionKeys(), partValues);
         if (principalName == null) {
           return getMS().listPartitionColumnGrantsAll(dbName, tableName, partName, columnName);
@@ -4764,7 +4830,7 @@ public class HiveMetaStore extends Thrif
         if (dbName == null) {
           return getMS().listPrincipalPartitionGrantsAll(principalName, principalType);
         }
-        Table tbl = get_table(dbName, tableName);
+        Table tbl = get_table_core(dbName, tableName);
         String partName = Warehouse.makePartName(tbl.getPartitionKeys(), partValues);
         if (principalName == null) {
           return getMS().listPartitionGrantsAll(dbName, tableName, partName);
@@ -5422,7 +5488,7 @@ public class HiveMetaStore extends Thrif
     }
   }
 
-  
+
   public static IHMSHandler newHMSHandler(String name, HiveConf hiveConf) throws MetaException {
     return newHMSHandler(name, hiveConf, false);
   }

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java Tue Sep 30 22:09:39 2014
@@ -28,7 +28,6 @@ import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.net.InetAddress;
 import java.net.URI;
-import java.net.URISyntaxException;
 import java.net.UnknownHostException;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
@@ -98,7 +97,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.PartitionEventType;
-import org.apache.hadoop.hive.metastore.api.PartitionSpec;
 import org.apache.hadoop.hive.metastore.api.PartitionsByExprRequest;
 import org.apache.hadoop.hive.metastore.api.PartitionsByExprResult;
 import org.apache.hadoop.hive.metastore.api.PartitionsStatsRequest;
@@ -122,7 +120,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
 import org.apache.hadoop.hive.metastore.api.UnlockRequest;
-import org.apache.hadoop.hive.metastore.partition.spec.CompositePartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
 import org.apache.hadoop.hive.metastore.txn.TxnHandler;
 import org.apache.hadoop.hive.shims.HadoopShims;
@@ -779,6 +776,7 @@ public class HiveMetaStoreClient impleme
    *                data from warehouse
    * @see #dropTable(String, String, boolean, boolean, EnvironmentContext)
    */
+  @Override
   public void dropTable(String dbname, String name, boolean deleteData,
       boolean ignoreUnknownTab, boolean ifPurge)
       throws MetaException, TException, NoSuchObjectException, UnsupportedOperationException {
@@ -1313,6 +1311,7 @@ public class HiveMetaStoreClient impleme
   }
 
   /** {@inheritDoc} */
+  @Override
   public boolean setPartitionColumnStatistics(SetPartitionsStatsRequest request)
     throws NoSuchObjectException, InvalidObjectException, MetaException, TException,
     InvalidInputException{

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreEventContext.java Tue Sep 30 22:09:39 2014
@@ -38,6 +38,8 @@ public abstract class PreEventContext {
     DROP_DATABASE,
     LOAD_PARTITION_DONE,
     AUTHORIZATION_API_CALL,
+    READ_TABLE,
+    READ_DATABASE
   }
 
   private final PreEventType eventType;

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java?rev=1628562&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadDatabaseEvent.java Tue Sep 30 22:09:39 2014
@@ -0,0 +1,42 @@
+/**
+ * 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.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+
+/**
+ * Database read event
+ */
+public class PreReadDatabaseEvent extends PreEventContext {
+  private final Database db;
+
+  public PreReadDatabaseEvent(Database db, HMSHandler handler) {
+    super(PreEventType.READ_DATABASE, handler);
+    this.db = db;
+  }
+
+  /**
+   * @return the db
+   */
+  public Database getDatabase() {
+    return db;
+  }
+
+}

Added: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java?rev=1628562&view=auto
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java (added)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/events/PreReadTableEvent.java Tue Sep 30 22:09:39 2014
@@ -0,0 +1,43 @@
+/**
+ * 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.metastore.events;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.Table;
+
+/**
+ * Table read event
+ */
+public class PreReadTableEvent extends PreEventContext {
+
+  private final Table table;
+
+  public PreReadTableEvent(Table table, HMSHandler handler) {
+    super(PreEventType.READ_TABLE, handler);
+    this.table = table;
+  }
+
+  /**
+   * @return the table
+   */
+  public Table getTable() {
+    return table;
+  }
+
+}

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java Tue Sep 30 22:09:39 2014
@@ -23,7 +23,9 @@ import java.util.List;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.common.classification.InterfaceAudience.Private;
 import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.metastore.TableType;
@@ -40,6 +42,8 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.events.PreDropPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.PreDropTableEvent;
 import org.apache.hadoop.hive.metastore.events.PreEventContext;
+import org.apache.hadoop.hive.metastore.events.PreReadDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreReadTableEvent;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
@@ -54,6 +58,7 @@ import org.apache.hadoop.hive.ql.securit
  * metastore PreEventContexts, such as the adding/dropping and altering
  * of databases, tables and partitions.
  */
+@Private
 public class AuthorizationPreEventListener extends MetaStorePreEventListener {
 
   public static final Log LOG = LogFactory.getLog(
@@ -136,6 +141,12 @@ public class AuthorizationPreEventListen
     case ALTER_TABLE:
       authorizeAlterTable((PreAlterTableEvent)context);
       break;
+    case READ_TABLE:
+      authorizeReadTable((PreReadTableEvent)context);
+      break;
+    case READ_DATABASE:
+      authorizeReadDatabase((PreReadDatabaseEvent)context);
+      break;
     case ADD_PARTITION:
       authorizeAddPartition((PreAddPartitionEvent)context);
       break;
@@ -162,6 +173,44 @@ public class AuthorizationPreEventListen
 
   }
 
+  private void authorizeReadTable(PreReadTableEvent context) throws InvalidOperationException,
+      MetaException {
+    if (!isReadAuthzEnabled()) {
+      return;
+    }
+    try {
+      org.apache.hadoop.hive.ql.metadata.Table wrappedTable = new TableWrapper(context.getTable());
+      for (HiveMetastoreAuthorizationProvider authorizer : tAuthorizers.get()) {
+        authorizer.authorize(wrappedTable, new Privilege[] { Privilege.SELECT }, null);
+      }
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeReadDatabase(PreReadDatabaseEvent context)
+      throws InvalidOperationException, MetaException {
+    if (!isReadAuthzEnabled()) {
+      return;
+    }
+    try {
+      for (HiveMetastoreAuthorizationProvider authorizer : tAuthorizers.get()) {
+        authorizer.authorize(new Database(context.getDatabase()),
+            new Privilege[] { Privilege.SELECT }, null);
+      }
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private boolean isReadAuthzEnabled() {
+    return tConfig.get().getBoolean(ConfVars.HIVE_METASTORE_AUTHORIZATION_AUTH_READS.varname, true);
+  }
+
   private void authorizeAuthorizationAPICall() throws InvalidOperationException, MetaException {
     for (HiveMetastoreAuthorizationProvider authorizer : tAuthorizers.get()) {
       try {
@@ -358,7 +407,7 @@ public class AuthorizationPreEventListen
     public PartitionWrapper(org.apache.hadoop.hive.metastore.api.Partition mapiPart,
         PreEventContext context) throws HiveException, NoSuchObjectException, MetaException {
       org.apache.hadoop.hive.metastore.api.Partition wrapperApiPart = mapiPart.deepCopy();
-      org.apache.hadoop.hive.metastore.api.Table t = context.getHandler().get_table(
+      org.apache.hadoop.hive.metastore.api.Table t = context.getHandler().get_table_core(
           mapiPart.getDbName(), mapiPart.getTableName());
       if (wrapperApiPart.getSd() == null){
         // In the cases of create partition, by the time this event fires, the partition

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java?rev=1628562&r1=1628561&r2=1628562&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java Tue Sep 30 22:09:39 2014
@@ -85,7 +85,7 @@ public abstract class HiveAuthorizationP
         return hiveClient.getDatabase(dbName);
       } else {
         try {
-          return handler.get_database(dbName);
+          return handler.get_database_core(dbName);
         } catch (NoSuchObjectException e) {
           throw new HiveException(e);
         } catch (MetaException e) {