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 2013/10/05 21:25:11 UTC

svn commit: r1529500 - in /hive/branches/branch-0.12/ql/src: java/org/apache/hadoop/hive/ql/security/authorization/ test/org/apache/hadoop/hive/ql/security/

Author: thejas
Date: Sat Oct  5 19:25:10 2013
New Revision: 1529500

URL: http://svn.apache.org/r1529500
Log:
HIVE-5402 : StorageBasedAuthorizationProvider is not correctly able to determine that it is running from client-side (Sushanth Sowmyan via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
Removed:
    hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestDefaultHiveMetastoreAuthorizationProvider.java
Modified:
    hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java

Modified: hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java?rev=1529500&r1=1529499&r2=1529500&view=diff
==============================================================================
--- hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java (original)
+++ hive/branches/branch-0.12/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java Sat Oct  5 19:25:10 2013
@@ -32,11 +32,13 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.Warehouse;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.Partition;
 import org.apache.hadoop.hive.ql.metadata.Table;
@@ -62,19 +64,30 @@ public class StorageBasedAuthorizationPr
     implements HiveMetastoreAuthorizationProvider {
 
   private Warehouse wh;
+  private boolean isRunFromMetaStore = false;
 
   /**
    * Make sure that the warehouse variable is set up properly.
    * @throws MetaException if unable to instantiate
    */
-  private void initWh() throws MetaException {
+  private void initWh() throws MetaException, HiveException {
     if (wh == null){
-      if(!hive_db.isRunFromMetaStore()){
+      if(!isRunFromMetaStore){
+        // Note, although HiveProxy has a method that allows us to check if we're being
+        // called from the metastore or from the client, we don't have an initialized HiveProxy
+        // till we explicitly initialize it as being from the client side. So, we have a
+        // chicken-and-egg problem. So, we now track whether or not we're running from client-side
+        // in the SBAP itself.
+        hive_db = new HiveProxy(Hive.get(new HiveConf(getConf(), StorageBasedAuthorizationProvider.class)));
         this.wh = new Warehouse(getConf());
+        if (this.wh == null){
+          // If wh is still null after just having initialized it, bail out - something's very wrong.
+          throw new IllegalStateException("Unable to initialize Warehouse from clientside.");
+        }
       }else{
         // not good if we reach here, this was initialized at setMetaStoreHandler() time.
         // this means handler.getWh() is returning null. Error out.
-        throw new IllegalStateException("Unitialized Warehouse from MetastoreHandler");
+        throw new IllegalStateException("Uninitialized Warehouse from MetastoreHandler");
       }
     }
   }
@@ -164,6 +177,7 @@ public class StorageBasedAuthorizationPr
   public void setMetaStoreHandler(HMSHandler handler) {
     hive_db.setHandler(handler);
     this.wh = handler.getWh();
+    this.isRunFromMetaStore = true;
   }
 
   /**

Added: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java?rev=1529500&view=auto
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java (added)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestClientSideAuthorizationProvider.java Sat Oct  5 19:25:10 2013
@@ -0,0 +1,192 @@
+/**
+ * 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.util.ArrayList;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+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.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.DefaultHiveAuthorizationProvider;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * TestClientSideAuthorizationProvider : Simple base test for client side
+ * Authorization Providers. By default, tests DefaultHiveAuthorizationProvider
+ */
+public class TestClientSideAuthorizationProvider extends TestCase {
+  protected HiveConf clientHiveConf;
+  protected HiveMetaStoreClient msc;
+  protected Driver driver;
+  protected UserGroupInformation ugi;
+
+
+  protected String getAuthorizationProvider(){
+    return DefaultHiveAuthorizationProvider.class.getName();
+  }
+
+
+  @Override
+  protected void setUp() throws Exception {
+
+    super.setUp();
+
+    int port = MetaStoreUtils.findFreePort();
+
+    // Turn off metastore-side authorization
+    System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
+        "");
+
+    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
+
+    clientHiveConf = new HiveConf(this.getClass());
+
+    // Turn on client-side authorization
+    clientHiveConf.setBoolVar(HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED,true);
+    clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER.varname,
+        getAuthorizationProvider());
+    clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER.varname,
+        InjectableDummyAuthenticator.class.getName());
+    clientHiveConf.set(HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS.varname, "");
+
+    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);
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  private void validateCreateDb(Database expectedDb, String dbName) {
+    assertEquals(expectedDb.getName(), dbName);
+  }
+
+  private void validateCreateTable(Table expectedTable, String tblName, String dbName) {
+    assertNotNull(expectedTable);
+    assertEquals(expectedTable.getTableName(),tblName);
+    assertEquals(expectedTable.getDbName(),dbName);
+  }
+
+  public void testSimplePrivileges() throws Exception {
+    String dbName = "smpdb";
+    String tblName = "smptbl";
+
+    String userName = ugi.getUserName();
+
+    CommandProcessorResponse ret = driver.run("create database " + dbName);
+    assertEquals(0,ret.getResponseCode());
+    Database db = msc.getDatabase(dbName);
+    String dbLocn = db.getLocationUri();
+
+    validateCreateDb(db,dbName);
+    disallowCreateInDb(dbName, userName, dbLocn);
+
+    driver.run("use " + dbName);
+    ret = driver.run(
+        String.format("create table %s (a string) partitioned by (b string)", tblName));
+
+    // failure from not having permissions to create table
+    assertNoPrivileges(ret);
+
+    allowCreateInDb(dbName, userName, dbLocn);
+
+    driver.run("use " + dbName);
+    ret = driver.run(
+        String.format("create table %s (a string) partitioned by (b string)", tblName));
+
+    assertEquals(0,ret.getResponseCode()); // now it succeeds.
+    Table tbl = msc.getTable(dbName, tblName);
+
+    validateCreateTable(tbl,tblName, dbName);
+
+    String fakeUser = "mal";
+    List<String> fakeGroupNames = new ArrayList<String>();
+    fakeGroupNames.add("groupygroup");
+
+    InjectableDummyAuthenticator.injectUserName(fakeUser);
+    InjectableDummyAuthenticator.injectGroupNames(fakeGroupNames);
+    InjectableDummyAuthenticator.injectMode(true);
+
+    ret = driver.run(
+        String.format("create table %s (a string) partitioned by (b string)", tblName+"mal"));
+
+    assertNoPrivileges(ret);
+
+    disallowCreateInTbl(tbl.getTableName(), userName, tbl.getSd().getLocation());
+    ret = driver.run("alter table "+tblName+" add partition (b='2011')");
+    assertNoPrivileges(ret);
+
+    InjectableDummyAuthenticator.injectMode(false);
+    allowCreateInTbl(tbl.getTableName(), userName, tbl.getSd().getLocation());
+
+    ret = driver.run("alter table "+tblName+" add partition (b='2011')");
+    assertEquals(0,ret.getResponseCode());
+
+  }
+
+  protected void allowCreateInTbl(String tableName, String userName, String location)
+      throws Exception{
+    driver.run("grant create on table "+tableName+" to user "+userName);
+  }
+
+  protected void disallowCreateInTbl(String tableName, String userName, String location)
+      throws Exception {
+    // nothing needed here by default
+  }
+
+
+  protected void allowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    driver.run("grant create on database "+dbName+" to user "+userName);
+  }
+
+  protected void disallowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    // nothing needed here by default
+  }
+
+  protected void assertNoPrivileges(CommandProcessorResponse ret){
+    assertNotNull(ret);
+    assertFalse(0 == ret.getResponseCode());
+    assertTrue(ret.getErrorMessage().indexOf("No privilege") != -1);
+  }
+
+
+}

