You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by xu...@apache.org on 2015/05/15 06:31:54 UTC

[27/50] [abbrv] hive git commit: Revert "HIVE-9736 : StorageBasedAuthProvider should batch namenode-calls where possible (Mithun Radhakrishnan, reviewed by Chris Nauroth, Sushanth Sowmyan)"

Revert "HIVE-9736 : StorageBasedAuthProvider should batch namenode-calls where possible (Mithun Radhakrishnan, reviewed by Chris Nauroth, Sushanth Sowmyan)"

This reverts commit 19886150121b6081127bf1e581b24d8dcc12f1df.


Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/ecde4ae9
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/ecde4ae9
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/ecde4ae9

Branch: refs/heads/beeline-cli
Commit: ecde4ae96f88ed88141a6593e1f935126d6466f3
Parents: 306e61a
Author: Sushanth Sowmyan <kh...@gmail.com>
Authored: Wed May 6 17:53:00 2015 -0700
Committer: Sushanth Sowmyan <kh...@gmail.com>
Committed: Wed May 6 17:53:00 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hive/common/FileUtils.java    | 155 ++++++-------------
 .../org/apache/hadoop/hive/conf/HiveConf.java   |   7 -
 .../StorageBasedAuthorizationProvider.java      | 114 +-------------
 .../apache/hadoop/hive/shims/Hadoop23Shims.java |  29 +---
 .../org/apache/hadoop/fs/DefaultFileAccess.java |  65 +++-----
 .../apache/hadoop/hive/shims/HadoopShims.java   |  24 +--
 .../hadoop/hive/shims/HadoopShimsSecure.java    |   8 -
 7 files changed, 84 insertions(+), 318 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
index 536fe11..c2c54bc 100644
--- a/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
+++ b/common/src/java/org/apache/hadoop/hive/common/FileUtils.java
@@ -25,16 +25,12 @@ import java.net.URISyntaxException;
 import java.security.AccessControlException;
 import java.security.PrivilegedExceptionAction;
 import java.util.BitSet;
-import java.util.EnumSet;
-import java.util.Iterator;
 import java.util.List;
 
-import com.google.common.base.Function;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.DefaultFileAccess;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FileUtil;
@@ -373,54 +369,26 @@ public final class FileUtils {
   public static void checkFileAccessWithImpersonation(final FileSystem fs,
       final FileStatus stat, final FsAction action, final String user)
           throws IOException, AccessControlException, InterruptedException, Exception {
-    checkFileAccessWithImpersonation(fs,
-                                     Iterators.singletonIterator(stat),
-                                     EnumSet.of(action),
-                                     user);
-  }
-
-  /**
-   * Perform a check to determine if the user is able to access the file passed in.
-   * If the user name passed in is different from the current user, this method will
-   * attempt to do impersonate the user to do the check; the current user should be
-   * able to create proxy users in this case.
-   * @param fs   FileSystem of the path to check
-   * @param statuses FileStatus instances representing the file
-   * @param actions The FsActions that will be checked
-   * @param user User name of the user that will be checked for access.  If the user name
-   *             is null or the same as the current user, no user impersonation will be done
-   *             and the check will be done as the current user. Otherwise the file access
-   *             check will be performed within a doAs() block to use the access privileges
-   *             of this user. In this case the user must be configured to impersonate other
-   *             users, otherwise this check will fail with error.
-   * @throws IOException
-   * @throws AccessControlException
-   * @throws InterruptedException
-   * @throws Exception
-   */
-  public static void checkFileAccessWithImpersonation(final FileSystem fs,
-      final Iterator<FileStatus> statuses, final EnumSet<FsAction> actions, final String user)
-          throws IOException, AccessControlException, InterruptedException, Exception {
     UserGroupInformation ugi = Utils.getUGI();
     String currentUser = ugi.getShortUserName();
 
     if (user == null || currentUser.equals(user)) {
       // No need to impersonate user, do the checks as the currently configured user.
-      ShimLoader.getHadoopShims().checkFileAccess(fs, statuses, actions);
-    }
-    else {
-      // Otherwise, try user impersonation. Current user must be configured to do user impersonation.
-      UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(
-          user, UserGroupInformation.getLoginUser());
-      proxyUser.doAs(new PrivilegedExceptionAction<Object>() {
-        @Override
-        public Object run() throws Exception {
-          FileSystem fsAsUser = FileSystem.get(fs.getUri(), fs.getConf());
-          ShimLoader.getHadoopShims().checkFileAccess(fsAsUser, statuses, actions);
-          return null;
-        }
-      });
+      ShimLoader.getHadoopShims().checkFileAccess(fs, stat, action);
+      return;
     }
+
+    // Otherwise, try user impersonation. Current user must be configured to do user impersonation.
+    UserGroupInformation proxyUser = UserGroupInformation.createProxyUser(
+        user, UserGroupInformation.getLoginUser());
+    proxyUser.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        FileSystem fsAsUser = FileSystem.get(fs.getUri(), fs.getConf());
+        ShimLoader.getHadoopShims().checkFileAccess(fsAsUser, stat, action);
+        return null;
+      }
+    });
   }
 
   /**
@@ -709,91 +677,70 @@ public final class FileUtils {
    * @param path
    * @param conf
    * @param user
+   * @throws AccessControlException
+   * @throws InterruptedException
    * @throws Exception
    */
