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 [1/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...

Author: hashutosh
Date: Tue Dec 11 23:29:37 2012
New Revision: 1420483

URL: http://svn.apache.org/viewvc?rev=1420483&view=rev
Log:
HIVE-3705 :Adding authorization capability to the metastore (Sushanth Sowmyan via Ashutosh Chauhan)

Added:
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/BitSetCheckedAuthorizationProvider.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
    hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/DummyHiveMetastoreAuthorizationProvider.java
    hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/InjectableDummyAuthenticator.java
    hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestAuthorizationPreEventListener.java
    hive/branches/branch-0.10/ql/src/test/org/apache/hadoop/hive/ql/security/TestDefaultHiveMetastoreAuthorizationProvider.java
Modified:
    hive/branches/branch-0.10/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/branch-0.10/conf/hive-default.xml.template
    hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
    hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultAuthenticator.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
    hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java

Modified: hive/branches/branch-0.10/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/branch-0.10/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Dec 11 23:29:37 2012
@@ -610,6 +610,11 @@ public class HiveConf extends Configurat
         "org.apache.hadoop.hive.ql.security.authorization.DefaultHiveAuthorizationProvider"),
     HIVE_AUTHENTICATOR_MANAGER("hive.security.authenticator.manager",
         "org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator"),
+    HIVE_METASTORE_AUTHORIZATION_MANAGER("hive.security.metastore.authorization.manager",
+        "org.apache.hadoop.hive.ql.security.authorization."
+        + "DefaultHiveMetastoreAuthorizationProvider"),
+    HIVE_METASTORE_AUTHENTICATOR_MANAGER("hive.security.metastore.authenticator.manager",
+        "org.apache.hadoop.hive.ql.security.HadoopDefaultMetastoreAuthenticator"),
     HIVE_AUTHORIZATION_TABLE_USER_GRANTS("hive.security.authorization.createtable.user.grants", ""),
     HIVE_AUTHORIZATION_TABLE_GROUP_GRANTS("hive.security.authorization.createtable.group.grants",
         ""),

Modified: hive/branches/branch-0.10/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/conf/hive-default.xml.template?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/conf/hive-default.xml.template (original)
+++ hive/branches/branch-0.10/conf/hive-default.xml.template Tue Dec 11 23:29:37 2012
@@ -1264,6 +1264,22 @@
 </property>
 
 <property>
+  <name>hive.security.metastore.authorization.manager</name>
+  <value>org.apache.hadoop.hive.ql.security.authorization.DefaultHiveMetastoreAuthorizationProvider</value>
+  <description>authorization manager class name to be used in the metastore for authorization.
+  The user defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider. 
+  </description>
+</property>
+
+<property>
+  <name>hive.security.metastore.authorization.manager</name>
+  <value>org.apache.hadoop.hive.ql.security.authorization.DefaultHiveMetastoreAuthorizationProvider</value>
+  <description>authorization manager class name to be used in the metastore for authorization.
+  The user defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider. 
+  </description>
+</property>
+
+<property>
   <name>hive.security.authenticator.manager</name>
   <value>org.apache.hadoop.hive.ql.security.HadoopDefaultAuthenticator</value>
   <description>hive client authenticator manager class name.
@@ -1271,6 +1287,20 @@
 </property>
 
 <property>
+  <name>hive.security.metastore.authenticator.manager</name>
+  <value>org.apache.hadoop.hive.ql.security.HadoopDefaultMetastoreAuthenticator</value>
+  <description>authenticator manager class name to be used in the metastore for authentication. 
+  The user defined authenticator should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.</description>
+</property>
+
+<property>
+  <name>hive.security.metastore.authenticator.manager</name>
+  <value>org.apache.hadoop.hive.ql.security.HadoopDefaultMetastoreAuthenticator</value>
+  <description>authenticator manager class name to be used in the metastore for authentication. 
+  The user defined authenticator should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.</description>
+</property>
+
+<property>
   <name>hive.security.authorization.createtable.user.grants</name>
   <value></value>
   <description>the privileges automatically granted to some users whenever a table gets created.

Modified: hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Tue Dec 11 23:29:37 2012
@@ -59,7 +59,6 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
 import org.apache.hadoop.hive.metastore.api.ConfigValSecurityException;
-import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.EnvironmentContext;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -87,6 +86,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.UnknownDBException;
 import org.apache.hadoop.hive.metastore.api.UnknownPartitionException;
 import org.apache.hadoop.hive.metastore.api.UnknownTableException;
+import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.events.AddPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterPartitionEvent;
 import org.apache.hadoop.hive.metastore.events.AlterTableEvent;
@@ -365,6 +365,10 @@ public class HiveMetaStore extends Thrif
       return conf;
     }
 
+    public Warehouse getWh() {
+      return wh;
+    }
+
     /**
      * Get a cached RawStore.
      *
@@ -397,7 +401,7 @@ public class HiveMetaStore extends Thrif
       } catch (NoSuchObjectException e) {
         ms.createDatabase(
             new Database(DEFAULT_DATABASE_NAME, DEFAULT_DATABASE_COMMENT,
-                getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString(), null));
+                wh.getDefaultDatabasePath(DEFAULT_DATABASE_NAME).toString(), null));
       }
       HMSHandler.createDefaultDB = true;
     }
@@ -515,22 +519,13 @@ public class HiveMetaStore extends Thrif
       return counters;
     }
 
-    private static final String DATABASE_WAREHOUSE_SUFFIX = ".db";
-
-    private Path getDefaultDatabasePath(String dbName) throws MetaException {
-      if (dbName.equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
-        return wh.getWhRoot();
-      }
-      return new Path(wh.getWhRoot(), dbName.toLowerCase() + DATABASE_WAREHOUSE_SUFFIX);
-    }
-
     private void create_database_core(RawStore ms, final Database db)
         throws AlreadyExistsException, InvalidObjectException, MetaException {
       if (!validateName(db.getName())) {
         throw new InvalidObjectException(db.getName() + " is not a valid database name");
       }
       if (null == db.getLocationUri()) {
-        db.setLocationUri(getDefaultDatabasePath(db.getName()).toString());
+        db.setLocationUri(wh.getDefaultDatabasePath(db.getName()).toString());
       } else {
         db.setLocationUri(wh.getDnsPath(new Path(db.getLocationUri())).toString());
       }

Modified: hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java (original)
+++ hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/MetaStoreUtils.java Tue Dec 11 23:29:37 2012
@@ -68,6 +68,8 @@ public class MetaStoreUtils {
   public static final String DEFAULT_DATABASE_NAME = "default";
   public static final String DEFAULT_DATABASE_COMMENT = "Default Hive database";
 
+  public static final String DATABASE_WAREHOUSE_SUFFIX = ".db";
+
   /**
    * printStackTrace
    *
@@ -1076,4 +1078,5 @@ public class MetaStoreUtils {
       throw new RuntimeException("Unable to instantiate " + theClass.getName(), e);
     }
   }
+
 }

Modified: hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java (original)
+++ hive/branches/branch-0.10/metastore/src/java/org/apache/hadoop/hive/metastore/Warehouse.java Tue Dec 11 23:29:37 2012
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.metastore;
 
+import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DATABASE_WAREHOUSE_SUFFIX;
 import static org.apache.hadoop.hive.metastore.MetaStoreUtils.DEFAULT_DATABASE_NAME;
 
 import java.io.FileNotFoundException;
@@ -164,6 +165,14 @@ public class Warehouse {
     return new Path(db.getLocationUri());
   }
 
+  public Path getDefaultDatabasePath(String dbName) throws MetaException {
+    if (dbName.equalsIgnoreCase(DEFAULT_DATABASE_NAME)) {
+      return getWhRoot();
+    }
+    return new Path(getWhRoot(), dbName.toLowerCase() + DATABASE_WAREHOUSE_SUFFIX);
+  }
+
+
   public Path getTablePath(Database db, String tableName)
       throws MetaException {
     return getDnsPath(new Path(getDatabasePath(db), tableName.toLowerCase()));

Modified: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java (original)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/metadata/HiveUtils.java Tue Dec 11 23:29:37 2012
@@ -321,10 +321,10 @@ public final class HiveUtils {
 
   @SuppressWarnings("unchecked")
   public static HiveAuthorizationProvider getAuthorizeProviderManager(
-      Configuration conf, HiveAuthenticationProvider authenticator) throws HiveException {
+      Configuration conf, HiveConf.ConfVars authorizationProviderConfKey,
+      HiveAuthenticationProvider authenticator) throws HiveException {
 
-    String clsStr = HiveConf.getVar(conf,
-        HiveConf.ConfVars.HIVE_AUTHORIZATION_MANAGER);
+    String clsStr = HiveConf.getVar(conf, authorizationProviderConfKey);
 
     HiveAuthorizationProvider ret = null;
     try {
@@ -346,11 +346,11 @@ public final class HiveUtils {
   }
 
   @SuppressWarnings("unchecked")
-  public static HiveAuthenticationProvider getAuthenticator(Configuration conf)
-      throws HiveException {
+  public static HiveAuthenticationProvider getAuthenticator(
+      Configuration conf, HiveConf.ConfVars authenticatorConfKey
+      ) throws HiveException {
 
-    String clsStr = HiveConf.getVar(conf,
-        HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER);
+    String clsStr = HiveConf.getVar(conf, authenticatorConfKey);
 
     HiveAuthenticationProvider ret = null;
     try {
@@ -370,6 +370,7 @@ public final class HiveUtils {
     return ret;
   }
 
+
   /**
    * Convert FieldSchemas to columnNames with backticks around them.
    */