Added: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java?rev=1529500&view=auto
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java (added)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestMetastoreAuthorizationProvider.java Sat Oct  5 19:25:10 2013
@@ -0,0 +1,266 @@
+/**
+ * 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.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+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.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.Partition;
+import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+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.DefaultHiveMetastoreAuthorizationProvider;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.serde.serdeConstants;
+import org.apache.hadoop.hive.shims.ShimLoader;
+import org.apache.hadoop.security.UserGroupInformation;
+
+/**
+ * TestHiveMetastoreAuthorizationProvider. Test case for
+ * HiveMetastoreAuthorizationProvider, and by default,
+ * for DefaultHiveMetaStoreAuthorizationProvider
+ * using {@link org.apache.hadoop.hive.metastore.AuthorizationPreEventListener}
+ * and {@link org.apache.hadoop.hive.}
+ *
+ * Note that while we do use the hive driver to test, that is mostly for test
+ * writing ease, and it has the same effect as using a metastore client directly
+ * because we disable hive client-side authorization for this test, and only
+ * turn on server-side auth.
+ *
+ * This test is also intended to be extended to provide tests for other
+ * authorization providers like StorageBasedAuthorizationProvider
+ */
+public class TestMetastoreAuthorizationProvider extends TestCase {
+  protected HiveConf clientHiveConf;
+  protected HiveMetaStoreClient msc;
+  protected Driver driver;
+  protected UserGroupInformation ugi;
+
+
+  protected String getAuthorizationProvider(){
+    return DefaultHiveMetastoreAuthorizationProvider.class.getName();
+  }
+
+
+  @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());
+    System.setProperty(HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_OWNER_GRANTS.varname, "");
+
+
+    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
+
+    clientHiveConf = new HiveConf(this.getClass());
+
+    // 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);
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  private void validateCreateDb(Database expectedDb, String dbName) {
+    assertEquals(expectedDb.getName(), dbName);
+  }
+
+  private void validateCreateTable(Table expectedTable, String tblName, String dbName) {
+    assertNotNull(expectedTable);
+    assertEquals(expectedTable.getTableName(),tblName);
+    assertEquals(expectedTable.getDbName(),dbName);
+  }
+
+  public void testSimplePrivileges() throws Exception {
+    String dbName = "smpdb";
+    String tblName = "smptbl";
+
+    String userName = ugi.getUserName();
+
+    CommandProcessorResponse ret = driver.run("create database " + dbName);
+    assertEquals(0,ret.getResponseCode());
+    Database db = msc.getDatabase(dbName);
+    String dbLocn = db.getLocationUri();
+
+    validateCreateDb(db,dbName);
+    disallowCreateInDb(dbName, userName, dbLocn);
+
+    driver.run("use " + dbName);
+    ret = driver.run(
+        String.format("create table %s (a string) partitioned by (b string)", tblName));
+
+    assertEquals(1,ret.getResponseCode());
+    // failure from not having permissions to create table
+
+    ArrayList<FieldSchema> fields = new ArrayList<FieldSchema>(2);
+    fields.add(new FieldSchema("a", serdeConstants.STRING_TYPE_NAME, ""));
+
+    Table ttbl = new Table();
+    ttbl.setDbName(dbName);
+    ttbl.setTableName(tblName);
+    StorageDescriptor sd = new StorageDescriptor();
+    ttbl.setSd(sd);
+    sd.setCols(fields);
+    sd.setParameters(new HashMap<String, String>());
+    sd.getParameters().put("test_param_1", "Use this for comments etc");
+    sd.setSerdeInfo(new SerDeInfo());
+    sd.getSerdeInfo().setName(ttbl.getTableName());
+    sd.getSerdeInfo().setParameters(new HashMap<String, String>());
+    sd.getSerdeInfo().getParameters().put(
+        org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT, "1");
+    sd.getSerdeInfo().setSerializationLib(
+        org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe.class.getName());
+    ttbl.setPartitionKeys(new ArrayList<FieldSchema>());
+
+    MetaException me = null;
+    try {
+      msc.createTable(ttbl);
+    } catch (MetaException e){
+      me = e;
+    }
+    assertNoPrivileges(me);
+
+    allowCreateInDb(dbName, userName, dbLocn);
+
+    driver.run("use " + dbName);
+    ret = driver.run(
+        String.format("create table %s (a string) partitioned by (b string)", tblName));
+
+    assertEquals(0,ret.getResponseCode()); // now it succeeds.
+    Table tbl = msc.getTable(dbName, tblName);
+
+    validateCreateTable(tbl,tblName, dbName);
+
+    String fakeUser = "mal";
+    List<String> fakeGroupNames = new ArrayList<String>();
+    fakeGroupNames.add("groupygroup");
+
+    InjectableDummyAuthenticator.injectUserName(fakeUser);
+    InjectableDummyAuthenticator.injectGroupNames(fakeGroupNames);
+    InjectableDummyAuthenticator.injectMode(true);
+
+    ret = driver.run(
+        String.format("create table %s (a string) partitioned by (b string)", tblName+"mal"));
+
+    assertEquals(1,ret.getResponseCode());
+
+    ttbl.setTableName(tblName+"mal");
+    me = null;
+    try {
+      msc.createTable(ttbl);
+    } catch (MetaException e){
+      me = e;
+    }
+    assertNoPrivileges(me);
+
+    disallowCreateInTbl(tbl.getTableName(), userName, tbl.getSd().getLocation());
+    ret = driver.run("alter table "+tblName+" add partition (b='2011')");
+    assertEquals(1,ret.getResponseCode());
+
+    List<String> ptnVals = new ArrayList<String>();
+    ptnVals.add("b=2011");
+    Partition tpart = new Partition();
+    tpart.setDbName(dbName);
+    tpart.setTableName(tblName);
+    tpart.setValues(ptnVals);
+    tpart.setParameters(new HashMap<String, String>());
+    tpart.setSd(tbl.getSd().deepCopy());
+    tpart.getSd().setSerdeInfo(tbl.getSd().getSerdeInfo().deepCopy());
+    tpart.getSd().setLocation(tbl.getSd().getLocation() + "/tpart");
+
+    me = null;
+    try {
+      msc.add_partition(tpart);
+    } catch (MetaException e){
+      me = e;
+    }
+    assertNoPrivileges(me);
+
+    InjectableDummyAuthenticator.injectMode(false);
+    allowCreateInTbl(tbl.getTableName(), userName, tbl.getSd().getLocation());
+
+    ret = driver.run("alter table "+tblName+" add partition (b='2011')");
+    assertEquals(0,ret.getResponseCode());
+
+  }
+
+  protected void allowCreateInTbl(String tableName, String userName, String location)
+      throws Exception{
+    driver.run("grant create on table "+tableName+" to user "+userName);
+  }
+
+  protected void disallowCreateInTbl(String tableName, String userName, String location)
+      throws Exception {
+    driver.run("revoke create on table "+tableName+" from user "+userName);
+  }
+
+
+  protected void allowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    driver.run("grant create on database "+dbName+" to user "+userName);
+  }
+
+  protected void disallowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    driver.run("revoke create on database "+dbName+" from user "+userName);
+  }
+
+  protected void assertNoPrivileges(MetaException me){
+    assertNotNull(me);
+    assertTrue(me.getMessage().indexOf("No privilege") != -1);
+  }
+
+}