-  public static void checkDeletePermission(Path path, Configuration conf, String user) throws  Exception {
+  public static void checkDeletePermission(Path path, Configuration conf, String user)
+      throws AccessControlException, InterruptedException, Exception {
+   // This requires ability to delete the given path.
+    // The following 2 conditions should be satisfied for this-
+    // 1. Write permissions on parent dir
+    // 2. If sticky bit is set on parent dir then one of following should be
+    // true
+    //   a. User is owner of the current dir/file
+    //   b. User is owner of the parent dir
+    //   Super users are also allowed to drop the file, but there is no good way of checking
+    //   if a user is a super user. Also super users running hive queries is not a common
+    //   use case. super users can also do a chown to be able to drop the file
 
     if(path == null) {
       // no file/dir to be deleted
       return;
     }
 
-    // check user has write permissions on the parent dir
     final FileSystem fs = path.getFileSystem(conf);
+    // check user has write permissions on the parent dir
     FileStatus stat = null;
     try {
       stat = fs.getFileStatus(path);
     } catch (FileNotFoundException e) {
       // ignore
     }
-
     if (stat == null) {
       // no file/dir to be deleted
       return;
     }
-
-    checkDeletePermission(fs, Lists.newArrayList(stat), conf, user);
-  }
-
-  /**
-   * Checks if delete can be performed on given path by given user.
-   * If file does not exist it just returns without throwing an Exception
-   * @param fs The FileSystem instance
-   * @param fileStatuses The FileStatus instances for the paths being checked.
-   * @param conf Configuration, corresponding to the FileSystem.
-   * @param user The user, whose permission is to be checked.
-   * @throws Exception
-   */
-  public static void checkDeletePermission(FileSystem fs, Iterable<FileStatus> fileStatuses,
-                                           Configuration conf, String user) throws Exception {
-
-    // This requires ability to delete the given path.
-    // The following 2 conditions should be satisfied for this-
-    // 1. Write permissions on parent dir
-    // 2. If sticky bit is set on parent dir then one of following should be
-    // true
-    //   a. User is owner of the current dir/file
-    //   b. User is owner of the parent dir
-    FileUtils.checkFileAccessWithImpersonation(fs, fileStatuses.iterator(), EnumSet.of(FsAction.WRITE), user);
+    FileUtils.checkFileAccessWithImpersonation(fs, stat, FsAction.WRITE, user);
 
     HadoopShims shims = ShimLoader.getHadoopShims();
     if (!shims.supportStickyBit()) {
-      // No support for sticky-bit.
+      // not supports sticky bit
       return;
     }
 
-    List<Path> allParentPaths =
-        Lists.newArrayList(
-            Iterators.transform(fileStatuses.iterator(), new Function<FileStatus, Path>() {
-              @Override
-              public Path apply(FileStatus input) {
-                return input.getPath().getParent();
-              }
-            })
-        );
-
-    Iterator<FileStatus> childStatusIterator = fileStatuses.iterator();
-    for (List<Path> parentPaths : Lists.partition(allParentPaths, getListStatusBatchSize(conf))) {
-      for (FileStatus parentFileStatus : fs.listStatus(parentPaths.toArray(new Path[parentPaths.size()]))) {
-        assert childStatusIterator.hasNext() : "Number of parent-file-statuses doesn't match children.";
-        FileStatus childFileStatus = childStatusIterator.next();
-        // Check sticky-bits on parent-dirs.
-        if (shims.hasStickyBit(parentFileStatus.getPermission())
-            && !parentFileStatus.getOwner().equals(user)
-            && !childFileStatus.getOwner().equals(user)) {
-          throw new IOException(String.format("Permission Denied: User %s can't delete %s because sticky bit is\""
-              + " set on the parent dir and user does not own this file or its parent\"", user, childFileStatus.getPath()));
-        }
-      } // for_each( parent_path );
-    } // for_each( batch_of_parentPaths );
+    // check if sticky bit is set on the parent dir
+    FileStatus parStatus = fs.getFileStatus(path.getParent());
+    if (!shims.hasStickyBit(parStatus.getPermission())) {
+      // no sticky bit, so write permission on parent dir is sufficient
+      // no further checks needed
+      return;
+    }
 
-    assert !childStatusIterator.hasNext() : "Did not process all file-statuses.";
+    // check if user is owner of parent dir
+    if (parStatus.getOwner().equals(user)) {
+      return;
+    }
 
-  } // static void checkDeletePermission();
+    // check if user is owner of current dir/file
+    FileStatus childStatus = fs.getFileStatus(path);
+    if (childStatus.getOwner().equals(user)) {
+      return;
+    }
+    String msg = String.format("Permission Denied: User %s can't delete %s because sticky bit is"
+        + " set on the parent dir and user does not own this file or its parent", user, path);
+    throw new IOException(msg);
 
-  private static int getListStatusBatchSize(Configuration configuration) {
-    return HiveConf.getIntVar(configuration,
-        HiveConf.ConfVars.HIVE_AUTHORIZATION_HDFS_LIST_STATUS_BATCH_SIZE);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------
diff --git a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
index db17f0f..54e154c 100644
--- a/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
+++ b/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
@@ -1629,13 +1629,6 @@ public class HiveConf extends Configuration {
         "of updating the original list means that you can append to the defaults\n" +
         "set by SQL standard authorization instead of replacing it entirely."),
 
-    HIVE_AUTHORIZATION_HDFS_LIST_STATUS_BATCH_SIZE(
-      "hive.authprovider.hdfs.liststatus.batch.size", 1000,
-      "Number of FileStatus objects to be queried for when listing files, for HDFS-based authorization.\n" +
-          "Note: If this exceeds dfs.ls.limit (as set in hdfs-site.xml), DFSClient might use the smaller value as \n" +
-          "the batch-size, internally."
-    ),
-
     HIVE_CLI_PRINT_HEADER("hive.cli.print.header", false, "Whether to print the names of the columns in query output."),
 
     HIVE_ERROR_ON_EMPTY_PARTITION("hive.error.on.empty.partition", false,

http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
index 6a5c510..8f81ef9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/StorageBasedAuthorizationProvider.java
@@ -18,20 +18,15 @@
 
 package org.apache.hadoop.hive.ql.security.authorization;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.AccessControlException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.EnumSet;
-import java.util.Iterator;
 import java.util.List;
 
 import javax.security.auth.login.LoginException;
 
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -40,7 +35,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hive.common.FileUtils;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler;
 import org.apache.hadoop.hive.metastore.TableType;
 import org.apache.hadoop.hive.metastore.Warehouse;
@@ -69,7 +63,7 @@ import org.apache.hadoop.hive.ql.metadata.Table;
  * out to the parent directory recursively to determine its permissions till
  * it finds a parent that does exist.
  */
-public class StorageBasedAuthorizationProvider extends HiveMultiPartitionAuthorizationProviderBase
+public class StorageBasedAuthorizationProvider extends HiveAuthorizationProviderBase
     implements HiveMetastoreAuthorizationProvider {
 
   private Warehouse wh;
@@ -248,89 +242,6 @@ public class StorageBasedAuthorizationProvider extends HiveMultiPartitionAuthori
     }
   }
 
-  @Override
-  public void authorize(Table table, Iterable<Partition> partitions,
-                        Privilege[] requiredReadPrivileges, Privilege[] requiredWritePrivileges)
-       throws HiveException, AuthorizationException {
-
-    try {
-      class MustCheckTablePermissions { // For closure.
-        public boolean value = false;
-      }
-
-      final MustCheckTablePermissions mustCheckTablePermissions = new MustCheckTablePermissions();
-      final FileSystem fs = table.getDataLocation().getFileSystem(getConf());
-
-      // Get partition paths. Filter out null-partitions, and partitions without data-locations.
-      Iterator<Partition> nonNullPartitions
-               = Iterators.filter(partitions.iterator(), new Predicate<Partition>() {
-        @Override
-        public boolean apply(Partition partition) {
-          try {
-            boolean isValidPartitionPath = partition != null
-                                           && partition.getDataLocation() != null
-                                           && fs.exists(partition.getDataLocation());
-            mustCheckTablePermissions.value |= isValidPartitionPath;
-            return isValidPartitionPath;
-          }
-          catch (IOException exception){
-            throw new RuntimeException("Could not find location for partition: " + partition, exception);
-          }
-        }
-      });
-
-      if (mustCheckTablePermissions.value) {
-        // At least one partition was null, or had a non-existent path. So check table-permissions, once.
-        // 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.
-        // Partition itself can also be null, in cases where this gets called as a generic
-        // catch-all call in cases like those with CTAS onto an unpartitioned table (see HIVE-1887)
-
-        // this should be the case only if this is a create partition.
-        // The privilege needed on the table should be ALTER_DATA, and not CREATE
-        authorize(table, new Privilege[]{}, new Privilege[]{Privilege.ALTER_DATA});
-      }
-
-
-      // authorize drops if there was a drop privilege requirement
-      // extract drop privileges
-      DropPrivilegeExtractor privExtractor = new DropPrivilegeExtractor(requiredReadPrivileges, requiredWritePrivileges);
-      requiredReadPrivileges = privExtractor.getReadReqPriv();
-      requiredWritePrivileges = privExtractor.getWriteReqPriv();
-      EnumSet<FsAction> actions = getFsActions(requiredReadPrivileges);
-      actions.addAll(getFsActions(requiredWritePrivileges));
-
-      ArrayList<Path> allPartitionPaths
-                = Lists.newArrayList(Iterators.transform(nonNullPartitions, new Function<Partition, Path>() {
-        @Override
-        public Path apply(Partition input) {
-          return input.getDataLocation();
-        }
-      }));
-
-      for (List<Path> partitionPaths : Lists.partition(allPartitionPaths, getListStatusBatchSize(getConf()))) {
-
-        List<FileStatus> fileStatuses = Arrays.asList(
-            fs.listStatus(partitionPaths.toArray(new Path[partitionPaths.size()])));
-
-        if (privExtractor.hasDropPrivilege) {
-          FileUtils.checkDeletePermission(fs, fileStatuses, getConf(), authenticator.getUserName());
-        }
-
-        checkPermissions(fs, fileStatuses.iterator(), actions, authenticator.getUserName());
-      }
-
-    }
-    catch (Exception exception) {
-      throw hiveException(exception);
-    }
-  }
-
-  private static int getListStatusBatchSize(Configuration configuration) {
-    return HiveConf.getIntVar(configuration,
-                              HiveConf.ConfVars.HIVE_AUTHORIZATION_HDFS_LIST_STATUS_BATCH_SIZE);
-  }
-
   private void checkDeletePermission(Path dataLocation, Configuration conf, String userName)
       throws HiveException {
     try {
@@ -477,28 +388,17 @@ public class StorageBasedAuthorizationProvider extends HiveMultiPartitionAuthori
   protected static void checkPermissions(final FileSystem fs, final FileStatus stat,
       final EnumSet<FsAction> actions, String user) throws IOException,
       AccessControlException, HiveException {
-    checkPermissions(fs, Iterators.singletonIterator(stat), actions, user);
-  }
-
-  @SuppressWarnings("deprecation")
-  protected static void checkPermissions(final FileSystem fs, Iterator<FileStatus> fileStatuses,
-                                         final EnumSet<FsAction> actions, String user)
-      throws IOException, AccessControlException, HiveException {
 
+    if (stat == null) {
+      // File named by path doesn't exist; nothing to validate.
+      return;
+    }
     FsAction checkActions = FsAction.NONE;
     for (FsAction action : actions) {
       checkActions = checkActions.or(action);
     }
-
-    Iterator<FileStatus> nonNullFileStatuses = Iterators.filter(fileStatuses, new Predicate<FileStatus>() {
-      @Override
-      public boolean apply(FileStatus fileStatus) {
-        return fileStatus != null;
-      }
-    });
-
     try {
-      FileUtils.checkFileAccessWithImpersonation(fs, nonNullFileStatuses, EnumSet.of(checkActions), user);
+      FileUtils.checkFileAccessWithImpersonation(fs, stat, checkActions, user);
     } catch (Exception err) {
       // fs.permission.AccessControlException removed by HADOOP-11356, but Hive users on older
       // Hadoop versions may still see this exception .. have to reference by name.

http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
----------------------------------------------------------------------
diff --git a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
index 4547baa..d349068 100644
--- a/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
+++ b/shims/0.23/src/main/java/org/apache/hadoop/hive/shims/Hadoop23Shims.java
@@ -29,11 +29,11 @@ import java.security.AccessControlException;
 import java.security.NoSuchAlgorithmException;
 import java.util.ArrayList;
 import java.util.Comparator;
-import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
 
 import org.apache.commons.lang.StringUtils;
@@ -986,33 +986,6 @@ public class Hadoop23Shims extends HadoopShimsSecure {
     }
   }
 
-  @Override
-  public void checkFileAccess(FileSystem fs, Iterator<FileStatus> statuses, EnumSet<FsAction> actions)
-      throws IOException, AccessControlException, Exception {
-    try {
-      if (accessMethod == null) {
-        // Have to rely on Hive implementation of filesystem permission checks.
-        DefaultFileAccess.checkFileAccess(fs, statuses, actions);
-      }
-      else {
-        while (statuses.hasNext()) {
-          accessMethod.invoke(fs, statuses.next(), combine(actions));
-        }
-      }
-
-    } catch (Exception err) {
-      throw wrapAccessException(err);
-    }
-  }
-
-  private static FsAction combine(EnumSet<FsAction> actions) {
-    FsAction resultantAction = FsAction.NONE;
-    for (FsAction action : actions) {
-      resultantAction = resultantAction.or(action);
-    }
-    return resultantAction;
-  }
-
   /**
    * If there is an AccessException buried somewhere in the chain of failures, wrap the original
    * exception in an AccessException. Othewise just return the original exception.

http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/shims/common/src/main/java/org/apache/hadoop/fs/DefaultFileAccess.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/fs/DefaultFileAccess.java b/shims/common/src/main/java/org/apache/hadoop/fs/DefaultFileAccess.java
index c4261cb..45ca210 100644
--- a/shims/common/src/main/java/org/apache/hadoop/fs/DefaultFileAccess.java
+++ b/shims/common/src/main/java/org/apache/hadoop/fs/DefaultFileAccess.java
@@ -18,22 +18,23 @@
 
 package org.apache.hadoop.fs;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.security.AccessControlException;
+import java.util.ArrayList;
 import java.util.Arrays;
-import java.util.Collections;
 import java.util.EnumSet;
-import java.util.Iterator;
 import java.util.List;
 
 import javax.security.auth.login.LoginException;
 
-import com.google.common.collect.Iterators;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.hive.shims.Utils;
 import org.apache.hadoop.security.UserGroupInformation;
 
@@ -46,7 +47,7 @@ public class DefaultFileAccess {
 
   private static Log LOG = LogFactory.getLog(DefaultFileAccess.class);
 
-  private static List<String> emptyGroups = Collections.emptyList();
+  private static List<String> emptyGroups = new ArrayList<String>(0);
 
   public static void checkFileAccess(FileSystem fs, FileStatus stat, FsAction action)
       throws IOException, AccessControlException, LoginException {
@@ -59,62 +60,34 @@ public class DefaultFileAccess {
 
   public static void checkFileAccess(FileSystem fs, FileStatus stat, FsAction action,
       String user, List<String> groups) throws IOException, AccessControlException {
-    checkFileAccess(fs, Iterators.singletonIterator(stat), EnumSet.of(action), user, groups);
-  }
-
-  public static void checkFileAccess(FileSystem fs, Iterator<FileStatus> statuses, EnumSet<FsAction> actions,
-                                     String user, List<String> groups)
-    throws IOException, AccessControlException {
 
     if (groups == null) {
       groups = emptyGroups;
     }
 
-    // Short-circuit for super-users.
     String superGroupName = getSuperGroupName(fs.getConf());
     if (userBelongsToSuperGroup(superGroupName, groups)) {
       LOG.info("User \"" + user + "\" belongs to super-group \"" + superGroupName + "\". " +
-          "Permission granted for actions: " + actions + ".");
+          "Permission granted for action: " + action + ".");
       return;
     }
 
-    while (statuses.hasNext()) {
-
-      FileStatus stat = statuses.next();
-      final FsPermission dirPerms = stat.getPermission();
-      final String grp = stat.getGroup();
+    final FsPermission dirPerms = stat.getPermission();
+    final String grp = stat.getGroup();
 
-      FsAction combinedAction = combine(actions);
-      if (user.equals(stat.getOwner())) {
-        if (dirPerms.getUserAction().implies(combinedAction)) {
-          continue;
-        }
-      } else if (groups.contains(grp)) {
-        if (dirPerms.getGroupAction().implies(combinedAction)) {
-          continue;
-        }
-      } else if (dirPerms.getOtherAction().implies(combinedAction)) {
-        continue;
+    if (user.equals(stat.getOwner())) {
+      if (dirPerms.getUserAction().implies(action)) {
+        return;
       }
-
-      throw new AccessControlException("action " + combinedAction + " not permitted on path "
-          + stat.getPath() + " for user " + user);
-
-    } // for_each(fileStatus);
-  }
-
-  private static FsAction combine(EnumSet<FsAction> actions) {
-    FsAction resultantAction = FsAction.NONE;
-    for (FsAction action : actions) {
-      resultantAction = resultantAction.or(action);
+    } else if (groups.contains(grp)) {
+      if (dirPerms.getGroupAction().implies(action)) {
+        return;
+      }
+    } else if (dirPerms.getOtherAction().implies(action)) {
+      return;
     }
-    return resultantAction;
-  }
-
-  public static void checkFileAccess(FileSystem fs, Iterator<FileStatus> statuses, EnumSet<FsAction> actions)
-    throws IOException, AccessControlException, LoginException {
-    UserGroupInformation ugi = Utils.getUGI();
-    checkFileAccess(fs, statuses, actions, ugi.getShortUserName(), Arrays.asList(ugi.getGroupNames()));
+    throw new AccessControlException("action " + action + " not permitted on path "
+        + stat.getPath() + " for user " + user);
   }
 
   private static String getSuperGroupName(Configuration configuration) {

http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
index 4b79d95..5a6bc44 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShims.java
@@ -24,17 +24,19 @@ import java.net.URI;
 import java.nio.ByteBuffer;
 import java.security.AccessControlException;
 import java.security.NoSuchAlgorithmException;
+import java.security.PrivilegedExceptionAction;
 import java.util.Comparator;
-import java.util.EnumSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.TreeMap;
 
+import javax.security.auth.login.LoginException;
 
 import com.google.common.annotations.VisibleForTesting;
 
-import com.google.common.annotations.VisibleForTesting;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.FSDataInputStream;
@@ -45,6 +47,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hive.shims.HadoopShims.StoragePolicyValue;
 import org.apache.hadoop.io.LongWritable;
 import org.apache.hadoop.mapred.ClusterStatus;
 import org.apache.hadoop.mapred.JobConf;
@@ -527,21 +530,6 @@ public interface HadoopShims {
       throws IOException, AccessControlException, Exception;
 
   /**
-   * Check if the configured UGI has access to the path for the given file system action.
-   * Method will return successfully if action is permitted. AccessControlExceptoin will
-   * be thrown if user does not have access to perform the action. Other exceptions may
-   * be thrown for non-access related errors.
-   * @param fs The FileSystem instance
-   * @param statuses The FileStatuses for the paths being checked
-   * @param actions The FsActions being checked
-   * @throws IOException
-   * @throws AccessControlException
-   * @throws Exception
-   */
-  public void checkFileAccess(FileSystem fs, Iterator<FileStatus> statuses, EnumSet<FsAction> actions)
-      throws Exception;
-
-  /**
    * Use password API (if available) to fetch credentials/password
    * @param conf
    * @param name

http://git-wip-us.apache.org/repos/asf/hive/blob/ecde4ae9/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
----------------------------------------------------------------------
diff --git a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
index 8e51c02..89d7798 100644
--- a/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
+++ b/shims/common/src/main/java/org/apache/hadoop/hive/shims/HadoopShimsSecure.java
@@ -25,9 +25,7 @@ import java.net.URI;
 import java.security.AccessControlException;
 import java.util.ArrayList;
 import java.util.Collections;
-import java.util.EnumSet;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.Set;
 
 import org.apache.commons.lang.ArrayUtils;
@@ -393,11 +391,5 @@ public abstract class HadoopShimsSecure implements HadoopShims {
   }
 
   @Override
-  public void checkFileAccess(FileSystem fs, Iterator<FileStatus> statuses, EnumSet<FsAction> action)
-      throws IOException, AccessControlException, Exception {
-    DefaultFileAccess.checkFileAccess(fs, statuses, action);
-  }
-
-  @Override
   abstract public void addDelegationTokens(FileSystem fs, Credentials cred, String uname) throws IOException;
 }