Modified: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultAuthenticator.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultAuthenticator.java (original)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultAuthenticator.java Tue Dec 11 23:29:37 2012
@@ -30,7 +30,7 @@ public class HadoopDefaultAuthenticator 
 
   private String userName;
   private List<String> groupNames;
-  
+
   private Configuration conf;
 
   @Override

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HadoopDefaultMetastoreAuthenticator.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,31 @@
+/**
+ * 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.metastore.HiveMetaStore.HMSHandler;
+
+public class HadoopDefaultMetastoreAuthenticator extends HadoopDefaultAuthenticator
+  implements HiveMetastoreAuthenticationProvider {
+
+  @Override
+  public void setMetaStoreHandler(HMSHandler handler) {
+    setConf(handler.getHiveConf());
+  }
+
+}

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/HiveMetastoreAuthenticationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,41 @@
+/**
+ * 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.metastore.HiveMetaStore.HMSHandler;
+
+
+/**
+ * HiveMetastoreAuthenticationProvider is an interface extension
+ * from HiveAuthenticationProvider for authentication from the
+ * metastore side. The implementation should return userNames
+ * and groupNames, and take care that if the metastore is running
+ * a particular command as a user, it returns that user.
+ */
+public interface HiveMetastoreAuthenticationProvider extends HiveAuthenticationProvider{
+
+  /**
+   * Allows invoker of HiveMetastoreAuthenticationProvider to send in a
+   * hive metastore handler that can be used to provide data for any
+   * authentication that needs to be done.
+   * @param handler
+   */
+  void setMetaStoreHandler(HMSHandler handler);
+
+}

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/AuthorizationPreEventListener.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,276 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security.authorization;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.MetaStorePreEventListener;
+import org.apache.hadoop.hive.metastore.MetaStoreUtils;
+import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.InvalidOperationException;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.events.PreAddPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterPartitionEvent;
+import org.apache.hadoop.hive.metastore.events.PreAlterTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateDatabaseEvent;
+import org.apache.hadoop.hive.metastore.events.PreCreateTableEvent;
+import org.apache.hadoop.hive.metastore.events.PreDropDatabaseEvent;
+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.ql.metadata.AuthorizationException;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.security.HiveMetastoreAuthenticationProvider;
+
+/**
+ * AuthorizationPreEventListener : A MetaStorePreEventListener that
+ * performs authorization/authentication checks on the metastore-side.
+ *
+ * Note that this can only perform authorization checks on defined
+ * metastore PreEventContexts, such as the adding/dropping and altering
+ * of databases, tables and partitions.
+ */
+public class AuthorizationPreEventListener extends MetaStorePreEventListener {
+
+  public static final Log LOG = LogFactory.getLog(
+      AuthorizationPreEventListener.class);
+
+  private static HiveConf conf;
+  private static HiveMetastoreAuthorizationProvider authorizer;
+  private static HiveMetastoreAuthenticationProvider authenticator;
+
+  public AuthorizationPreEventListener(Configuration config) throws HiveException {
+    super(config);
+
+    authenticator = (HiveMetastoreAuthenticationProvider) HiveUtils.getAuthenticator(
+        config, HiveConf.ConfVars.HIVE_METASTORE_AUTHENTICATOR_MANAGER);
+    authorizer = (HiveMetastoreAuthorizationProvider) HiveUtils.getAuthorizeProviderManager(
+        config, HiveConf.ConfVars.HIVE_METASTORE_AUTHORIZATION_MANAGER, authenticator);
+  }
+
+  @Override
+  public void onEvent(PreEventContext context) throws MetaException, NoSuchObjectException,
+      InvalidOperationException {
+
+    authenticator.setMetaStoreHandler(context.getHandler());
+    authorizer.setMetaStoreHandler(context.getHandler());
+
+    switch (context.getEventType()) {
+    case CREATE_TABLE:
+      authorizeCreateTable((PreCreateTableEvent)context);
+      break;
+    case DROP_TABLE:
+      authorizeDropTable((PreDropTableEvent)context);
+      break;
+    case ALTER_TABLE:
+      authorizeAlterTable((PreAlterTableEvent)context);
+      break;
+    case ADD_PARTITION:
+      authorizeAddPartition((PreAddPartitionEvent)context);
+      break;
+    case DROP_PARTITION:
+      authorizeDropPartition((PreDropPartitionEvent)context);
+      break;
+    case ALTER_PARTITION:
+      authorizeAlterPartition((PreAlterPartitionEvent)context);
+      break;
+    case CREATE_DATABASE:
+      authorizeCreateDatabase((PreCreateDatabaseEvent)context);
+      break;
+    case DROP_DATABASE:
+      authorizeDropDatabase((PreDropDatabaseEvent)context);
+      break;
+    case LOAD_PARTITION_DONE:
+      // noop for now
+      break;
+    default:
+      break;
+    }
+
+  }
+
+  private void authorizeCreateDatabase(PreCreateDatabaseEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      authorizer.authorize(new Database(context.getDatabase()),
+          HiveOperation.CREATEDATABASE.getInputRequiredPrivileges(),
+          HiveOperation.CREATEDATABASE.getOutputRequiredPrivileges());
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeDropDatabase(PreDropDatabaseEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      authorizer.authorize(new Database(context.getDatabase()),
+          HiveOperation.DROPDATABASE.getInputRequiredPrivileges(),
+          HiveOperation.DROPDATABASE.getOutputRequiredPrivileges());
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeCreateTable(PreCreateTableEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      authorizer.authorize(getTableFromApiTable(context.getTable()),
+          HiveOperation.CREATETABLE.getInputRequiredPrivileges(),
+          HiveOperation.CREATETABLE.getOutputRequiredPrivileges());
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeDropTable(PreDropTableEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      authorizer.authorize(getTableFromApiTable(context.getTable()),
+          HiveOperation.DROPTABLE.getInputRequiredPrivileges(),
+          HiveOperation.DROPTABLE.getOutputRequiredPrivileges());
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeAlterTable(PreAlterTableEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      authorizer.authorize(getTableFromApiTable(context.getOldTable()),
+          null,
+          new Privilege[]{Privilege.ALTER_METADATA});
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeAddPartition(PreAddPartitionEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      org.apache.hadoop.hive.metastore.api.Partition mapiPart = context.getPartition();
+      authorizer.authorize(getPartitionFromApiPartition(mapiPart, context),
+          HiveOperation.ALTERTABLE_ADDPARTS.getInputRequiredPrivileges(),
+          HiveOperation.ALTERTABLE_ADDPARTS.getOutputRequiredPrivileges());
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (NoSuchObjectException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeDropPartition(PreDropPartitionEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      org.apache.hadoop.hive.metastore.api.Partition mapiPart = context.getPartition();
+      authorizer.authorize(getPartitionFromApiPartition(mapiPart, context),
+          HiveOperation.ALTERTABLE_DROPPARTS.getInputRequiredPrivileges(),
+          HiveOperation.ALTERTABLE_DROPPARTS.getOutputRequiredPrivileges());
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (NoSuchObjectException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private void authorizeAlterPartition(PreAlterPartitionEvent context)
+      throws InvalidOperationException, MetaException {
+    try {
+      org.apache.hadoop.hive.metastore.api.Partition mapiPart = context.getNewPartition();
+      authorizer.authorize(getPartitionFromApiPartition(mapiPart, context),
+          null,
+          new Privilege[]{Privilege.ALTER_METADATA});
+    } catch (AuthorizationException e) {
+      throw invalidOperationException(e);
+    } catch (NoSuchObjectException e) {
+      throw invalidOperationException(e);
+    } catch (HiveException e) {
+      throw metaException(e);
+    }
+  }
+
+  private Table getTableFromApiTable(org.apache.hadoop.hive.metastore.api.Table apiTable) {
+    org.apache.hadoop.hive.metastore.api.Table tTable = apiTable.deepCopy();
+    if (tTable.getTableType() == null){
+      // TableType specified was null, we need to figure out what type it was.
+      if (MetaStoreUtils.isExternalTable(tTable)){
+        tTable.setTableType(TableType.EXTERNAL_TABLE.toString());
+      } else if (MetaStoreUtils.isIndexTable(tTable)) {
+        tTable.setTableType(TableType.INDEX_TABLE.toString());
+      } else if ((tTable.getSd() == null) || (tTable.getSd().getLocation() == null)) {
+        tTable.setTableType(TableType.VIRTUAL_VIEW.toString());
+      } else {
+        tTable.setTableType(TableType.MANAGED_TABLE.toString());
+      }
+    }
+    Table tbl = new Table(tTable);
+    return tbl;
+  }
+
+  private Partition getPartitionFromApiPartition(
+      org.apache.hadoop.hive.metastore.api.Partition mapiPart,
+      PreEventContext context) throws HiveException, NoSuchObjectException, MetaException {
+    org.apache.hadoop.hive.metastore.api.Partition tPart = mapiPart.deepCopy();
+    org.apache.hadoop.hive.metastore.api.Table t = context.getHandler().get_table(
+        mapiPart.getDbName(), mapiPart.getTableName());
+    if (tPart.getSd() == null){
+      // In the cases of create partition, by the time this event fires, the partition
+      // object has not yet come into existence, and thus will not yet have a
+      // location or an SD, but these are needed to create a ql.metadata.Partition,
+      // so we use the table's SD. The only place this is used is by the
+      // authorization hooks, so we will not affect code flow in the metastore itself.
+      tPart.setSd(t.getSd());
+    }
+    return new Partition(getTableFromApiTable(t),tPart);
+  }
+
+  private InvalidOperationException invalidOperationException(Exception e) {
+    InvalidOperationException ex = new InvalidOperationException(e.getMessage());
+    ex.initCause(e.getCause());
+    return ex;
+  }
+
+  private MetaException metaException(HiveException e) {
+    MetaException ex =  new MetaException(e.getMessage());
+    ex.initCause(e);
+    return ex;
+  }
+
+}

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/BitSetCheckedAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/BitSetCheckedAuthorizationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/BitSetCheckedAuthorizationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/BitSetCheckedAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,502 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security.authorization;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
+import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
+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;
+
+public abstract class BitSetCheckedAuthorizationProvider extends
+    HiveAuthorizationProviderBase {
+
+  static class BitSetChecker {
+
+    boolean[] inputCheck = null;
+    boolean[] outputCheck = null;
+
+    public static BitSetChecker getBitSetChecker(Privilege[] inputRequiredPriv,
+        Privilege[] outputRequiredPriv) {
+      BitSetChecker checker = new BitSetChecker();
+      if (inputRequiredPriv != null) {
+        checker.inputCheck = new boolean[inputRequiredPriv.length];
+        for (int i = 0; i < checker.inputCheck.length; i++) {
+          checker.inputCheck[i] = false;
+        }
+      }
+      if (outputRequiredPriv != null) {
+        checker.outputCheck = new boolean[outputRequiredPriv.length];
+        for (int i = 0; i < checker.outputCheck.length; i++) {
+          checker.outputCheck[i] = false;
+        }
+      }
+
+      return checker;
+    }
+
+  }
+
+  @Override
+  public void authorize(Privilege[] inputRequiredPriv,
+      Privilege[] outputRequiredPriv) throws HiveException, AuthorizationException {
+
+    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
+        outputRequiredPriv);
+    boolean[] inputCheck = checker.inputCheck;
+    boolean[] outputCheck = checker.outputCheck;
+
+    authorizeUserPriv(inputRequiredPriv, inputCheck, outputRequiredPriv,
+        outputCheck);
+    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
+        inputCheck, outputCheck, null, null, null, null);
+  }
+
+  @Override
+  public void authorize(Database db, Privilege[] inputRequiredPriv,
+      Privilege[] outputRequiredPriv) throws HiveException, AuthorizationException {
+
+    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
+        outputRequiredPriv);
+    boolean[] inputCheck = checker.inputCheck;
+    boolean[] outputCheck = checker.outputCheck;
+
+    authorizeUserAndDBPriv(db, inputRequiredPriv, outputRequiredPriv,
+        inputCheck, outputCheck);
+
+    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
+        inputCheck, outputCheck, db.getName(), null, null, null);
+  }
+
+  @Override
+  public void authorize(Table table, Privilege[] inputRequiredPriv,
+      Privilege[] outputRequiredPriv) throws HiveException {
+
+    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
+        outputRequiredPriv);
+    boolean[] inputCheck = checker.inputCheck;
+    boolean[] outputCheck = checker.outputCheck;
+
+    authorizeUserDBAndTable(table, inputRequiredPriv,
+        outputRequiredPriv, inputCheck, outputCheck);
+    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
+        inputCheck, outputCheck, table.getDbName(), table.getTableName(),
+        null, null);
+  }
+
+  @Override
+  public void authorize(Partition part, Privilege[] inputRequiredPriv,
+      Privilege[] outputRequiredPriv) throws HiveException {
+
+    //if the partition does not have partition level privilege, go to table level.
+    Table table = part.getTable();
+    if (table.getParameters().get("PARTITION_LEVEL_PRIVILEGE") == null || ("FALSE"
+        .equalsIgnoreCase(table.getParameters().get(
+            "PARTITION_LEVEL_PRIVILEGE")))) {
+      this.authorize(part.getTable(), inputRequiredPriv, outputRequiredPriv);
+      return;
+    }
+
+    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
+        outputRequiredPriv);
+    boolean[] inputCheck = checker.inputCheck;
+    boolean[] outputCheck = checker.outputCheck;
+
+    if (authorizeUserDbAndPartition(part, inputRequiredPriv, outputRequiredPriv,
+        inputCheck, outputCheck)){
+      return;
+    }
+
+    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
+        inputCheck, outputCheck, part.getTable().getDbName(), part
+            .getTable().getTableName(), part.getName(), null);
+  }
+
+  @Override
+  public void authorize(Table table, Partition part, List<String> columns,
+      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv)
+      throws HiveException {
+
+    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
+        outputRequiredPriv);
+    boolean[] inputCheck = checker.inputCheck;
+    boolean[] outputCheck = checker.outputCheck;
+
+    String partName = null;
+    List<String> partValues = null;
+    if (part != null
+        && (table.getParameters().get("PARTITION_LEVEL_PRIVILEGE") != null && ("TRUE"
+            .equalsIgnoreCase(table.getParameters().get(
+                "PARTITION_LEVEL_PRIVILEGE"))))) {
+      partName = part.getName();
+      partValues = part.getValues();
+    }
+
+    if (partValues == null) {
+      if (authorizeUserDBAndTable(table, inputRequiredPriv, outputRequiredPriv,
+          inputCheck, outputCheck)) {
+        return;
+      }
+    } else {
+      if (authorizeUserDbAndPartition(part, inputRequiredPriv,
+          outputRequiredPriv, inputCheck, outputCheck)) {
+        return;
+      }
+    }
+
+    for (String col : columns) {
+
+      BitSetChecker checker2 = BitSetChecker.getBitSetChecker(
+          inputRequiredPriv, outputRequiredPriv);
+      boolean[] inputCheck2 = checker2.inputCheck;
+      boolean[] outputCheck2 = checker2.outputCheck;
+
+      PrincipalPrivilegeSet partColumnPrivileges = hive_db
+          .get_privilege_set(HiveObjectType.COLUMN, table.getDbName(), table.getTableName(),
+              partValues, col, this.getAuthenticator().getUserName(), this
+                  .getAuthenticator().getGroupNames());
+
+      authorizePrivileges(partColumnPrivileges, inputRequiredPriv, inputCheck2,
+          outputRequiredPriv, outputCheck2);
+
+      if (inputCheck2 != null) {
+        booleanArrayOr(inputCheck2, inputCheck);
+      }
+      if (outputCheck2 != null) {
+        booleanArrayOr(inputCheck2, inputCheck);
+      }
+
+      checkAndThrowAuthorizationException(inputRequiredPriv,
+          outputRequiredPriv, inputCheck2, outputCheck2, table.getDbName(),
+          table.getTableName(), partName, col);
+    }
+  }
+
+  protected boolean authorizeUserPriv(Privilege[] inputRequiredPriv,
+      boolean[] inputCheck, Privilege[] outputRequiredPriv,
+      boolean[] outputCheck) throws HiveException {
+    PrincipalPrivilegeSet privileges = hive_db.get_privilege_set(
+        HiveObjectType.GLOBAL, null, null, null, null, this.getAuthenticator()
+            .getUserName(), this.getAuthenticator().getGroupNames());
+    return authorizePrivileges(privileges, inputRequiredPriv, inputCheck,
+        outputRequiredPriv, outputCheck);
+  }
+
+  /**
+   * Check privileges on User and DB. This is used before doing a check on
+   * table/partition objects, first check the user and DB privileges. If it
+   * passed on this check, no need to check against the table/partition hive
+   * object.
+   *
+   * @param db
+   * @param inputRequiredPriv
+   * @param outputRequiredPriv
+   * @param inputCheck
+   * @param outputCheck
+   * @return true if the check on user and DB privilege passed, which means no
+   *         need for privilege check on concrete hive objects.
+   * @throws HiveException
+   */
+  private boolean authorizeUserAndDBPriv(Database db,
+      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
+      boolean[] inputCheck, boolean[] outputCheck) throws HiveException {
+    if (authorizeUserPriv(inputRequiredPriv, inputCheck, outputRequiredPriv,
+        outputCheck)) {
+      return true;
+    }
+
+    PrincipalPrivilegeSet dbPrivileges = hive_db.get_privilege_set(
+        HiveObjectType.DATABASE, db.getName(), null, null, null, this
+            .getAuthenticator().getUserName(), this.getAuthenticator()
+            .getGroupNames());
+
+    if (authorizePrivileges(dbPrivileges, inputRequiredPriv, inputCheck,
+        outputRequiredPriv, outputCheck)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Check privileges on User, DB and table objects.
+   *
+   * @param table
+   * @param inputRequiredPriv
+   * @param outputRequiredPriv
+   * @param inputCheck
+   * @param outputCheck
+   * @return true if the check passed
+   * @throws HiveException
+   */
+  private boolean authorizeUserDBAndTable(Table table,
+      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
+      boolean[] inputCheck, boolean[] outputCheck) throws HiveException {
+
+    if (authorizeUserAndDBPriv(hive_db.getDatabase(table.getDbName()),
+        inputRequiredPriv, outputRequiredPriv, inputCheck, outputCheck)) {
+      return true;
+    }
+
+    PrincipalPrivilegeSet tablePrivileges = hive_db.get_privilege_set(
+        HiveObjectType.TABLE, table.getDbName(), table.getTableName(), null,
+        null, this.getAuthenticator().getUserName(), this.getAuthenticator()
+            .getGroupNames());
+
+    if (authorizePrivileges(tablePrivileges, inputRequiredPriv, inputCheck,
+        outputRequiredPriv, outputCheck)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  /**
+   * Check privileges on User, DB and table/Partition objects.
+   *
+   * @param part
+   * @param inputRequiredPriv
+   * @param outputRequiredPriv
+   * @param inputCheck
+   * @param outputCheck
+   * @return true if the check passed
+   * @throws HiveException
+   */
+  private boolean authorizeUserDbAndPartition(Partition part,
+      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
+      boolean[] inputCheck, boolean[] outputCheck) throws HiveException {
+
+    if (authorizeUserAndDBPriv(
+        hive_db.getDatabase(part.getTable().getDbName()), inputRequiredPriv,
+        outputRequiredPriv, inputCheck, outputCheck)) {
+      return true;
+    }
+
+    PrincipalPrivilegeSet partPrivileges = part.getTPartition().getPrivileges();
+    if (partPrivileges == null) {
+      partPrivileges = hive_db.get_privilege_set(HiveObjectType.PARTITION, part
+          .getTable().getDbName(), part.getTable().getTableName(), part
+          .getValues(), null, this.getAuthenticator().getUserName(), this
+          .getAuthenticator().getGroupNames());
+    }
+
+    if (authorizePrivileges(partPrivileges, inputRequiredPriv, inputCheck,
+        outputRequiredPriv, outputCheck)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  protected boolean authorizePrivileges(PrincipalPrivilegeSet privileges,
+      Privilege[] inputPriv, boolean[] inputCheck, Privilege[] outputPriv,
+      boolean[] outputCheck) throws HiveException {
+
+    boolean pass = true;
+    if (inputPriv != null) {
+      pass = pass && matchPrivs(inputPriv, privileges, inputCheck);
+    }
+    if (outputPriv != null) {
+      pass = pass && matchPrivs(outputPriv, privileges, outputCheck);
+    }
+    return pass;
+  }
+
+  /**
+   * try to match an array of privileges from user/groups/roles grants.
+   *
+   * @param container
+   */
+  private boolean matchPrivs(Privilege[] inputPriv,
+      PrincipalPrivilegeSet privileges, boolean[] check) {
+
+    if (inputPriv == null) {
+      return true;
+    }
+
+    if (privileges == null) {
+      return false;
+    }
+
+    /*
+     * user grants
+     */
+    Set<String> privSet = new HashSet<String>();
+    if (privileges.getUserPrivileges() != null
+        && privileges.getUserPrivileges().size() > 0) {
+      Collection<List<PrivilegeGrantInfo>> privCollection = privileges.getUserPrivileges().values();
+
+      List<String> userPrivs = getPrivilegeStringList(privCollection);
+      if (userPrivs != null && userPrivs.size() > 0) {
+        for (String priv : userPrivs) {
+          if (priv == null || priv.trim().equals("")) {
+            continue;
+          }
+            if (priv.equalsIgnoreCase(Privilege.ALL.toString())) {
+              setBooleanArray(check, true);
+              return true;
+            }
+            privSet.add(priv.toLowerCase());
+        }
+      }
+    }
+
+    /*
+     * group grants
+     */
+    if (privileges.getGroupPrivileges() != null
+        && privileges.getGroupPrivileges().size() > 0) {
+      Collection<List<PrivilegeGrantInfo>> groupPrivCollection = privileges
+          .getGroupPrivileges().values();
+      List<String> groupPrivs = getPrivilegeStringList(groupPrivCollection);
+      if (groupPrivs != null && groupPrivs.size() > 0) {
+        for (String priv : groupPrivs) {
+          if (priv == null || priv.trim().equals("")) {
+            continue;
+          }
+          if (priv.equalsIgnoreCase(Privilege.ALL.toString())) {
+            setBooleanArray(check, true);
+            return true;
+          }
+          privSet.add(priv.toLowerCase());
+        }
+      }
+    }
+
+    /*
+     * roles grants
+     */
+    if (privileges.getRolePrivileges() != null
+        && privileges.getRolePrivileges().size() > 0) {
+      Collection<List<PrivilegeGrantInfo>> rolePrivsCollection = privileges
+          .getRolePrivileges().values();
+      ;
+      List<String> rolePrivs = getPrivilegeStringList(rolePrivsCollection);
+      if (rolePrivs != null && rolePrivs.size() > 0) {
+        for (String priv : rolePrivs) {
+          if (priv == null || priv.trim().equals("")) {
+            continue;
+          }
+          if (priv.equalsIgnoreCase(Privilege.ALL.toString())) {
+            setBooleanArray(check, true);
+            return true;
+          }
+          privSet.add(priv.toLowerCase());
+        }
+      }
+    }
+
+    for (int i = 0; i < inputPriv.length; i++) {
+      String toMatch = inputPriv[i].toString();
+      if (!check[i]) {
+        check[i] = privSet.contains(toMatch.toLowerCase());
+      }
+    }
+
+    return firstFalseIndex(check) <0;
+  }
+
+  private List<String> getPrivilegeStringList(
+      Collection<List<PrivilegeGrantInfo>> privCollection) {
+    List<String> userPrivs = new ArrayList<String>();
+    if (privCollection!= null && privCollection.size()>0) {
+      for (List<PrivilegeGrantInfo> grantList : privCollection) {
+        if (grantList == null){
+          continue;
+        }
+        for (int i = 0; i < grantList.size(); i++) {
+          PrivilegeGrantInfo grant = grantList.get(i);
+          userPrivs.add(grant.getPrivilege());
+        }
+      }
+    }
+    return userPrivs;
+  }
+
+  private static void setBooleanArray(boolean[] check, boolean b) {
+    for (int i = 0; i < check.length; i++) {
+      check[i] = b;
+    }
+  }
+
+  private static void booleanArrayOr(boolean[] output, boolean[] input) {
+    for (int i = 0; i < output.length && i < input.length; i++) {
+      output[i] = output[i] || input[i];
+    }
+  }
+
+  private void checkAndThrowAuthorizationException(
+      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
+      boolean[] inputCheck, boolean[] outputCheck,String dbName,
+      String tableName, String partitionName, String columnName) {
+
+    String hiveObject = "{ ";
+    if (dbName != null) {
+      hiveObject = hiveObject + "database:" + dbName;
+    }
+    if (tableName != null) {
+      hiveObject = hiveObject + ", table:" + tableName;
+    }
+    if (partitionName != null) {
+      hiveObject = hiveObject + ", partitionName:" + partitionName;
+    }
+    if (columnName != null) {
+      hiveObject = hiveObject + ", columnName:" + columnName;
+    }
+    hiveObject = hiveObject + "}";
+
+    if (inputCheck != null) {
+      int input = this.firstFalseIndex(inputCheck);
+      if (input >= 0) {
+        throw new AuthorizationException("No privilege '"
+            + inputRequiredPriv[input].toString() + "' found for inputs "
+            + hiveObject);
+      }
+    }
+
+    if (outputCheck != null) {
+      int output = this.firstFalseIndex(outputCheck);
+      if (output >= 0) {
+        throw new AuthorizationException("No privilege '"
+            + outputRequiredPriv[output].toString() + "' found for outputs "
+            + hiveObject);
+      }
+    }
+  }
+
+  private int firstFalseIndex(boolean[] inputCheck) {
+    if (inputCheck != null) {
+      for (int i = 0; i < inputCheck.length; i++) {
+        if (!inputCheck[i]) {
+          return i;
+        }
+      }
+    }
+    return -1;
+  }
+}

Modified: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java (original)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -18,478 +18,16 @@
 
 package org.apache.hadoop.hive.ql.security.authorization;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.HiveObjectType;
-import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
-import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
-import org.apache.hadoop.hive.ql.metadata.AuthorizationException;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
+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;
 
 public class DefaultHiveAuthorizationProvider extends
-    HiveAuthorizationProviderBase {
-
-  static class BitSetChecker {
-
-    boolean[] inputCheck = null;
-    boolean[] outputCheck = null;
-
-    public static BitSetChecker getBitSetChecker(Privilege[] inputRequiredPriv,
-        Privilege[] outputRequiredPriv) {
-      BitSetChecker checker = new BitSetChecker();
-      if (inputRequiredPriv != null) {
-        checker.inputCheck = new boolean[inputRequiredPriv.length];
-        for (int i = 0; i < checker.inputCheck.length; i++) {
-          checker.inputCheck[i] = false;
-        }
-      }
-      if (outputRequiredPriv != null) {
-        checker.outputCheck = new boolean[outputRequiredPriv.length];
-        for (int i = 0; i < checker.outputCheck.length; i++) {
-          checker.outputCheck[i] = false;
-        }
-      }
-
-      return checker;
-    }
-
-  }
-
-  @Override
-  public void authorize(Privilege[] inputRequiredPriv,
-      Privilege[] outputRequiredPriv) throws HiveException, AuthorizationException {
-
-    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
-        outputRequiredPriv);
-    boolean[] inputCheck = checker.inputCheck;
-    boolean[] outputCheck = checker.outputCheck;
-
-    authorizeUserPriv(inputRequiredPriv, inputCheck, outputRequiredPriv,
-        outputCheck);
-    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
-        inputCheck, outputCheck, null, null, null, null);
-  }
-  
-  @Override
-  public void authorize(Database db, Privilege[] inputRequiredPriv,
-      Privilege[] outputRequiredPriv) throws HiveException, AuthorizationException {
-
-    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
-        outputRequiredPriv);
-    boolean[] inputCheck = checker.inputCheck;
-    boolean[] outputCheck = checker.outputCheck;
-
-    authorizeUserAndDBPriv(db, inputRequiredPriv, outputRequiredPriv,
-        inputCheck, outputCheck);
-    
-    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
-        inputCheck, outputCheck, db.getName(), null, null, null);
-  }
-  
-  @Override
-  public void authorize(Table table, Privilege[] inputRequiredPriv,
-      Privilege[] outputRequiredPriv) throws HiveException {
-    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
-        outputRequiredPriv);
-    boolean[] inputCheck = checker.inputCheck;
-    boolean[] outputCheck = checker.outputCheck;
-
-    authorizeUserDBAndTable(table, inputRequiredPriv,
-        outputRequiredPriv, inputCheck, outputCheck);
-    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
-        inputCheck, outputCheck, table.getDbName(), table.getTableName(),
-        null, null);
-  }
-  
-  @Override
-  public void authorize(Partition part, Privilege[] inputRequiredPriv,
-      Privilege[] outputRequiredPriv) throws HiveException {
-
-    //if the partition does not have partition level privilege, go to table level.
-    Table table = part.getTable();
-    if (table.getParameters().get("PARTITION_LEVEL_PRIVILEGE") == null || ("FALSE"
-        .equalsIgnoreCase(table.getParameters().get(
-            "PARTITION_LEVEL_PRIVILEGE")))) {
-      this.authorize(part.getTable(), inputRequiredPriv, outputRequiredPriv);
-      return;
-    }
-    
-    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
-        outputRequiredPriv);
-    boolean[] inputCheck = checker.inputCheck;
-    boolean[] outputCheck = checker.outputCheck;
-    
-    if (authorizeUserDbAndPartition(part, inputRequiredPriv, outputRequiredPriv,
-        inputCheck, outputCheck)){
-      return;
-    }
-
-    checkAndThrowAuthorizationException(inputRequiredPriv, outputRequiredPriv,
-        inputCheck, outputCheck, part.getTable().getDbName(), part
-            .getTable().getTableName(), part.getName(), null);
-  }
-
-  @Override
-  public void authorize(Table table, Partition part, List<String> columns,
-      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv)
-      throws HiveException {
-    BitSetChecker checker = BitSetChecker.getBitSetChecker(inputRequiredPriv,
-        outputRequiredPriv);
-    boolean[] inputCheck = checker.inputCheck;
-    boolean[] outputCheck = checker.outputCheck;
-
-    String partName = null;
-    List<String> partValues = null;
-    if (part != null
-        && (table.getParameters().get("PARTITION_LEVEL_PRIVILEGE") != null && ("TRUE"
-            .equalsIgnoreCase(table.getParameters().get(
-                "PARTITION_LEVEL_PRIVILEGE"))))) {
-      partName = part.getName();
-      partValues = part.getValues();
-    }
-
-    if (partValues == null) {
-      if (authorizeUserDBAndTable(table, inputRequiredPriv, outputRequiredPriv,
-          inputCheck, outputCheck)) {
-        return;
-      }
-    } else {
-      if (authorizeUserDbAndPartition(part, inputRequiredPriv,
-          outputRequiredPriv, inputCheck, outputCheck)) {
-        return;
-      }
-    }
-
-    for (String col : columns) {
-      
-      BitSetChecker checker2 = BitSetChecker.getBitSetChecker(
-          inputRequiredPriv, outputRequiredPriv);
-      boolean[] inputCheck2 = checker2.inputCheck;
-      boolean[] outputCheck2 = checker2.outputCheck;
-
-      PrincipalPrivilegeSet partColumnPrivileges = hive_db
-          .get_privilege_set(HiveObjectType.COLUMN, table.getDbName(), table.getTableName(),
-              partValues, col, this.getAuthenticator().getUserName(), this
-                  .getAuthenticator().getGroupNames());
-      
-      authorizePrivileges(partColumnPrivileges, inputRequiredPriv, inputCheck2,
-          outputRequiredPriv, outputCheck2);
-  
-      if (inputCheck2 != null) {
-        booleanArrayOr(inputCheck2, inputCheck);
-      }
-      if (outputCheck2 != null) {
-        booleanArrayOr(inputCheck2, inputCheck);
-      }
-      
-      checkAndThrowAuthorizationException(inputRequiredPriv,
-          outputRequiredPriv, inputCheck2, outputCheck2, table.getDbName(),
-          table.getTableName(), partName, col);
-    }
-  }
-
-  protected boolean authorizeUserPriv(Privilege[] inputRequiredPriv,
-      boolean[] inputCheck, Privilege[] outputRequiredPriv,
-      boolean[] outputCheck) throws HiveException {
-    PrincipalPrivilegeSet privileges = hive_db.get_privilege_set(
-        HiveObjectType.GLOBAL, null, null, null, null, this.getAuthenticator()
-            .getUserName(), this.getAuthenticator().getGroupNames());
-    return authorizePrivileges(privileges, inputRequiredPriv, inputCheck,
-        outputRequiredPriv, outputCheck);
-  }
-
-  /**
-   * Check privileges on User and DB. This is used before doing a check on
-   * table/partition objects, first check the user and DB privileges. If it
-   * passed on this check, no need to check against the table/partition hive
-   * object.
-   * 
-   * @param db
-   * @param inputRequiredPriv
-   * @param outputRequiredPriv
-   * @param inputCheck
-   * @param outputCheck
-   * @return true if the check on user and DB privilege passed, which means no
-   *         need for privilege check on concrete hive objects.
-   * @throws HiveException
-   */
-  private boolean authorizeUserAndDBPriv(Database db,
-      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
-      boolean[] inputCheck, boolean[] outputCheck) throws HiveException {
-    if (authorizeUserPriv(inputRequiredPriv, inputCheck, outputRequiredPriv,
-        outputCheck)) {
-      return true;
-    }
-
-    PrincipalPrivilegeSet dbPrivileges = hive_db.get_privilege_set(
-        HiveObjectType.DATABASE, db.getName(), null, null, null, this
-            .getAuthenticator().getUserName(), this.getAuthenticator()
-            .getGroupNames());
-
-    if (authorizePrivileges(dbPrivileges, inputRequiredPriv, inputCheck,
-        outputRequiredPriv, outputCheck)) {
-      return true;
-    }
-
-    return false;
-  }
-
-  /**
-   * Check privileges on User, DB and table objects.
-   * 
-   * @param table
-   * @param inputRequiredPriv
-   * @param outputRequiredPriv
-   * @param inputCheck
-   * @param outputCheck
-   * @return true if the check passed
-   * @throws HiveException
-   */
-  private boolean authorizeUserDBAndTable(Table table,
-      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
-      boolean[] inputCheck, boolean[] outputCheck) throws HiveException {
-    
-    if (authorizeUserAndDBPriv(hive_db.getDatabase(table.getDbName()),
-        inputRequiredPriv, outputRequiredPriv, inputCheck, outputCheck)) {
-      return true;
-    }
-
-    PrincipalPrivilegeSet tablePrivileges = hive_db.get_privilege_set(
-        HiveObjectType.TABLE, table.getDbName(), table.getTableName(), null,
-        null, this.getAuthenticator().getUserName(), this.getAuthenticator()
-            .getGroupNames());
-
-    if (authorizePrivileges(tablePrivileges, inputRequiredPriv, inputCheck,
-        outputRequiredPriv, outputCheck)) {
-      return true;
-    }
-
-    return false;
-  }
-  
-  /**
-   * Check privileges on User, DB and table/Partition objects.
-   * 
-   * @param part
-   * @param inputRequiredPriv
-   * @param outputRequiredPriv
-   * @param inputCheck
-   * @param outputCheck
-   * @return true if the check passed
-   * @throws HiveException
-   */
-  private boolean authorizeUserDbAndPartition(Partition part,
-      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
-      boolean[] inputCheck, boolean[] outputCheck) throws HiveException {
-
-    if (authorizeUserAndDBPriv(
-        hive_db.getDatabase(part.getTable().getDbName()), inputRequiredPriv,
-        outputRequiredPriv, inputCheck, outputCheck)) {
-      return true;
-    }
-
-    PrincipalPrivilegeSet partPrivileges = part.getTPartition().getPrivileges();
-    if (partPrivileges == null) {
-      partPrivileges = hive_db.get_privilege_set(HiveObjectType.PARTITION, part
-          .getTable().getDbName(), part.getTable().getTableName(), part
-          .getValues(), null, this.getAuthenticator().getUserName(), this
-          .getAuthenticator().getGroupNames());
-    }
+    BitSetCheckedAuthorizationProvider {
 
-    if (authorizePrivileges(partPrivileges, inputRequiredPriv, inputCheck,
-        outputRequiredPriv, outputCheck)) {
-      return true;
-    }
-
-    return false;
-  }
-
-  protected boolean authorizePrivileges(PrincipalPrivilegeSet privileges,
-      Privilege[] inputPriv, boolean[] inputCheck, Privilege[] outputPriv,
-      boolean[] outputCheck) throws HiveException {
-
-    boolean pass = true;
-    if (inputPriv != null) {
-      pass = pass && matchPrivs(inputPriv, privileges, inputCheck);
-    }
-    if (outputPriv != null) {
-      pass = pass && matchPrivs(outputPriv, privileges, outputCheck);
-    }
-    return pass;
+  public void init(Configuration conf) throws HiveException {
+    hive_db = new HiveProxy(Hive.get(new HiveConf(conf, HiveAuthorizationProvider.class)));
   }
 
-  /**
-   * try to match an array of privileges from user/groups/roles grants.
-   * 
-   * @param container
-   */
-  private boolean matchPrivs(Privilege[] inputPriv,
-      PrincipalPrivilegeSet privileges, boolean[] check) {
-
-    if (inputPriv == null)
-      return true;
-    
-    if (privileges == null)
-      return false;
-
-    /*
-     * user grants
-     */
-    Set<String> privSet = new HashSet<String>();
-    if (privileges.getUserPrivileges() != null
-        && privileges.getUserPrivileges().size() > 0) {
-      Collection<List<PrivilegeGrantInfo>> privCollection = privileges.getUserPrivileges().values();
-      
-      List<String> userPrivs = getPrivilegeStringList(privCollection);
-      if (userPrivs != null && userPrivs.size() > 0) {
-        for (String priv : userPrivs) {
-          if (priv == null || priv.trim().equals(""))
-            continue;
-            if (priv.equalsIgnoreCase(Privilege.ALL.toString())) {
-              setBooleanArray(check, true);
-              return true;
-            }
-            privSet.add(priv.toLowerCase());
-        }
-      }
-    }
-
-    /*
-     * group grants
-     */
-    if (privileges.getGroupPrivileges() != null
-        && privileges.getGroupPrivileges().size() > 0) {
-      Collection<List<PrivilegeGrantInfo>> groupPrivCollection = privileges
-          .getGroupPrivileges().values();
-      List<String> groupPrivs = getPrivilegeStringList(groupPrivCollection);
-      if (groupPrivs != null && groupPrivs.size() > 0) {
-        for (String priv : groupPrivs) {
-          if (priv == null || priv.trim().equals(""))
-            continue;
-          if (priv.equalsIgnoreCase(Privilege.ALL.toString())) {
-            setBooleanArray(check, true);
-            return true;
-          }
-          privSet.add(priv.toLowerCase());
-        }
-      }
-    }
-
-    /*
-     * roles grants
-     */
-    if (privileges.getRolePrivileges() != null
-        && privileges.getRolePrivileges().size() > 0) {
-      Collection<List<PrivilegeGrantInfo>> rolePrivsCollection = privileges
-          .getRolePrivileges().values();
-      ;
-      List<String> rolePrivs = getPrivilegeStringList(rolePrivsCollection);
-      if (rolePrivs != null && rolePrivs.size() > 0) {
-        for (String priv : rolePrivs) {
-          if (priv == null || priv.trim().equals(""))
-            continue;
-          if (priv.equalsIgnoreCase(Privilege.ALL.toString())) {
-            setBooleanArray(check, true);
-            return true;
-          }
-          privSet.add(priv.toLowerCase());
-        }
-      }
-    }
-
-    for (int i = 0; i < inputPriv.length; i++) {
-      String toMatch = inputPriv[i].toString();
-      if (!check[i]) {
-        check[i] = privSet.contains(toMatch.toLowerCase());
-      }
-    }
-
-    return firstFalseIndex(check) <0;
-  }
-
-  private List<String> getPrivilegeStringList(
-      Collection<List<PrivilegeGrantInfo>> privCollection) {
-    List<String> userPrivs = new ArrayList<String>();
-    if (privCollection!= null && privCollection.size()>0) {
-      for (List<PrivilegeGrantInfo> grantList : privCollection) {
-        if (grantList == null){
-          continue;
-        }
-        for (int i = 0; i < grantList.size(); i++) {
-          PrivilegeGrantInfo grant = grantList.get(i);
-          userPrivs.add(grant.getPrivilege());
-        }
-      }        
-    }
-    return userPrivs;
-  }
-
-  private static void setBooleanArray(boolean[] check, boolean b) {
-    for (int i = 0; i < check.length; i++) {
-      check[i] = b;
-    }
-  }
-
-  private static void booleanArrayOr(boolean[] output, boolean[] input) {
-    for (int i = 0; i < output.length && i < input.length; i++) {
-      output[i] = output[i] || input[i];
-    }
-  }
-  
-  private void checkAndThrowAuthorizationException(
-      Privilege[] inputRequiredPriv, Privilege[] outputRequiredPriv,
-      boolean[] inputCheck, boolean[] outputCheck,String dbName,
-      String tableName, String partitionName, String columnName) {
-
-    String hiveObject = "{ ";
-    if (dbName != null) {
-      hiveObject = hiveObject + "database:" + dbName;
-    }
-    if (tableName != null) {
-      hiveObject = hiveObject + ", table:" + tableName;
-    }
-    if (partitionName != null) {
-      hiveObject = hiveObject + ", partitionName:" + partitionName;
-    }
-    if (columnName != null) {
-      hiveObject = hiveObject + ", columnName:" + columnName;
-    }
-    hiveObject = hiveObject + "}";
-
-    if (inputCheck != null) {
-      int input = this.firstFalseIndex(inputCheck);
-      if (input >= 0) {
-        throw new AuthorizationException("No privilege '"
-            + inputRequiredPriv[input].toString() + "' found for inputs "
-            + hiveObject);
-      }
-    }
-
-    if (outputCheck != null) {
-      int output = this.firstFalseIndex(outputCheck);
-      if (output >= 0) {
-        throw new AuthorizationException("No privilege '"
-            + outputRequiredPriv[output].toString() + "' found for outputs "
-            + hiveObject);
-      }
-    }
-  }
-
-  private int firstFalseIndex(boolean[] inputCheck) {
-    if (inputCheck != null) {
-      for (int i = 0; i < inputCheck.length; i++) {
-        if (!inputCheck[i]) {
-          return i;
-        }
-      }
-    }
-    return -1;
-  }
 }

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveMetastoreAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,39 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security.authorization;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+public class DefaultHiveMetastoreAuthorizationProvider extends BitSetCheckedAuthorizationProvider
+  implements HiveMetastoreAuthorizationProvider {
+
+  @Override
+  public void init(Configuration conf) throws HiveException {
+    hive_db = new HiveProxy();
+  }
+
+  @Override
+  public void setMetaStoreHandler(HMSHandler handler) {
+    hive_db.setHandler(handler);
+  }
+
+
+}

Modified: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java?rev=1420483&r1=1420482&r2=1420483&view=diff
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java (original)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java Tue Dec 11 23:29:37 2012
@@ -18,21 +18,90 @@
 
 package org.apache.hadoop.hive.ql.security.authorization;
 
+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.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+import org.apache.hadoop.hive.metastore.api.Database;
+import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
+import org.apache.hadoop.hive.metastore.api.HiveObjectType;
+import org.apache.hadoop.hive.metastore.api.MetaException;
+import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
+import org.apache.hadoop.hive.metastore.api.PrincipalPrivilegeSet;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.thrift.TException;
 
 public abstract class HiveAuthorizationProviderBase implements
     HiveAuthorizationProvider {
-  
+
+  protected class HiveProxy {
+
+    private final Hive hiveClient;
+    private HMSHandler handler;
+
+    public HiveProxy(Hive hive) {
+      this.hiveClient = hive;
+      this.handler = null;
+    }
+
+    public HiveProxy() {
+      this.hiveClient = null;
+      this.handler = null;
+    }
+
+    public void setHandler(HMSHandler handler){
+      this.handler = handler;
+    }
+
+    public PrincipalPrivilegeSet get_privilege_set(HiveObjectType column, String dbName,
+        String tableName, List<String> partValues, String col, String userName,
+        List<String> groupNames) throws HiveException {
+      if (hiveClient != null) {
+        return hiveClient.get_privilege_set(
+            column, dbName, tableName, partValues, col, userName, groupNames);
+      } else {
+        HiveObjectRef hiveObj = new HiveObjectRef(column, dbName,
+            tableName, partValues, col);
+        try {
+          return handler.get_privilege_set(hiveObj, userName, groupNames);
+        } catch (MetaException e) {
+          throw new HiveException(e);
+        } catch (TException e) {
+          throw new HiveException(e);
+        }
+      }
+    }
+
+    public Database getDatabase(String dbName) throws HiveException {
+      if (hiveClient != null) {
+        return hiveClient.getDatabase(dbName);
+      } else {
+        try {
+          return handler.get_database(dbName);
+        } catch (NoSuchObjectException e) {
+          throw new HiveException(e);
+        } catch (MetaException e) {
+          throw new HiveException(e);
+        }
+      }
+    }
+
+  }
+
+  protected HiveProxy hive_db;
+
   protected HiveAuthenticationProvider authenticator;
 
-  protected Hive hive_db;
-  
   private Configuration conf;
 
+  public static final Log LOG = LogFactory.getLog(
+      HiveAuthorizationProvider.class);
+
+
   public void setConf(Configuration conf) {
     this.conf = conf;
     try {
@@ -42,10 +111,6 @@ public abstract class HiveAuthorizationP
     }
   }
 
-  public void init(Configuration conf) throws HiveException {
-    hive_db = Hive.get(new HiveConf(conf, HiveAuthorizationProvider.class));
-  }
-  
   public Configuration getConf() {
     return this.conf;
   }

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveMetastoreAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,40 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security.authorization;
+
+import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
+
+/**
+ * HiveMetastoreAuthorizationProvider : An extension of HiveAuthorizaytionProvider
+ * that is intended to be called from the metastore-side. It will be invoked
+ * by AuthorizationPreEventListener.
+ *
+ */
+public interface HiveMetastoreAuthorizationProvider extends HiveAuthorizationProvider {
+
+  /**
+   * Allows invoker of HiveMetaStoreAuthorizationProvider to send in a
+   * hive metastore handler that can be used to make calls to test
+   * whether or not authorizations can/will succeed. Intended to be called
+   * before any of the authorize methods are called.
+   * @param handler
+   */
+  void setMetaStoreHandler(HMSHandler handler);
+
+}

Added: hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java?rev=1420483&view=auto
==============================================================================
--- hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java (added)
+++ hive/branches/branch-0.10/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java Tue Dec 11 23:29:37 2012
@@ -0,0 +1,338 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security.authorization;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.security.AccessControlException;
+import java.util.EnumSet;
+import java.util.List;
+
+import javax.security.auth.login.LoginException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+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.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.HiveException;
+import org.apache.hadoop.hive.ql.metadata.Partition;
+import org.apache.hadoop.hive.ql.metadata.Table;
+
+/**
+ * StorageBasedAuthorizationProvider is an implementation of
+ * HiveMetastoreAuthorizationProvider that tries to look at the hdfs
+ * permissions of files and directories associated with objects like
+ * databases, tables and partitions to determine whether or not an
+ * operation is allowed. The rule of thumb for which location to check
+ * in hdfs is as follows:
+ *
+ * CREATE : on location specified, or on location determined from metadata
+ * READS : not checked (the preeventlistener does not have an event to fire)
+ * UPDATES : on location in metadata
+ * DELETES : on location in metadata
+ *
+ * If the location does not yet exist, as the case is with creates, it steps
+ * out to the parent directory recursively to determine its permissions till
+ * it finds a parent that does exist.
+ */
+public class StorageBasedAuthorizationProvider extends HiveAuthorizationProviderBase
+    implements HiveMetastoreAuthorizationProvider {
+
+  private Warehouse wh;
+
+  @Override
+  public void init(Configuration conf) throws HiveException {
+    hive_db = new HiveProxy();
+  }
+
+  @Override
+  public void authorize(Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    // Currently not used in hive code-base, but intended to authorize actions
+    // that are directly user-level. As there's no storage based aspect to this,
+    // we can follow one of two routes:
+    // a) We can allow by default - that way, this call stays out of the way
+    // b) We can deny by default - that way, no privileges are authorized that
+    // is not understood and explicitly allowed.
+    // Both approaches have merit, but given that things like grants and revokes
+    // that are user-level do not make sense from the context of storage-permission
+    // based auth, denying seems to be more canonical here.
+
+    throw new AuthorizationException(StorageBasedAuthorizationProvider.class.getName() +
+        " does not allow user-level authorization");
+  }
+
+  @Override
+  public void authorize(Database db, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    Path path = getDbLocation(db);
+    authorize(path, readRequiredPriv, writeRequiredPriv);
+  }
+
+  @Override
+  public void authorize(Table table, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+
+    // Table path can be null in the case of a new create table - in this case,
+    // we try to determine what the path would be after the create table is issued.
+    Path path = null;
+    try {
+      String location = table.getTTable().getSd().getLocation();
+      if (location == null || location.isEmpty()) {
+        path = wh.getTablePath(hive_db.getDatabase(table.getDbName()), table.getTableName());
+      } else {
+        path = new Path(location);
+      }
+    } catch (MetaException ex) {
+      throw hiveException(ex);
+    }
+
+    authorize(path, readRequiredPriv, writeRequiredPriv);
+  }
+
+  @Override
+  public void authorize(Partition part, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    authorize(part.getTable(), part, readRequiredPriv, writeRequiredPriv);
+  }
+
+  private void authorize(Table table, Partition part, Privilege[] readRequiredPriv,
+      Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+
+    // Partition path can be null in the case of a new create partition - in this case,
+    // we try to default to checking the permissions of the parent table
+    if (part.getLocation() == null) {
+      authorize(table, readRequiredPriv, writeRequiredPriv);
+    } else {
+      authorize(part.getPartitionPath(), readRequiredPriv, writeRequiredPriv);
+    }
+  }
+
+  @Override
+  public void authorize(Table table, Partition part, List<String> columns,
+      Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv) throws HiveException,
+      AuthorizationException {
+    // In a simple storage-based auth, we have no information about columns
+    // living in different files, so we do simple partition-auth and ignore
+    // the columns parameter.
+
+    authorize(part.getTable(), part, readRequiredPriv, writeRequiredPriv);
+  }
+
+  @Override
+  public void setMetaStoreHandler(HMSHandler handler) {
+    hive_db.setHandler(handler);
+    this.wh = handler.getWh();
+  }
+
+  /**
+   * Given a privilege, return what FsActions are required
+   */
+  protected FsAction getFsAction(Privilege priv) {
+
+    switch (priv.getPriv()) {
+    case ALL:
+      return FsAction.READ_WRITE;
+    case ALTER_DATA:
+      return FsAction.WRITE;
+    case ALTER_METADATA:
+      return FsAction.WRITE;
+    case CREATE:
+      return FsAction.WRITE;
+    case DROP:
+      return FsAction.WRITE;
+    case INDEX:
+      throw new AuthorizationException(
+          "StorageBasedAuthorizationProvider cannot handle INDEX privilege");
+    case LOCK:
+      throw new AuthorizationException(
+          "StorageBasedAuthorizationProvider cannot handle LOCK privilege");
+    case SELECT:
+      return FsAction.READ;
+    case SHOW_DATABASE:
+      return FsAction.READ;
+    case UNKNOWN:
+    default:
+      throw new AuthorizationException("Unknown privilege");
+    }
+  }
+
+  /**
+   * Given a Privilege[], find out what all FsActions are required
+   */
+  protected EnumSet<FsAction> getFsActions(Privilege[] privs) {
+    EnumSet<FsAction> actions = EnumSet.noneOf(FsAction.class);
+
+    if (privs == null) {
+      return actions;
+    }
+
+    for (Privilege priv : privs) {
+      actions.add(getFsAction(priv));
+    }
+
+    return actions;
+  }
+
+  /**
+   * Authorization privileges against a path.
+   *
+   * @param path
+   *          a filesystem path
+   * @param readRequiredPriv
+   *          a list of privileges needed for inputs.
+   * @param writeRequiredPriv
+   *          a list of privileges needed for outputs.
+   */
+  public void authorize(Path path, Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException {
+    try {
+      EnumSet<FsAction> actions = getFsActions(readRequiredPriv);
+      actions.addAll(getFsActions(writeRequiredPriv));
+      if (actions.isEmpty()) {
+        return;
+      }
+
+      checkPermissions(getConf(), path, actions);
+
+    } catch (AccessControlException ex) {
+      throw authorizationException(ex);
+    } catch (LoginException ex) {
+      throw authorizationException(ex);
+    } catch (IOException ex) {
+      throw hiveException(ex);
+    }
+  }
+
+
+  /**
+   * Checks the permissions for the given path and current user on Hadoop FS.
+   * If the given path does not exists, it checks for its parent folder.
+   */
+  protected void checkPermissions(final Configuration conf, final Path path,
+      final EnumSet<FsAction> actions) throws IOException, LoginException {
+
+    if (path == null) {
+      throw new IllegalArgumentException("path is null");
+    }
+
+    final FileSystem fs = path.getFileSystem(conf);
+
+    if (fs.exists(path)) {
+      checkPermissions(fs, path, actions,
+          authenticator.getUserName(), authenticator.getGroupNames());
+    } else if (path.getParent() != null) {
+      // find the ancestor which exists to check its permissions
+      Path par = path.getParent();
+      while (par != null) {
+        if (fs.exists(par)) {
+          break;
+        }
+        par = par.getParent();
+      }
+
+      checkPermissions(fs, par, actions,
+          authenticator.getUserName(), authenticator.getGroupNames());
+    }
+  }
+
+  /**
+   * Checks the permissions for the given path and current user on Hadoop FS. If the given path
+   * does not exists, it returns.
+   */
+  @SuppressWarnings("deprecation")
+  protected static void checkPermissions(final FileSystem fs, final Path path,
+      final EnumSet<FsAction> actions, String user, List<String> groups) throws IOException,
+      AccessControlException {
+
+    final FileStatus stat;
+
+    try {
+      stat = fs.getFileStatus(path);
+    } catch (FileNotFoundException fnfe) {
+      // File named by path doesn't exist; nothing to validate.
+      return;
+    } catch (org.apache.hadoop.fs.permission.AccessControlException ace) {
+      // Older hadoop version will throw this @deprecated Exception.
+      throw accessControlException(ace);
+    }
+
+    final FsPermission dirPerms = stat.getPermission();
+    final String grp = stat.getGroup();
+
+    for (FsAction action : actions) {
+      if (user.equals(stat.getOwner())) {
+        if (dirPerms.getUserAction().implies(action)) {
+          continue;
+        }
+      }
+      if (groups.contains(grp)) {
+        if (dirPerms.getGroupAction().implies(action)) {
+          continue;
+        }
+      }
+      if (dirPerms.getOtherAction().implies(action)) {
+        continue;
+      }
+      throw new AccessControlException("action " + action + " not permitted on path "
+          + path + " for user " + user);
+    }
+  }
+
+  protected Path getDbLocation(Database db) throws HiveException {
+    try {
+      String location = db.getLocationUri();
+      if (location == null) {
+        return wh.getDefaultDatabasePath(db.getName());
+      } else {
+        return wh.getDnsPath(wh.getDatabasePath(db));
+      }
+    } catch (MetaException ex) {
+      throw hiveException(ex);
+    }
+  }
+
+  private HiveException hiveException(Exception e) {
+    HiveException ex = new HiveException(e);
+    ex.initCause(e);
+    return ex;
+  }
+
+  private AuthorizationException authorizationException(Exception e) {
+    AuthorizationException ex = new AuthorizationException(e);
+    ex.initCause(e);
+    return ex;
+  }
+
+  private static AccessControlException accessControlException(
+      org.apache.hadoop.fs.permission.AccessControlException e) {
+    AccessControlException ace = new AccessControlException(e.getMessage());
+    ace.initCause(e);
+    return ace;
+  }
+
+}