You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by ha...@apache.org on 2012/12/12 00:29:42 UTC

svn commit: r1420483 [2/2] - in /hive/branches/branch-0.10: common/src/java/org/apache/hadoop/hive/conf/ conf/ metastore/src/java/org/apache/hadoop/hive/metastore/ ql/src/java/org/apache/hadoop/hive/ql/metadata/ ql/src/java/org/apache/hadoop/hive/ql/se...

Modified: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Tue Dec 11 23:29:37 2012
@@ -282,10 +282,11 @@ public class SessionState {
     }
 
     try {
-      startSs.authenticator = HiveUtils.getAuthenticator(startSs
-          .getConf());
-      startSs.authorizer = HiveUtils.getAuthorizeProviderManager(startSs
-          .getConf(), startSs.authenticator);
+      startSs.authenticator = HiveUtils.getAuthenticator(
+          startSs.getConf(),HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
+      startSs.authorizer = HiveUtils.getAuthorizeProviderManager(
+          startSs.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER,
+          startSs.authenticator);
       startSs.createTableGrants = CreateTableAutomaticGrant.create(startSs
           .getConf());
     } catch (HiveException e) {

Added: hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,204 @@
+/**
+ * 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 org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+
+public class DummyHiveMetastoreAuthorizationProvider implements HiveMetastoreAuthorizationProvider {
+
+
+  protected HiveAuthenticationProvider authenticator;
+
+  public enum AuthCallContextType {
+    USER,
+    DB,
+    TABLE,
+    PARTITION,
+    TABLE_AND_PARTITION
+  };
+
+  class AuthCallContext {
+
+    public AuthCallContextType type;
+    public List<Object> authObjects;
+    public Privilege[] readRequiredPriv;
+    public Privilege[] writeRequiredPriv;
+
+    AuthCallContext(AuthCallContextType typeOfCall,
+        Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) {
+      this.type = typeOfCall;
+      this.authObjects = new ArrayList<Object>();
+      this.readRequiredPriv = readRequiredPriv;
+      this.writeRequiredPriv = writeRequiredPriv;
+    }
+    AuthCallContext(AuthCallContextType typeOfCall, Object authObject,
+        Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) {
+      this(typeOfCall,readRequiredPriv,writeRequiredPriv);
+      this.authObjects.add(authObject);
+    }
+    AuthCallContext(AuthCallContextType typeOfCall, List<? extends Object> authObjects,
+        Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) {
+      this(typeOfCall,readRequiredPriv,writeRequiredPriv);
+      this.authObjects.addAll(authObjects);
+    }
+  }
+
+  public static final List<AuthCallContext> authCalls = new ArrayList<AuthCallContext>();
+
+  private Configuration conf;
+  public static final Log LOG = LogFactory.getLog(
+      DummyHiveMetastoreAuthorizationProvider.class);;
+
+  @Override
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    try {
+      init(conf);
+    } catch (HiveException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  @Override
+  public HiveAuthenticationProvider getAuthenticator() {
+    return authenticator;
+  }
+
+  @Override
+  public void setAuthenticator(HiveAuthenticationProvider authenticator) {
+    this.authenticator = authenticator;
+  }
+
+  @Override
+  public void init(Configuration conf) throws HiveException {
+    debugLog("DHMAP.init");
+  }
+
+  @Override
+  public void authorize(Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    debugLog("DHMAP.authorize " +
+      "read:" + debugPrivPrint(readRequiredPriv) +
+      " , write:" + debugPrivPrint(writeRequiredPriv)
+      );
+    authCalls.add(new AuthCallContext(AuthCallContextType.USER,
+        readRequiredPriv, writeRequiredPriv));
+  }
+
+  @Override
+  public void authorize(Database db, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    debugLog("DHMAP.authorizedb " +
+        "db:" + db.getName() +
+        " , read:" + debugPrivPrint(readRequiredPriv) +
+        " , write:" + debugPrivPrint(writeRequiredPriv)
+        );
+    authCalls.add(new AuthCallContext(AuthCallContextType.DB,
+        db, readRequiredPriv, writeRequiredPriv));
+  }
+
+  @Override
+  public void authorize(Table table, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    debugLog("DHMAP.authorizetbl " +
+        "tbl:" + table.getCompleteName() +
+        " , read:" + debugPrivPrint(readRequiredPriv) +
+        " , write:" + debugPrivPrint(writeRequiredPriv)
+        );
+    authCalls.add(new AuthCallContext(AuthCallContextType.TABLE,
+        table, readRequiredPriv, writeRequiredPriv));
+
+  }
+
+  @Override
+  public void authorize(Partition part, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    debugLog("DHMAP.authorizepart " +
+        "tbl:" + part.getTable().getCompleteName() +
+        " , part: " + part.getName() +
+        " , read:" + debugPrivPrint(readRequiredPriv) +
+        " , write:" + debugPrivPrint(writeRequiredPriv)
+        );
+    authCalls.add(new AuthCallContext(AuthCallContextType.PARTITION,
+        part, readRequiredPriv, writeRequiredPriv));
+
+  }
+
+  @Override
+  public void authorize(Table table, Partition part, List<String> columns,
+      Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) throws HiveException,
+      AuthorizationException {
+    debugLog("DHMAP.authorizecols " +
+        "tbl:" + table.getCompleteName() +
+        " , part: " + part.getName() +
+        " . cols: " + columns.toString() +
+        " , read:" + debugPrivPrint(readRequiredPriv) +
+        " , write:" + debugPrivPrint(writeRequiredPriv)
+        );
+    List<Object> authObjects = new ArrayList<Object>();
+    authObjects.add(table);
+    authObjects.add(part);
+    authCalls.add(new AuthCallContext(AuthCallContextType.TABLE_AND_PARTITION,
+        authObjects, readRequiredPriv, writeRequiredPriv));
+
+  }
+
+  private void debugLog(String s) {
+    LOG.debug(s);
+  }
+
+  private String debugPrivPrint(Privilege[] privileges) {
+    StringBuffer sb = new StringBuffer();
+    sb.append("Privileges{");
+    if (privileges != null){
+    for (Privilege p : privileges){
+      sb.append(p.toString());
+    }
+    }else{
+      sb.append("null");
+    }
+    sb.append("}");
+    return sb.toString();
+  }
+
+  @Override
+  public void setMetaStoreHandler(HMSHandler handler) {
+    debugLog("DHMAP.setMetaStoreHandler");
+  }
+
+
+}

Added: hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java (added)
+++ hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,105 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+/**
+ *
+ * InjectableDummyAuthenticator - An implementation of HiveMetastoreAuthenticationProvider
+ * that wraps another Authenticator, but when asked to inject a user provided username
+ * and groupnames, does so. This can be toggled back and forth to use in testing
+ */
+public class InjectableDummyAuthenticator implements HiveMetastoreAuthenticationProvider {
+
+  private static String userName;
+  private static List<String> groupNames;
+  private static boolean injectMode;
+  private static Class<? extends HiveMetastoreAuthenticationProvider> hmapClass =
+      HadoopDefaultMetastoreAuthenticator.class;
+  private HiveMetastoreAuthenticationProvider hmap;
+
+  public static void injectHmapClass(Class<? extends HiveMetastoreAuthenticationProvider> clazz){
+    hmapClass = clazz;
+  }
+
+  public static void injectUserName(String user){
+    userName = user;
+  }
+
+  public static void injectGroupNames(List<String> groups){
+    groupNames = groups;
+  }
+
+  public static void injectMode(boolean mode){
+    injectMode = mode;
+  }
+
+  @Override
+  public String getUserName() {
+    if (injectMode){
+      return userName;
+    } else {
+      return hmap.getUserName();
+    }
+  }
+
+  @Override
+  public List<String> getGroupNames() {
+    if (injectMode) {
+      return groupNames;
+    } else {
+      return hmap.getGroupNames();
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return hmap.getConf();
+  }
+
+  @Override
+  public void setConf(Configuration config) {
+    try {
+      hmap = (HiveMetastoreAuthenticationProvider) hmapClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException("Whoops, could not create an Authenticator of class " +
+          hmapClass.getName());
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException("Whoops, could not create an Authenticator of class " +
+          hmapClass.getName());
+    }
+
+    hmap.setConf(config);
+  }
+
+  @Override
+  public void setMetaStoreHandler(HMSHandler handler) {
+    hmap.setMetaStoreHandler(handler);
+  }
+
+  @Override
+  public void destroy() throws HiveException {
+    hmap.destroy();
+  }
+
+}

Added: hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java (added)
+++ hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,303 @@
+/**
+ * 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.io.IOException;
+import java.net.ServerSocket;
+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.Partition;
+import org.apache.hadoop.hive.metastore.api.Table;
+import org.apache.hadoop.hive.ql.Driver;
+import org.apache.hadoop.hive.ql.security.DummyHiveMetastoreAuthorizationProvider.AuthCallContext;
+import org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.apache.hadoop.hive.shims.ShimLoader;
+
+/**
+ * TestAuthorizationPreEventListener. Test case for
+ * {@link org.apache.hadoop.hive.metastore.AuthorizationPreEventListener} and
+ * {@link org.apache.hadoop.hive.metastore.MetaStorePreEventListener}
+ */
+public class TestAuthorizationPreEventListener extends TestCase {
+  private HiveConf clientHiveConf;
+  private HiveMetaStoreClient msc;
+  private Driver driver;
+
+  @Override
+  protected void setUp() throws Exception {
+
+    super.setUp();
+
+    int port = MetaStoreUtils.findFreePort();
+
+    System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
+        AuthorizationPreEventListener.class.getName());
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER.varname,
+        DummyHiveMetastoreAuthorizationProvider.class.getName());
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHENTICATOR_MANAGER.varname,
+        HadoopDefaultMetastoreAuthenticator.class.getName());
+
+    MetaStoreUtils.startMetaStore(port, ShimLoader.getHadoopThriftAuthBridge());
+
+    clientHiveConf = new HiveConf(this.getClass());
+
+    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, "");
+
+
+    SessionState.start(new CliSessionState(clientHiveConf));
+    msc = new HiveMetaStoreClient(clientHiveConf, null);
+    driver = new Driver(clientHiveConf);
+  }
+
+  private static String getFreeAvailablePort() throws IOException {
+    ServerSocket socket = new ServerSocket(0);
+    socket.setReuseAddress(true);
+    int port = socket.getLocalPort();
+    socket.close();
+    return "" + port;
+  }
+
+  @Override
+  protected void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  private void validateCreateDb(Database expectedDb, Database actualDb) {
+    assertEquals(expectedDb.getName(), actualDb.getName());
+    assertEquals(expectedDb.getLocationUri(), actualDb.getLocationUri());
+  }
+
+  private void validateTable(Table expectedTable, Table actualTable) {
+    assertEquals(expectedTable.getTableName(), actualTable.getTableName());
+    assertEquals(expectedTable.getDbName(), actualTable.getDbName());
+
+    // We won't try to be too strict in checking this because we're comparing
+    // table create intents with observed tables created.
+    // If it does have a location though, we will compare, as with external tables
+    if ((actualTable.getSd() != null) && (actualTable.getSd().getLocation() != null)){
+      assertEquals(expectedTable.getSd().getLocation(), actualTable.getSd().getLocation());
+    }
+  }
+
+  private void validateCreateTable(Table expectedTable, Table actualTable) {
+    validateTable(expectedTable, actualTable);
+  }
+
+  private void validateAddPartition(Partition expectedPartition, Partition actualPartition) {
+    validatePartition(expectedPartition,actualPartition);
+  }
+
+  private void validatePartition(Partition expectedPartition, Partition actualPartition) {
+    assertEquals(expectedPartition.getValues(),
+        actualPartition.getValues());
+    assertEquals(expectedPartition.getDbName(),
+        actualPartition.getDbName());
+    assertEquals(expectedPartition.getTableName(),
+        actualPartition.getTableName());
+
+    // assertEquals(expectedPartition.getSd().getLocation(),
+    //     actualPartition.getSd().getLocation());
+    // we don't compare locations, because the location can still be empty in
+    // the pre-event listener before it is created.
+
+    assertEquals(expectedPartition.getSd().getInputFormat(),
+        actualPartition.getSd().getInputFormat());
+    assertEquals(expectedPartition.getSd().getOutputFormat(),
+        actualPartition.getSd().getOutputFormat());
+    assertEquals(expectedPartition.getSd().getSerdeInfo(),
+        actualPartition.getSd().getSerdeInfo());
+
+  }
+
+  private void validateAlterPartition(Partition expectedOldPartition,
+      Partition expectedNewPartition, String actualOldPartitionDbName,
+      String actualOldPartitionTblName,List<String> actualOldPartitionValues,
+      Partition actualNewPartition) {
+    assertEquals(expectedOldPartition.getValues(), actualOldPartitionValues);
+    assertEquals(expectedOldPartition.getDbName(), actualOldPartitionDbName);
+    assertEquals(expectedOldPartition.getTableName(), actualOldPartitionTblName);
+
+    validatePartition(expectedNewPartition, actualNewPartition);
+  }
+
+  private void validateAlterTable(Table expectedOldTable, Table expectedNewTable,
+      Table actualOldTable, Table actualNewTable) {
+    validateTable(expectedOldTable, actualOldTable);
+    validateTable(expectedNewTable, actualNewTable);
+  }
+
+  private void validateDropPartition(Partition expectedPartition, Partition actualPartition) {
+    validatePartition(expectedPartition, actualPartition);
+  }
+
+  private void validateDropTable(Table expectedTable, Table actualTable) {
+    validateTable(expectedTable, actualTable);
+  }
+
+  private void validateDropDb(Database expectedDb, Database actualDb) {
+    assertEquals(expectedDb, actualDb);
+  }
+
+  public void testListener() throws Exception {
+    String dbName = "tmpdb";
+    String tblName = "tmptbl";
+    String renamed = "tmptbl2";
+    int listSize = 0;
+
+    List<AuthCallContext> authCalls = DummyHiveMetastoreAuthorizationProvider.authCalls;
+    assertEquals(authCalls.size(),listSize);
+
+    driver.run("create database " + dbName);
+    listSize++;
+    Database db = msc.getDatabase(dbName);
+
+    Database dbFromEvent = (Database)assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+        DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.DB);
+    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);
+
+    Table tblFromEvent = (
+        (org.apache.hadoop.hive.ql.metadata.Table)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
+            .getTTable();
+    validateCreateTable(tbl, tblFromEvent);
+
+    driver.run("alter table tmptbl add partition (b='2011')");
+    listSize++;
+    Partition part = msc.getPartition("tmpdb", "tmptbl", "b=2011");
+
+    Partition ptnFromEvent = (
+        (org.apache.hadoop.hive.ql.metadata.Partition)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.PARTITION))
+            .getTPartition();
+    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");
+
+    Partition ptnFromEventAfterAlter = (
+        (org.apache.hadoop.hive.ql.metadata.Partition)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.PARTITION))
+            .getTPartition();
+
+    validateAlterPartition(part, modifiedP, ptnFromEventAfterAlter.getDbName(),
+        ptnFromEventAfterAlter.getTableName(), ptnFromEventAfterAlter.getValues(),
+        ptnFromEventAfterAlter);
+
+
+    List<String> part_vals = new ArrayList<String>();
+    part_vals.add("c=2012");
+    Partition newPart = msc.appendPartition(dbName, tblName, part_vals);
+
+    listSize++;
+
+    Partition newPtnFromEvent = (
+        (org.apache.hadoop.hive.ql.metadata.Partition)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.PARTITION))
+            .getTPartition();
+    validateAddPartition(newPart,newPtnFromEvent);
+
+
+    driver.run(String.format("alter table %s rename to %s", tblName, renamed));
+    listSize++;
+
+    Table renamedTable = msc.getTable(dbName, renamed);
+    Table renamedTableFromEvent = (
+        (org.apache.hadoop.hive.ql.metadata.Table)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
+            .getTTable();
+
+    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++;
+
+    Partition ptnFromDropPartition = (
+        (org.apache.hadoop.hive.ql.metadata.Partition)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.PARTITION))
+            .getTPartition();
+
+    validateDropPartition(modifiedP, ptnFromDropPartition);
+
+    driver.run("drop table " + tblName);
+    listSize++;
+    Table tableFromDropTableEvent = (
+        (org.apache.hadoop.hive.ql.metadata.Table)
+        assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+            DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.TABLE))
+            .getTTable();
+
+
+    validateDropTable(tbl, tableFromDropTableEvent);
+
+    driver.run("drop database " + dbName);
+    listSize++;
+    Database dbFromDropDatabaseEvent =
+        (Database)assertAndExtractSingleObjectFromEvent(listSize, authCalls,
+        DummyHiveMetastoreAuthorizationProvider.AuthCallContextType.DB);
+
+    validateDropDb(db, dbFromDropDatabaseEvent);
+  }
+
+  public Object assertAndExtractSingleObjectFromEvent(int listSize,
+      List<AuthCallContext> authCalls,
+      DummyHiveMetastoreAuthorizationProvider.AuthCallContextType callType) {
+    assertEquals(listSize, authCalls.size());
+    assertEquals(1,authCalls.get(listSize-1).authObjects.size());
+
+    assertEquals(callType,authCalls.get(listSize-1).type);
+    return (authCalls.get(listSize-1).authObjects.get(0));
+  }
+
+}