Added: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java?rev=1529500&view=auto
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java (added)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedClientSideAuthorizationProvider.java Sat Oct  5 19:25:10 2013
@@ -0,0 +1,79 @@
+/**
+ * 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 org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
+
+/**
+ * TestStorageBasedClientSideAuthorizationProvider : Overrides
+ * TestClientSideAuthorizationProvider to test StorageBasedAuthorizationProvider
+ * on the client side.
+ */
+
+public class TestStorageBasedClientSideAuthorizationProvider extends
+    TestClientSideAuthorizationProvider {
+
+  @Override
+  protected String getAuthorizationProvider(){
+    return StorageBasedAuthorizationProvider.class.getName();
+  }
+
+  @Override
+  protected void allowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    setPermissions(location,"-rwxr--r--");
+  }
+
+  @Override
+  protected void disallowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    setPermissions(location,"-r--r--r--");
+  }
+
+  @Override
+  protected void allowCreateInTbl(String tableName, String userName, String location)
+      throws Exception{
+    setPermissions(location,"-rwxr--r--");
+  }
+
+
+  @Override
+  protected void disallowCreateInTbl(String tableName, String userName, String location)
+      throws Exception {
+    setPermissions(location,"-r--r--r--");
+  }
+
+  private void setPermissions(String locn, String permissions) throws Exception {
+    FileSystem fs = FileSystem.get(new URI(locn), clientHiveConf);
+    fs.setPermission(new Path(locn), FsPermission.valueOf(permissions));
+  }
+
+  @Override
+  protected void assertNoPrivileges(CommandProcessorResponse ret){
+    assertNotNull(ret);
+    assertFalse(0 == ret.getResponseCode());
+    assertTrue(ret.getErrorMessage().indexOf("not permitted") != -1);
+  }
+
+}

