You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2011/01/12 07:58:10 UTC

svn commit: r1057999 [16/22] - in /hive/trunk: ./ common/src/java/org/apache/hadoop/hive/conf/ conf/ metastore/if/ metastore/src/gen/thrift/gen-cpp/ metastore/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/ metastore/src/gen/thrift/ge...

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/DefaultHiveAuthorizationProvider.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,495 @@
+/**
+ * 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 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());
+    }
+
+    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].getPriv();
+      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].getPriv() + "' found for inputs "
+            + hiveObject);
+      }
+    }
+
+    if (outputCheck != null) {
+      int output = this.firstFalseIndex(outputCheck);
+      if (output >= 0) {
+        throw new AuthorizationException("No privilege '"
+            + outputRequiredPriv[output].getPriv() + "' 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/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProvider.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProvider.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProvider.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProvider.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,132 @@
+/**
+ * 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.List;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+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.HiveAuthenticationProvider;
+
+/**
+ * Hive's pluggable authorization provider interface
+ */
+public interface HiveAuthorizationProvider extends Configurable{
+
+  public void init(Configuration conf) throws HiveException;
+
+  public HiveAuthenticationProvider getAuthenticator();
+
+  public void setAuthenticator(HiveAuthenticationProvider authenticator);
+
+  /**
+   * Authorization user level privileges.
+   * 
+   * @param readRequiredPriv
+   *          a list of privileges needed for inputs.
+   * @param writeRequiredPriv
+   *          a list of privileges needed for outputs.
+   * @return
+   * @throws HiveException
+   * @throws AuthorizationException
+   */
+  public void authorize(Privilege[] readRequiredPriv,
+      Privilege[] writeRequiredPriv) throws HiveException,
+      AuthorizationException;
+
+  /**
+   * Authorization privileges against a database object.
+   * 
+   * @param db
+   *          database
+   * @param readRequiredPriv
+   *          a list of privileges needed for inputs.
+   * @param writeRequiredPriv
+   *          a list of privileges needed for outputs.
+   * @return
+   * @throws HiveException
+   * @throws AuthorizationException
+   */
+  public void authorize(Database db, Privilege[] readRequiredPriv,
+      Privilege[] writeRequiredPriv) throws HiveException,
+      AuthorizationException;
+
+  /**
+   * Authorization privileges against a hive table object.
+   * 
+   * @param table
+   *          table object
+   * @param readRequiredPriv
+   *          a list of privileges needed for inputs.
+   * @param writeRequiredPriv
+   *          a list of privileges needed for outputs.
+   * @return
+   * @throws HiveException
+   * @throws AuthorizationException
+   */
+  public void authorize(Table table, Privilege[] readRequiredPriv,
+      Privilege[] writeRequiredPriv) throws HiveException,
+      AuthorizationException;
+
+  /**
+   * Authorization privileges against a hive partition object.
+   * 
+   * @param part
+   *          partition object
+   * @param readRequiredPriv
+   *          a list of privileges needed for inputs.
+   * @param writeRequiredPriv
+   *          a list of privileges needed for outputs.
+   * @return
+   * @throws HiveException
+   * @throws AuthorizationException
+   */
+  public void authorize(Partition part, Privilege[] readRequiredPriv,
+      Privilege[] writeRequiredPriv) throws HiveException,
+      AuthorizationException;
+
+  /**
+   * Authorization privileges against a list of columns. If the partition object
+   * is not null, look at the column grants for the given partition. Otherwise
+   * look at the table column grants.
+   * 
+   * @param table
+   *          table object
+   * @param part
+   *          partition object
+   * @param columns
+   *          a list of columns
+   * @param readRequiredPriv
+   *          a list of privileges needed for inputs.
+   * @param writeRequiredPriv
+   *          a list of privileges needed for outputs.
+   * @return
+   * @throws HiveException
+   * @throws AuthorizationException
+   */
+  public void authorize(Table table, Partition part, List<String> columns,
+      Privilege[] readRequiredPriv, Privilege[] writeRequiredPriv)
+      throws HiveException, AuthorizationException;
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/HiveAuthorizationProviderBase.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,61 @@
+/**
+ * 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.conf.HiveConf;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+
+public abstract class HiveAuthorizationProviderBase implements
+    HiveAuthorizationProvider {
+  
+  protected HiveAuthenticationProvider authenticator;
+
+  protected Hive hive_db;
+  
+  private Configuration conf;
+
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+    try {
+      init(conf);
+    } catch (HiveException e) {
+      throw new RuntimeException(e);
+    }
+  }
+
+  public void init(Configuration conf) throws HiveException {
+    hive_db = Hive.get(new HiveConf(conf, HiveAuthorizationProvider.class));
+  }
+  
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+  public HiveAuthenticationProvider getAuthenticator() {
+    return authenticator;
+  }
+
+  public void setAuthenticator(HiveAuthenticationProvider authenticator) {
+    this.authenticator = authenticator;
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/Privilege.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/Privilege.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/Privilege.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/Privilege.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,102 @@
+/**
+ * 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.EnumSet;
+
+/**
+ * Privilege defines a privilege in Hive. Each privilege has a name and scope associated with it.
+ * This class contains all of the predefined privileges in Hive.
+ */
+public class Privilege {
+  
+  private String priv;
+  
+  private EnumSet<PrivilegeScope> supportedScopeSet;
+  
+  private Privilege(String priv, EnumSet<PrivilegeScope> scopeSet) {
+    super();
+    this.priv = priv;
+    this.supportedScopeSet = scopeSet;
+  }
+
+  public Privilege(String priv) {
+    super();
+    this.priv = priv;
+    
+  }
+
+  public String getPriv() {
+    return priv;
+  }
+
+  public void setPriv(String priv) {
+    this.priv = priv;
+  }
+  
+  public boolean supportColumnLevel() {
+    return supportedScopeSet != null
+        && supportedScopeSet.contains(PrivilegeScope.COLUMN_LEVEL_SCOPE);
+  }
+
+  public boolean supportDBLevel() {
+    return supportedScopeSet != null
+        && supportedScopeSet.contains(PrivilegeScope.DB_LEVEL_SCOPE);
+  }
+
+  public boolean supportTableLevel() {
+    return supportedScopeSet != null
+        && supportedScopeSet.contains(PrivilegeScope.TABLE_LEVEL_SCOPE);
+  }
+  
+  public String toString() {
+    return this.priv;
+  }
+
+  public Privilege() {
+  }
+
+  public static Privilege ALL = new Privilege("All",
+      PrivilegeScope.ALLSCOPE_EXCEPT_COLUMN);
+
+  public static Privilege ALTER_METADATA = new Privilege("Alter",
+      PrivilegeScope.ALLSCOPE_EXCEPT_COLUMN);
+
+  public static Privilege ALTER_DATA = new Privilege("Update",
+      PrivilegeScope.ALLSCOPE_EXCEPT_COLUMN);
+
+  public static Privilege CREATE = new Privilege("Create",
+      PrivilegeScope.ALLSCOPE_EXCEPT_COLUMN);
+
+  public static Privilege DROP = new Privilege("Drop",
+      PrivilegeScope.ALLSCOPE_EXCEPT_COLUMN);
+
+  public static Privilege INDEX = new Privilege("Index",
+      PrivilegeScope.ALLSCOPE);
+
+  public static Privilege LOCK = new Privilege("Lock",
+      PrivilegeScope.ALLSCOPE_EXCEPT_COLUMN);
+
+  public static Privilege SELECT = new Privilege("Select",
+      PrivilegeScope.ALLSCOPE);
+
+  public static Privilege SHOW_DATABASE = new Privilege("Show_Database",
+      EnumSet.of(PrivilegeScope.USER_LEVEL_SCOPE));
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeRegistry.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeRegistry.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeRegistry.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeRegistry.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,51 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.security.authorization;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * PrivilegeRegistry is used to do privilege lookups. Given a privilege name, it
+ * will return the Privilege object.
+ */
+public class PrivilegeRegistry {
+
+  protected static Map<String, Privilege> Registry = new HashMap<String, Privilege>();
+
+  static {
+    Registry.put(Privilege.ALL.getPriv().toLowerCase(), Privilege.ALL);
+    Registry.put(Privilege.ALTER_DATA.getPriv().toLowerCase(),
+        Privilege.ALTER_DATA);
+    Registry.put(Privilege.ALTER_METADATA.getPriv().toLowerCase(),
+        Privilege.ALTER_METADATA);
+    Registry.put(Privilege.CREATE.getPriv().toLowerCase(), Privilege.CREATE);
+    Registry.put(Privilege.DROP.getPriv().toLowerCase(), Privilege.DROP);
+    Registry.put(Privilege.INDEX.getPriv().toLowerCase(), Privilege.INDEX);
+    Registry.put(Privilege.LOCK.getPriv().toLowerCase(), Privilege.LOCK);
+    Registry.put(Privilege.SELECT.getPriv().toLowerCase(), Privilege.SELECT);
+    Registry.put(Privilege.SHOW_DATABASE.getPriv().toLowerCase(),
+        Privilege.SHOW_DATABASE);
+  }
+
+  public static Privilege getPrivilege(String privilegeName) {
+    return Registry.get(privilegeName.toLowerCase());
+  }
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeScope.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeScope.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeScope.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/PrivilegeScope.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,57 @@
+/**
+ * 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.EnumSet;
+
+/**
+ * PrivilegeScope describes a hive defined privilege's scope
+ * (global/database/table/column). For example some hive privileges are
+ * db-level only, some are global, and some are table only.
+ */
+public enum PrivilegeScope {
+  
+  USER_LEVEL_SCOPE((short) 0x01), 
+  DB_LEVEL_SCOPE((short) 0x02), 
+  TABLE_LEVEL_SCOPE((short) 0x04), 
+  COLUMN_LEVEL_SCOPE((short) 0x08);
+
+  private short mode;
+
+  private PrivilegeScope(short mode) {
+    this.mode = mode;
+  }
+
+  public short getMode() {
+    return mode;
+  }
+
+  public void setMode(short mode) {
+    this.mode = mode;
+  }
+  
+  public static EnumSet<PrivilegeScope> ALLSCOPE = EnumSet.of(
+      PrivilegeScope.USER_LEVEL_SCOPE, PrivilegeScope.DB_LEVEL_SCOPE,
+      PrivilegeScope.TABLE_LEVEL_SCOPE, PrivilegeScope.COLUMN_LEVEL_SCOPE);
+
+  public static EnumSet<PrivilegeScope> ALLSCOPE_EXCEPT_COLUMN = EnumSet.of(
+      PrivilegeScope.USER_LEVEL_SCOPE, PrivilegeScope.DB_LEVEL_SCOPE,
+      PrivilegeScope.TABLE_LEVEL_SCOPE);
+
+}

Added: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/CreateTableAutomaticGrant.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/CreateTableAutomaticGrant.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/CreateTableAutomaticGrant.java (added)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/CreateTableAutomaticGrant.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,124 @@
+/**
+ * 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.session;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.security.authorization.Privilege;
+import org.apache.hadoop.hive.ql.security.authorization.PrivilegeRegistry;
+
+public class CreateTableAutomaticGrant {
+  private Map<String, List<PrivilegeGrantInfo>> userGrants;
+  private Map<String, List<PrivilegeGrantInfo>> groupGrants;
+  private Map<String, List<PrivilegeGrantInfo>> roleGrants;
+
+  public static CreateTableAutomaticGrant create(HiveConf conf)
+      throws HiveException {
+    CreateTableAutomaticGrant grants = new CreateTableAutomaticGrant();
+    grants.userGrants = getGrantMap(HiveConf.getVar(conf,
+        HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_USER_GRANTS));
+    grants.groupGrants = getGrantMap(HiveConf.getVar(conf,
+        HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_GROUP_GRANTS));
+    grants.roleGrants = getGrantMap(HiveConf.getVar(conf,
+        HiveConf.ConfVars.HIVE_AUTHORIZATION_TABLE_ROLE_GRANTS));
+    
+    List<PrivilegeGrantInfo> ownerGrantInfoList = new ArrayList<PrivilegeGrantInfo>();
+    String grantor = null;
+    if (SessionState.get() != null
+        && SessionState.get().getAuthenticator() != null) {
+      grantor = SessionState.get().getAuthenticator().getUserName();
+      ownerGrantInfoList.add(new PrivilegeGrantInfo(Privilege.ALL.getPriv(), -1, grantor,
+          PrincipalType.USER, true));
+      if (grants.userGrants == null) {
+        grants.userGrants = new HashMap<String, List<PrivilegeGrantInfo>>();
+      }
+      grants.userGrants.put(grantor, ownerGrantInfoList);
+    }
+    return grants;
+  }
+
+  private static Map<String, List<PrivilegeGrantInfo>> getGrantMap(String grantMapStr)
+      throws HiveException {
+    if (grantMapStr != null && !grantMapStr.trim().equals("")) {
+      String[] grantArrayStr = grantMapStr.split(";");
+      Map<String, List<PrivilegeGrantInfo>> grantsMap = new HashMap<String, List<PrivilegeGrantInfo>>();
+      for (String grantStr : grantArrayStr) {
+        String[] principalListAndPrivList = grantStr.split(":");
+        if (principalListAndPrivList.length != 2
+            || principalListAndPrivList[0] == null
+            || principalListAndPrivList[0].trim().equals("")) {
+          throw new HiveException(
+              "Can not understand the config privilege definition " + grantStr);
+        }
+        String userList = principalListAndPrivList[0];
+        String privList = principalListAndPrivList[1];
+        checkPrivilege(privList);
+        
+        String[] grantArray = privList.split(",");
+        List<PrivilegeGrantInfo> grantInfoList = new ArrayList<PrivilegeGrantInfo>();
+        String grantor = null;
+        if (SessionState.get().getAuthenticator() != null) {
+          grantor = SessionState.get().getAuthenticator().getUserName();  
+        }
+        for (String grant : grantArray) {
+          grantInfoList.add(new PrivilegeGrantInfo(grant, -1, grantor,
+              PrincipalType.USER, true));
+        }
+        
+        String[] users = userList.split(",");
+        for (String user : users) {
+          grantsMap.put(user, grantInfoList);
+        }
+      }
+      return grantsMap;
+    }
+    return null;
+  }
+
+  private static void checkPrivilege(String ownerGrantsInConfig)
+      throws HiveException {
+    String[] ownerGrantArray = ownerGrantsInConfig.split(",");
+    // verify the config
+    for (String ownerGrant : ownerGrantArray) {
+      Privilege prive = PrivilegeRegistry.getPrivilege(ownerGrant);
+      if (prive == null) {
+        throw new HiveException("Privilege " + ownerGrant + " is not found.");
+      }
+    }
+  }
+
+  public Map<String, List<PrivilegeGrantInfo>> getUserGrants() {
+    return userGrants;
+  }
+
+  public Map<String, List<PrivilegeGrantInfo>> getGroupGrants() {
+    return groupGrants;
+  }
+
+  public Map<String, List<PrivilegeGrantInfo>> getRoleGrants() {
+    return roleGrants;
+  }
+}
\ No newline at end of file

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java?rev=1057999&r1=1057998&r2=1057999&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/session/SessionState.java Wed Jan 12 06:58:04 2011
@@ -40,6 +40,11 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.Utilities;
 import org.apache.hadoop.hive.ql.history.HiveHistory;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+import org.apache.hadoop.hive.ql.metadata.HiveUtils;
+import org.apache.hadoop.hive.ql.plan.HiveOperation;
+import org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider;
+import org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider;
 import org.apache.hadoop.hive.ql.util.DosToUnix;
 import org.apache.log4j.LogManager;
 import org.apache.log4j.PropertyConfigurator;
@@ -77,8 +82,14 @@ public class SessionState {
   /**
    * type of the command.
    */
-  private String commandType;
-
+  private HiveOperation commandType;
+  
+  private HiveAuthorizationProvider authorizer;
+  
+  private HiveAuthenticationProvider authenticator;
+  
+  private CreateTableAutomaticGrant createTableGrants;
+  
   /**
    * Lineage state.
    */
@@ -150,11 +161,16 @@ public class SessionState {
 
   /**
    * start a new session and set it to current session.
+   * @throws HiveException 
    */
-  public static SessionState start(HiveConf conf) {
+  public static SessionState start(HiveConf conf) throws HiveException {
     SessionState ss = new SessionState(conf);
     ss.getConf().setVar(HiveConf.ConfVars.HIVESESSIONID, makeSessionId());
     ss.hiveHist = new HiveHistory(ss);
+    ss.authenticator = HiveUtils.getAuthenticator(conf);
+    ss.authorizer = HiveUtils.getAuthorizeProviderManager(
+        conf, ss.authenticator);
+    ss.createTableGrants = CreateTableAutomaticGrant.create(conf);
     tss.set(ss);
     return (ss);
   }
@@ -163,6 +179,7 @@ public class SessionState {
    * set current session to existing session object if a thread is running
    * multiple sessions - it must call this method with the new session object
    * when switching from one session to another.
+   * @throws HiveException 
    */
   public static SessionState start(SessionState startSs) {
 
@@ -176,6 +193,18 @@ public class SessionState {
     if (startSs.hiveHist == null) {
       startSs.hiveHist = new HiveHistory(startSs);
     }
+    
+    try {
+      startSs.authenticator = HiveUtils.getAuthenticator(startSs
+          .getConf());
+      startSs.authorizer = HiveUtils.getAuthorizeProviderManager(startSs
+          .getConf(), startSs.authenticator);
+      startSs.createTableGrants = CreateTableAutomaticGrant.create(startSs
+          .getConf());
+    } catch (HiveException e) {
+      throw new RuntimeException(e);
+    }
+    
     return startSs;
   }
 
@@ -539,10 +568,41 @@ public class SessionState {
   }
 
   public String getCommandType() {
+    if (commandType == null) {
+      return null;
+    }
+    return commandType.getOperationName();
+  }
+  
+  public HiveOperation getHiveOperation() {
     return commandType;
   }
 
-  public void setCommandType(String commandType) {
+  public void setCommandType(HiveOperation commandType) {
     this.commandType = commandType;
   }
+  
+  public HiveAuthorizationProvider getAuthorizer() {
+    return authorizer;
+  }
+
+  public void setAuthorizer(HiveAuthorizationProvider authorizer) {
+    this.authorizer = authorizer;
+  }
+
+  public HiveAuthenticationProvider getAuthenticator() {
+    return authenticator;
+  }
+
+  public void setAuthenticator(HiveAuthenticationProvider authenticator) {
+    this.authenticator = authenticator;
+  }
+  
+  public CreateTableAutomaticGrant getCreateTableGrants() {
+    return createTableGrants;
+  }
+
+  public void setCreateTableGrants(CreateTableAutomaticGrant createTableGrants) {
+    this.createTableGrants = createTableGrants;
+  }
 }

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1057999&r1=1057998&r2=1057999&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/QTestUtil.java Wed Jan 12 06:58:04 2011
@@ -24,10 +24,12 @@ import java.io.BufferedInputStream;
 import java.io.DataInputStream;
 import java.io.File;
 import java.io.FileInputStream;
+import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.FileWriter;
 import java.io.PrintStream;
 import java.io.Serializable;
+import java.io.UnsupportedEncodingException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Deque;
@@ -389,6 +391,9 @@ public class QTestUtil {
   }
 
   public void createSources() throws Exception {
+    
+    startSessionState();
+    
     // Create a bunch of tables with columns key and value
     LinkedList<String> cols = new LinkedList<String>();
     cols.add("key");
@@ -490,7 +495,8 @@ public class QTestUtil {
     testWarehouse = conf.getVar(HiveConf.ConfVars.METASTOREWAREHOUSE);
     // conf.logVars(System.out);
     // System.out.flush();
-
+    
+    SessionState.start(conf);
     db = Hive.get(conf);
     fs = FileSystem.get(conf);
     drv = new Driver(conf);
@@ -541,6 +547,8 @@ public class QTestUtil {
       createSources();
     }
 
+    HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
+    "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
     CliSessionState ss = new CliSessionState(conf);
     assert ss != null;
     ss.in = System.in;
@@ -554,7 +562,7 @@ public class QTestUtil {
     ss.err = ss.out;
     ss.setIsSilent(true);
     SessionState oldSs = SessionState.get();
-    if (oldSs != null) {
+    if (oldSs != null && oldSs.out != null && oldSs.out != System.out) {
       oldSs.out.close();
     }
     SessionState.start(ss);
@@ -566,6 +574,19 @@ public class QTestUtil {
     cliDriver.processInitFiles(ss);
   }
 
+  private CliSessionState startSessionState()
+      throws FileNotFoundException, UnsupportedEncodingException {
+    
+    HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
+        "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
+
+    CliSessionState ss = new CliSessionState(conf);
+    assert ss != null;
+
+    SessionState.start(ss);
+    return ss;
+  }
+
   public int executeOne(String tname) {
     String q = qMap.get(tname);
 
@@ -898,6 +919,7 @@ public class QTestUtil {
         "-I", "at junit",
         "-I", "Caused by:",
         "-I", "LOCK_QUERYID:",
+        "-I", "grantTime",
         "-I", "[.][.][.] [0-9]* more",
         (new File(logDir, tname + ".out")).getPath(),
         outFileName };

Modified: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java?rev=1057999&r1=1057998&r2=1057999&view=diff
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java (original)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/metadata/TestSemanticAnalyzerHookLoading.java Wed Jan 12 06:58:04 2011
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.conf.HiveC
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
 import org.apache.hadoop.hive.ql.Driver;
 import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse;
+import org.apache.hadoop.hive.ql.session.SessionState;
 
 public class TestSemanticAnalyzerHookLoading extends TestCase {
 
@@ -35,6 +36,7 @@ public class TestSemanticAnalyzerHookLoa
     HiveConf conf = new HiveConf(this.getClass());
     conf.set(ConfVars.SEMANTIC_ANALYZER_HOOK.varname, DummySemanticAnalyzerHook.class.getName());
     conf.set(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    SessionState.start(conf);
     Driver driver = new Driver(conf);
 
     driver.run("drop table testDL");

Added: hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java (added)
+++ hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/security/DummyAuthenticator.java Wed Jan 12 06:58:04 2011
@@ -0,0 +1,46 @@
+package org.apache.hadoop.hive.ql.security;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.ql.metadata.HiveException;
+
+public class DummyAuthenticator implements HiveAuthenticationProvider {
+  
+  private List<String> groupNames;
+  private String userName;
+  private Configuration conf;
+  
+  public DummyAuthenticator() {
+    this.groupNames = new ArrayList<String>();
+    groupNames.add("hive_test_group1");
+    groupNames.add("hive_test_group2");
+    userName = "hive_test_user";
+  }
+
+  @Override
+  public void destroy() throws HiveException{
+    return;
+  }
+
+  @Override
+  public List<String> getGroupNames() {
+    return groupNames;
+  }
+
+  @Override
+  public String getUserName() {
+    return userName;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+  
+  public Configuration getConf() {
+    return this.conf;
+  }
+
+}

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_1.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_1.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,9 @@
+create table authorization_fail_1 (key int, value string);
+set hive.security.authorization.enabled=true;
+
+revoke `ALL` on table authorization_fail_1 from user hive_test_user;
+
+grant `Create` on table authorization_fail_1 to user hive_test_user;
+grant `Create` on table authorization_fail_1 to user hive_test_user;
+
+

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_2.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_2.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_2.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_2.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,9 @@
+create table authorization_fail_2 (key int, value string) partitioned by (ds string);
+
+revoke `ALL` on table authorization_fail_2 from user hive_test_user;
+
+set hive.security.authorization.enabled=true;
+
+alter table authorization_fail_2 add partition (ds='2010');
+
+

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_3.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_3.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_3.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_3.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,12 @@
+create table authorization_fail_3 (key int, value string) partitioned by (ds string);
+set hive.security.authorization.enabled=true;
+
+revoke `ALL` on table authorization_fail_3 from user hive_test_user;
+
+grant `Create` on table authorization_fail_3 to user hive_test_user;
+alter table authorization_fail_3 add partition (ds='2010');
+
+show grant user hive_test_user on table authorization_fail_3;
+show grant user hive_test_user on table authorization_fail_3 partition (ds='2010');
+
+select key from authorization_fail_3 where ds='2010';

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_4.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_4.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_4.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_4.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,15 @@
+create table authorization_fail_4 (key int, value string) partitioned by (ds string);
+
+revoke `ALL` on table authorization_fail_4 from user hive_test_user;
+
+set hive.security.authorization.enabled=true;
+grant `Alter` on table authorization_fail_4 to user hive_test_user;
+ALTER TABLE authorization_fail_4 SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE");
+
+grant `Create` on table authorization_fail_4 to user hive_test_user;
+alter table authorization_fail_4 add partition (ds='2010');
+
+show grant user hive_test_user on table authorization_fail_4;
+show grant user hive_test_user on table authorization_fail_4 partition (ds='2010');
+
+select key from authorization_fail_4 where ds='2010';

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_5.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_5.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_5.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_5.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,20 @@
+create table authorization_fail (key int, value string) partitioned by (ds string);
+set hive.security.authorization.enabled=true;
+
+revoke `ALL` on table authorization_fail from user hive_test_user;
+
+grant `Alter` on table authorization_fail to user hive_test_user;
+ALTER TABLE authorization_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE");
+
+grant `Create` on table authorization_fail to user hive_test_user;
+grant `Select` on table authorization_fail to user hive_test_user;
+alter table authorization_fail add partition (ds='2010');
+
+show grant user hive_test_user on table authorization_fail;
+show grant user hive_test_user on table authorization_fail partition (ds='2010');
+
+revoke `Select` on table authorization_fail partition (ds='2010') from user hive_test_user;
+
+show grant user hive_test_user on table authorization_fail partition (ds='2010');
+
+select key from authorization_fail where ds='2010';
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_6.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_6.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_6.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_6.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,5 @@
+create table authorization_part_fail (key int, value string) partitioned by (ds string);
+revoke `ALL` on table authorization_part_fail from user hive_test_user;
+set hive.security.authorization.enabled=true;
+
+ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE");

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_7.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_7.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_7.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_fail_7.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,16 @@
+create table authorization_fail (key int, value string);
+revoke `ALL` on table authorization_fail from user hive_test_user;
+
+set hive.security.authorization.enabled=true;
+
+create role hive_test_role_fail;
+
+grant role hive_test_role_fail to user hive_test_user;
+grant `select` on table authorization_fail to role hive_test_role_fail;
+show role grant user hive_test_user;
+
+show grant role hive_test_role_fail on table authorization_fail;
+
+drop role hive_test_role_fail;
+
+select key from authorization_fail;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientnegative/authorization_part.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientnegative/authorization_part.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientnegative/authorization_part.q (added)
+++ hive/trunk/ql/src/test/queries/clientnegative/authorization_part.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,35 @@
+create table authorization_part_fail (key int, value string) partitioned by (ds string);
+ALTER TABLE authorization_part_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE");
+set hive.security.authorization.enabled=true;
+
+revoke `ALL` on table authorization_part_fail from user hive_test_user;
+
+grant `Create` on table authorization_part_fail to user hive_test_user;
+grant `Update` on table authorization_part_fail to user hive_test_user;
+grant `Drop` on table authorization_part_fail to user hive_test_user;
+grant `select` on table src to user hive_test_user;
+
+-- column grant to group
+
+grant `select`(key) on table authorization_part_fail to group hive_test_group1;
+grant `select` on table authorization_part_fail to group hive_test_group1;
+
+show grant group hive_test_group1 on table authorization_part_fail;
+
+insert overwrite table authorization_part_fail partition (ds='2010') select key, value from src; 
+show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2010');
+show grant group hive_test_group1 on table authorization_part_fail partition (ds='2010');
+select key, value from authorization_part_fail where ds='2010' order by key limit 20;
+
+insert overwrite table authorization_part_fail partition (ds='2011') select key, value from src; 
+show grant group hive_test_group1 on table authorization_part_fail(key) partition (ds='2011');
+show grant group hive_test_group1 on table authorization_part_fail partition (ds='2011');
+select key, value from authorization_part_fail where ds='2011' order by key limit 20;
+
+select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20;
+
+revoke `select` on table authorization_part_fail partition (ds='2010') from group hive_test_group1;
+
+select key,value, ds from authorization_part_fail where ds>='2010' order by key, ds limit 20;
+
+drop table authorization_part_fail;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_1.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_1.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,89 @@
+create table src_autho_test as select * from src;
+
+revoke `ALL` on table src_autho_test from user hive_test_user;
+
+set hive.security.authorization.enabled=true;
+
+--table grant to user
+
+grant `select` on table src_autho_test to user hive_test_user;
+
+show grant user hive_test_user on table src_autho_test;
+show grant user hive_test_user on table src_autho_test(key);
+
+select key from src_autho_test order by key limit 20;
+
+revoke `select` on table src_autho_test from user hive_test_user;
+show grant user hive_test_user on table src_autho_test;
+show grant user hive_test_user on table src_autho_test(key);
+
+--column grant to user
+
+grant `select`(key) on table src_autho_test to user hive_test_user;
+
+show grant user hive_test_user on table src_autho_test;
+show grant user hive_test_user on table src_autho_test(key);
+
+select key from src_autho_test order by key limit 20;
+
+revoke `select`(key) on table src_autho_test from user hive_test_user;
+show grant user hive_test_user on table src_autho_test;
+show grant user hive_test_user on table src_autho_test(key); 
+
+--table grant to group
+
+grant `select` on table src_autho_test to group hive_test_group1;
+
+show grant group hive_test_group1 on table src_autho_test;
+show grant group hive_test_group1 on table src_autho_test(key);
+
+select key from src_autho_test order by key limit 20;
+
+revoke `select` on table src_autho_test from group hive_test_group1;
+show grant group hive_test_group1 on table src_autho_test;
+show grant group hive_test_group1 on table src_autho_test(key);
+
+--column grant to group
+
+grant `select`(key) on table src_autho_test to group hive_test_group1;
+
+show grant group hive_test_group1 on table src_autho_test;
+show grant group hive_test_group1 on table src_autho_test(key);
+
+select key from src_autho_test order by key limit 20;
+
+revoke `select`(key) on table src_autho_test from group hive_test_group1;
+show grant group hive_test_group1 on table src_autho_test;
+show grant group hive_test_group1 on table src_autho_test(key);
+
+--role
+create role src_role;
+grant role src_role to user hive_test_user;
+show role grant user hive_test_user;
+
+--column grant to role
+
+grant `select`(key) on table src_autho_test to role src_role;
+
+show grant role src_role on table src_autho_test;
+show grant role src_role on table src_autho_test(key);
+
+select key from src_autho_test order by key limit 20;
+
+revoke `select`(key) on table src_autho_test from role src_role;
+
+--table grant to role
+
+grant `select` on table src_autho_test to role src_role;
+
+select key from src_autho_test order by key limit 20;
+
+show grant role src_role on table src_autho_test;
+show grant role src_role on table src_autho_test(key);
+revoke `select` on table src_autho_test from role src_role;
+
+-- drop role
+drop role src_role;
+
+set hive.security.authorization.enabled=false;
+drop table src_autho_test;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_2.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_2.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_2.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_2.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,111 @@
+create table authorization_part (key int, value string) partitioned by (ds string);
+
+revoke `ALL` on table authorization_part from user hive_test_user;
+
+ALTER TABLE authorization_part SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE");
+set hive.security.authorization.enabled=true;
+
+-- column grant to user
+grant `Create` on table authorization_part to user hive_test_user;
+grant `Update` on table authorization_part to user hive_test_user;
+grant `Drop` on table authorization_part to user hive_test_user;
+grant `select` on table src to user hive_test_user;
+
+show grant user hive_test_user on table authorization_part;
+
+alter table authorization_part add partition (ds='2010');
+show grant user hive_test_user on table authorization_part partition (ds='2010');
+
+grant `select`(key) on table authorization_part to user hive_test_user;
+alter table authorization_part drop partition (ds='2010');
+insert overwrite table authorization_part partition (ds='2010') select key, value from src; 
+show grant user hive_test_user on table authorization_part(key) partition (ds='2010');
+show grant user hive_test_user on table authorization_part(key);
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select`(key) on table authorization_part from user hive_test_user;
+show grant user hive_test_user on table authorization_part(key);
+show grant user hive_test_user on table authorization_part(key) partition (ds='2010');
+
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select`(key) on table authorization_part partition (ds='2010') from user hive_test_user;
+show grant user hive_test_user on table authorization_part(key) partition (ds='2010');
+
+alter table authorization_part drop partition (ds='2010');
+
+-- table grant to user
+show grant user hive_test_user on table authorization_part;
+
+alter table authorization_part add partition (ds='2010');
+show grant user hive_test_user on table authorization_part partition (ds='2010');
+
+grant `select` on table authorization_part to user hive_test_user;
+alter table authorization_part drop partition (ds='2010');
+insert overwrite table authorization_part partition (ds='2010') select key, value from src; 
+show grant user hive_test_user on table authorization_part partition (ds='2010');
+show grant user hive_test_user on table authorization_part;
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select` on table authorization_part from user hive_test_user;
+show grant user hive_test_user on table authorization_part;
+show grant user hive_test_user on table authorization_part partition (ds='2010');
+
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select` on table authorization_part partition (ds='2010') from user hive_test_user;
+show grant user hive_test_user on table authorization_part partition (ds='2010');
+
+alter table authorization_part drop partition (ds='2010');
+
+-- column grant to group
+
+show grant group hive_test_group1 on table authorization_part;
+
+alter table authorization_part add partition (ds='2010');
+show grant group hive_test_group1 on table authorization_part partition (ds='2010');
+
+grant `select`(key) on table authorization_part to group hive_test_group1;
+alter table authorization_part drop partition (ds='2010');
+insert overwrite table authorization_part partition (ds='2010') select key, value from src; 
+show grant group hive_test_group1 on table authorization_part(key) partition (ds='2010');
+show grant group hive_test_group1 on table authorization_part(key);
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select`(key) on table authorization_part from group hive_test_group1;
+show grant group hive_test_group1 on table authorization_part(key);
+show grant group hive_test_group1 on table authorization_part(key) partition (ds='2010');
+
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select`(key) on table authorization_part partition (ds='2010') from group hive_test_group1;
+show grant group hive_test_group1 on table authorization_part(key) partition (ds='2010');
+
+alter table authorization_part drop partition (ds='2010');
+
+-- table grant to group
+show grant group hive_test_group1 on table authorization_part;
+
+alter table authorization_part add partition (ds='2010');
+show grant group hive_test_group1 on table authorization_part partition (ds='2010');
+
+grant `select` on table authorization_part to group hive_test_group1;
+alter table authorization_part drop partition (ds='2010');
+insert overwrite table authorization_part partition (ds='2010') select key, value from src; 
+show grant group hive_test_group1 on table authorization_part partition (ds='2010');
+show grant group hive_test_group1 on table authorization_part;
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select` on table authorization_part from group hive_test_group1;
+show grant group hive_test_group1 on table authorization_part;
+show grant group hive_test_group1 on table authorization_part partition (ds='2010');
+
+select key from authorization_part where ds='2010' order by key limit 20;
+
+revoke `select` on table authorization_part partition (ds='2010') from group hive_test_group1;
+show grant group hive_test_group1 on table authorization_part partition (ds='2010');
+
+
+revoke `select` on table src from user hive_test_user;
+set hive.security.authorization.enabled=false;
+drop table authorization_part;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_3.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_3.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_3.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_3.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,19 @@
+create table src_autho_test as select * from src;
+
+revoke `ALL` on table src_autho_test from user hive_test_user;
+
+grant `drop` on table src_autho_test to user hive_test_user;
+grant `select` on table src_autho_test to user hive_test_user;
+
+show grant user hive_test_user on table src_autho_test;
+
+revoke `select` on table src_autho_test from user hive_test_user;
+revoke `drop` on table src_autho_test from user hive_test_user;
+
+grant `drop`,`select` on table src_autho_test to user hive_test_user;
+show grant user hive_test_user on table src_autho_test;
+revoke `drop`,`select` on table src_autho_test from user hive_test_user;
+
+grant `drop`,`select`(key), `select`(value) on table src to user hive_test_user;
+show grant user hive_test_user on table src_autho_test;
+revoke `drop`,`select`(key), `select`(value) on table src from user hive_test_user;
\ No newline at end of file

Added: hive/trunk/ql/src/test/queries/clientpositive/authorization_4.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/authorization_4.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/authorization_4.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/authorization_4.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,9 @@
+create table src_autho_test as select * from src;
+
+set hive.security.authorization.enabled=true;
+
+show grant user hive_test_user on table src_autho_test;
+
+select key from src_autho_test order by key limit 20;
+
+drop table src_autho_test;
\ No newline at end of file

Modified: hive/trunk/ql/src/test/queries/clientpositive/input19.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/input19.q?rev=1057999&r1=1057998&r2=1057999&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/input19.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/input19.q Wed Jan 12 06:58:04 2011
@@ -1,5 +1,5 @@
 
-create table apachelog(ipaddress STRING,identd STRING,user STRING,finishtime STRING,requestline string,returncode INT,size INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe' WITH SERDEPROPERTIES (  'serialization.format'= 'org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol',  'quote.delim'= '("|\\[|\\])',  'field.delim'=' ',  'serialization.null.format'='-'  ) STORED AS TEXTFILE;
+create table apachelog(ipaddress STRING,identd STRING,user_name STRING,finishtime STRING,requestline string,returncode INT,size INT) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.dynamic_type.DynamicSerDe' WITH SERDEPROPERTIES (  'serialization.format'= 'org.apache.hadoop.hive.serde2.thrift.TCTLSeparatedProtocol',  'quote.delim'= '("|\\[|\\])',  'field.delim'=' ',  'serialization.null.format'='-'  ) STORED AS TEXTFILE;
 LOAD DATA LOCAL INPATH '../data/files/apache.access.log' INTO TABLE apachelog;
 SELECT a.* FROM apachelog a;
 

Added: hive/trunk/ql/src/test/queries/clientpositive/keyword_1.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/keyword_1.q?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/keyword_1.q (added)
+++ hive/trunk/ql/src/test/queries/clientpositive/keyword_1.q Wed Jan 12 06:58:04 2011
@@ -0,0 +1,17 @@
+create table test_user (user string, `group` string);
+grant `select` on table test_user to user hive_test;
+
+explain select user from test_user;
+
+show grant user hive_test on table test_user;
+
+drop table test_user;
+
+create table test_user (role string, `group` string);
+grant `select` on table test_user to user hive_test;
+
+explain select role from test_user;
+
+show grant user hive_test on table test_user;
+
+drop table test_user;
\ No newline at end of file

Modified: hive/trunk/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q?rev=1057999&r1=1057998&r2=1057999&view=diff
==============================================================================
--- hive/trunk/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q (original)
+++ hive/trunk/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q Wed Jan 12 06:58:04 2011
@@ -21,5 +21,7 @@ SHOW INDEXES ON show_idx_full;
 EXPLAIN SHOW INDEXES ON show_idx_empty;
 SHOW INDEXES ON show_idx_empty;
 
+DROP INDEX idx_1 on show_idx_full;
+DROP INDEX idx_2 on show_idx_full;
 DROP TABLE show_idx_empty;
 DROP TABLE show_idx_full;
\ No newline at end of file

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_fail_1.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_fail_1.q.out?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_fail_1.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_fail_1.q.out Wed Jan 12 06:58:04 2011
@@ -0,0 +1,17 @@
+PREHOOK: query: create table authorization_fail_1 (key int, value string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table authorization_fail_1 (key int, value string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@authorization_fail_1
+PREHOOK: query: revoke `ALL` on table authorization_fail_1 from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: query: revoke `ALL` on table authorization_fail_1 from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: query: grant `Create` on table authorization_fail_1 to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Create` on table authorization_fail_1 to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: grant `Create` on table authorization_fail_1 to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+Error: java.lang.RuntimeException: InvalidObjectException(message:Create is already granted on table [default,authorization_fail_1] by hive_test_user)
+FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_fail_2.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_fail_2.q.out?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_fail_2.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_fail_2.q.out Wed Jan 12 06:58:04 2011
@@ -0,0 +1,10 @@
+PREHOOK: query: create table authorization_fail_2 (key int, value string) partitioned by (ds string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table authorization_fail_2 (key int, value string) partitioned by (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@authorization_fail_2
+PREHOOK: query: revoke `ALL` on table authorization_fail_2 from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: query: revoke `ALL` on table authorization_fail_2 from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+Authorization failed:No privilege 'Create' found for inputs { database:default, table:authorization_fail_2}. Use show grant to get more details.

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_fail_3.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_fail_3.q.out?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_fail_3.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_fail_3.q.out Wed Jan 12 06:58:04 2011
@@ -0,0 +1,37 @@
+PREHOOK: query: create table authorization_fail_3 (key int, value string) partitioned by (ds string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table authorization_fail_3 (key int, value string) partitioned by (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@authorization_fail_3
+PREHOOK: query: revoke `ALL` on table authorization_fail_3 from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: query: revoke `ALL` on table authorization_fail_3 from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: query: grant `Create` on table authorization_fail_3 to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Create` on table authorization_fail_3 to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: alter table authorization_fail_3 add partition (ds='2010')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@authorization_fail_3
+POSTHOOK: query: alter table authorization_fail_3 add partition (ds='2010')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@authorization_fail_3
+POSTHOOK: Output: default@authorization_fail_3@ds=2010
+PREHOOK: query: show grant user hive_test_user on table authorization_fail_3
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail_3
+POSTHOOK: type: SHOW_GRANT
+
+database	default	
+table	authorization_fail_3	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Create	
+grantTime	1292569774	
+grantor	hive_test_user	
+PREHOOK: query: show grant user hive_test_user on table authorization_fail_3 partition (ds='2010')
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail_3 partition (ds='2010')
+POSTHOOK: type: SHOW_GRANT
+Authorization failed:No privilege 'Select' found for inputs { database:default, table:authorization_fail_3, columnName:key}. Use show grant to get more details.

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_fail_4.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_fail_4.q.out?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_fail_4.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_fail_4.q.out Wed Jan 12 06:58:04 2011
@@ -0,0 +1,75 @@
+PREHOOK: query: create table authorization_fail_4 (key int, value string) partitioned by (ds string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table authorization_fail_4 (key int, value string) partitioned by (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@authorization_fail_4
+PREHOOK: query: revoke `ALL` on table authorization_fail_4 from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: query: revoke `ALL` on table authorization_fail_4 from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: query: grant `Alter` on table authorization_fail_4 to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Alter` on table authorization_fail_4 to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: ALTER TABLE authorization_fail_4 SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE")
+PREHOOK: type: ALTERTABLE_PROPERTIES
+PREHOOK: Input: default@authorization_fail_4
+PREHOOK: Output: default@authorization_fail_4
+POSTHOOK: query: ALTER TABLE authorization_fail_4 SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE")
+POSTHOOK: type: ALTERTABLE_PROPERTIES
+POSTHOOK: Input: default@authorization_fail_4
+POSTHOOK: Output: default@authorization_fail_4
+PREHOOK: query: grant `Create` on table authorization_fail_4 to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Create` on table authorization_fail_4 to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: alter table authorization_fail_4 add partition (ds='2010')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@authorization_fail_4
+POSTHOOK: query: alter table authorization_fail_4 add partition (ds='2010')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@authorization_fail_4
+POSTHOOK: Output: default@authorization_fail_4@ds=2010
+PREHOOK: query: show grant user hive_test_user on table authorization_fail_4
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail_4
+POSTHOOK: type: SHOW_GRANT
+
+database	default	
+table	authorization_fail_4	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Alter	
+grantTime	1292569775	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail_4	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Create	
+grantTime	1292569776	
+grantor	hive_test_user	
+PREHOOK: query: show grant user hive_test_user on table authorization_fail_4 partition (ds='2010')
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail_4 partition (ds='2010')
+POSTHOOK: type: SHOW_GRANT
+
+database	default	
+table	authorization_fail_4	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Alter	
+grantTime	1292569776	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail_4	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Create	
+grantTime	1292569776	
+grantor	hive_test_user	
+Authorization failed:No privilege 'Select' found for inputs { database:default, table:authorization_fail_4, partitionName:ds=2010, columnName:key}. Use show grant to get more details.

Added: hive/trunk/ql/src/test/results/clientnegative/authorization_fail_5.q.out
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/test/results/clientnegative/authorization_fail_5.q.out?rev=1057999&view=auto
==============================================================================
--- hive/trunk/ql/src/test/results/clientnegative/authorization_fail_5.q.out (added)
+++ hive/trunk/ql/src/test/results/clientnegative/authorization_fail_5.q.out Wed Jan 12 06:58:04 2011
@@ -0,0 +1,122 @@
+PREHOOK: query: create table authorization_fail (key int, value string) partitioned by (ds string)
+PREHOOK: type: CREATETABLE
+POSTHOOK: query: create table authorization_fail (key int, value string) partitioned by (ds string)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: default@authorization_fail
+PREHOOK: query: revoke `ALL` on table authorization_fail from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: query: revoke `ALL` on table authorization_fail from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: query: grant `Alter` on table authorization_fail to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Alter` on table authorization_fail to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: ALTER TABLE authorization_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE")
+PREHOOK: type: ALTERTABLE_PROPERTIES
+PREHOOK: Input: default@authorization_fail
+PREHOOK: Output: default@authorization_fail
+POSTHOOK: query: ALTER TABLE authorization_fail SET TBLPROPERTIES ("PARTITION_LEVEL_PRIVILEGE"="TRUE")
+POSTHOOK: type: ALTERTABLE_PROPERTIES
+POSTHOOK: Input: default@authorization_fail
+POSTHOOK: Output: default@authorization_fail
+PREHOOK: query: grant `Create` on table authorization_fail to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Create` on table authorization_fail to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: grant `Select` on table authorization_fail to user hive_test_user
+PREHOOK: type: GRANT_PRIVILEGE
+POSTHOOK: query: grant `Select` on table authorization_fail to user hive_test_user
+POSTHOOK: type: GRANT_PRIVILEGE
+PREHOOK: query: alter table authorization_fail add partition (ds='2010')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Input: default@authorization_fail
+POSTHOOK: query: alter table authorization_fail add partition (ds='2010')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Input: default@authorization_fail
+POSTHOOK: Output: default@authorization_fail@ds=2010
+PREHOOK: query: show grant user hive_test_user on table authorization_fail
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail
+POSTHOOK: type: SHOW_GRANT
+
+database	default	
+table	authorization_fail	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Alter	
+grantTime	1292570198	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Create	
+grantTime	1292570198	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Select	
+grantTime	1292570198	
+grantor	hive_test_user	
+PREHOOK: query: show grant user hive_test_user on table authorization_fail partition (ds='2010')
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail partition (ds='2010')
+POSTHOOK: type: SHOW_GRANT
+
+database	default	
+table	authorization_fail	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Alter	
+grantTime	1292570198	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Create	
+grantTime	1292570198	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Select	
+grantTime	1292570198	
+grantor	hive_test_user	
+PREHOOK: query: revoke `Select` on table authorization_fail partition (ds='2010') from user hive_test_user
+PREHOOK: type: REVOKE_PRIVILEGE
+POSTHOOK: query: revoke `Select` on table authorization_fail partition (ds='2010') from user hive_test_user
+POSTHOOK: type: REVOKE_PRIVILEGE
+PREHOOK: query: show grant user hive_test_user on table authorization_fail partition (ds='2010')
+PREHOOK: type: SHOW_GRANT
+POSTHOOK: query: show grant user hive_test_user on table authorization_fail partition (ds='2010')
+POSTHOOK: type: SHOW_GRANT
+
+database	default	
+table	authorization_fail	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Alter	
+grantTime	1292570198	
+grantor	hive_test_user	
+
+database	default	
+table	authorization_fail	
+partition	ds=2010	
+principalName	hive_test_user	
+principalType	USER	
+privilege	Create	
+grantTime	1292570198	
+grantor	hive_test_user	
+Authorization failed:No privilege 'Select' found for inputs { database:default, table:authorization_fail, partitionName:ds=2010, columnName:key}. Use show grant to get more details.