Added: hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestDefaultHiveMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestDefaultHiveMetastoreAuthorizationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestDefaultHiveMetastoreAuthorizationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestDefaultHiveMetastoreAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,226 @@
+/**
+ * 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;
+
+/**
+ * TestDefaultHiveMetaStoreAuthorizationProvider. Test case for
+ * DefaultHiveMetastoreAuthorizationProvider
+ * using {@link org.apache.hadoop.hive.metastore.AuthorizationPreEventListener}
+ *
+ * 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 TestDefaultHiveMetastoreAuthorizationProvider extends TestCase {
+  private HiveConf clientHiveConf;
+  private HiveMetaStoreClient msc;
+  private Driver driver;
+  private UserGroupInformation ugi;
+
+  @Override
+  protected void setUp() throws Exception {
+
+    super.setUp();
+
+    int port = MetaStoreUtils.findFreePort();
+
+    System.setProperty(HiveConf.ConfVars.METASTORE_PRE_EVENT_LISTENERS.varname,
+        AuthorizationPreEventListener.class.getName());
+    System.setProperty(HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER.varname,
+        DefaultHiveMetastoreAuthorizationProvider.class.getName());
+    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());
+
+    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);
+
+    validateCreateDb(db,dbName);
+
+    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;
+    }
+    assertNotNull(me);
+    assertTrue(me.getMessage().indexOf("No privilege") != -1);
+
+    driver.run("grant create on database "+dbName+" to user "+userName);
+
+    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;
+    }
+    assertNotNull(me);
+    assertTrue(me.getMessage().indexOf("No privilege") != -1);
+
+    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;
+    }
+    assertNotNull(me);
+    assertTrue(me.getMessage().indexOf("No privilege") != -1);
+
+    InjectableDummyAuthenticator.injectMode(false);
+
+    ret = driver.run("alter table "+tblName+" add partition (b='2011')");
+    assertEquals(0,ret.getResponseCode());
+
+  }
+
+}