Added: hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java?rev=1529500&view=auto
==============================================================================
--- hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java (added)
+++ hive/branches/branch-0.12/ql/src/test/org/apache/hadoop/hive/ql/security/TestStorageBasedMetastoreAuthorizationProvider.java Sat Oct  5 19:25:10 2013
@@ -0,0 +1,83 @@
+/**
+ * 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 org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.ql.security.authorization.StorageBasedAuthorizationProvider;
+
+/**
+ * TestStorageBasedMetastoreAuthorizationProvider. Test case for
+ * StorageBasedAuthorizationProvider, by overriding methods defined in
+ * TestMetastoreAuthorizationProvider
+ *
+ * Note that while we do use the hive driver to test, that is mostly for test
+ * writing ease, and it has the same effect as using a metastore client directly
+ * because we disable hive client-side authorization for this test, and only
+ * turn on server-side auth.
+ */
+public class TestStorageBasedMetastoreAuthorizationProvider extends
+    TestMetastoreAuthorizationProvider {
+
+  @Override
+  protected String getAuthorizationProvider(){
+    return StorageBasedAuthorizationProvider.class.getName();
+  }
+
+  @Override
+  protected void allowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    setPermissions(location,"-rwxr--r--");
+  }
+
+  @Override
+  protected void disallowCreateInDb(String dbName, String userName, String location)
+      throws Exception {
+    setPermissions(location,"-r--r--r--");
+  }
+
+  @Override
+  protected void allowCreateInTbl(String tableName, String userName, String location)
+      throws Exception{
+    setPermissions(location,"-rwxr--r--");
+  }
+
+
+  @Override
+  protected void disallowCreateInTbl(String tableName, String userName, String location)
+      throws Exception {
+    setPermissions(location,"-r--r--r--");
+  }
+
+  private void setPermissions(String locn, String permissions) throws Exception {
+    FileSystem fs = FileSystem.get(new URI(locn), clientHiveConf);
+    fs.setPermission(new Path(locn), FsPermission.valueOf(permissions));
+  }
+
+  @Override
+  protected void assertNoPrivileges(MetaException me){
+    assertNotNull(me);
+    assertTrue(me.getMessage().indexOf("not permitted") != -1);
+  }
+
+}