You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/09/22 23:31:45 UTC

[04/52] [abbrv] hive git commit: HIVE-11343 Merge branch 'master' into hbase-metastore

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/itests/hive-unit/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
----------------------------------------------------------------------
diff --cc metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
index e5a32fb,39ab9e7..d2177a5
--- a/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
+++ b/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
@@@ -299,8 -338,27 +339,28 @@@ public class ObjectStore implements Raw
          " created in the thread with id: " + Thread.currentThread().getId());
    }
  
+   /**
+    * Creates the proxy used to evaluate expressions. This is here to prevent circular
+    * dependency - ql -> metastore client <-&gt metastore server -&gt ql. If server and
+    * client are split, this can be removed.
+    * @param conf Configuration.
+    * @return The partition expression proxy.
+    */
+   private static PartitionExpressionProxy createExpressionProxy(Configuration conf) {
+     String className = HiveConf.getVar(conf, HiveConf.ConfVars.METASTORE_EXPRESSION_PROXY_CLASS);
+     try {
+       @SuppressWarnings("unchecked")
+       Class<? extends PartitionExpressionProxy> clazz =
+           (Class<? extends PartitionExpressionProxy>)MetaStoreUtils.getClass(className);
+       return MetaStoreUtils.newInstance(
+           clazz, new Class<?>[0], new Object[0]);
+     } catch (MetaException e) {
+       LOG.error("Error loading PartitionExpressionProxy", e);
+       throw new RuntimeException("Error loading PartitionExpressionProxy: " + e.getMessage());
+     }
+   }
  
 +
    /**
     * Properties specified in hive-default.xml override the properties specified
     * in jpox.properties.
@@@ -3258,8 -3426,9 +3350,9 @@@
          if (roleMember.size() > 0) {
            pm.deletePersistentAll(roleMember);
          }
+         queryWrapper.close();
          // then remove all the grants
 -        List<MGlobalPrivilege> userGrants = listPrincipalGlobalGrants(
 +        List<MGlobalPrivilege> userGrants = listPrincipalMGlobalGrants(
              mRol.getRoleName(), PrincipalType.ROLE);
          if (userGrants.size() > 0) {
            pm.deletePersistentAll(userGrants);
@@@ -3344,26 -3521,27 +3445,27 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MRoleMap> listRoles(String principalName, PrincipalType principalType) {
 +  public List<MRoleMap> listMRoles(String principalName,
 +      PrincipalType principalType) {
      boolean success = false;
-     List<MRoleMap> mRoleMember = null;
+     Query query = null;
+     List<MRoleMap> mRoleMember = new ArrayList<MRoleMap>();
+ 
      try {
-       openTransaction();
        LOG.debug("Executing listRoles");
-       Query query = pm
-           .newQuery(
-               MRoleMap.class,
-               "principalName == t1 && principalType == t2");
-       query
-           .declareParameters("java.lang.String t1, java.lang.String t2");
+ 
+       openTransaction();
+       query = pm.newQuery(MRoleMap.class, "principalName == t1 && principalType == t2");
+       query.declareParameters("java.lang.String t1, java.lang.String t2");
        query.setUnique(false);
-       mRoleMember = (List<MRoleMap>) query.executeWithArray(
-           principalName, principalType.toString());
-       LOG.debug("Done executing query for listMSecurityUserRoleMap");
-       pm.retrieveAll(mRoleMember);
+       List<MRoleMap> mRoles =
+           (List<MRoleMap>) query.executeWithArray(principalName, principalType.toString());
+       pm.retrieveAll(mRoles);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityUserRoleMap");
+ 
+       mRoleMember.addAll(mRoles);
+ 
+       LOG.debug("Done retrieving all objects for listRoles");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -3372,60 -3553,18 +3477,56 @@@
  
      if (principalType == PrincipalType.USER) {
        // All users belong to public role implicitly, add that role
-       if (mRoleMember == null) {
-         mRoleMember = new ArrayList<MRoleMap>();
-       } else {
-         mRoleMember = new ArrayList<MRoleMap>(mRoleMember);
-       }
        MRole publicRole = new MRole(HiveMetaStore.PUBLIC, 0, HiveMetaStore.PUBLIC);
-       mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0,
-           null, null, false));
+       mRoleMember.add(new MRoleMap(principalName, principalType.toString(), publicRole, 0, null,
+           null, false));
      }
-     return mRoleMember;
  
+     return mRoleMember;
    }
  
 +  @Override
 +  public List<Role> listRoles(String principalName, PrincipalType principalType) {
 +    List<Role> result = new ArrayList<Role>();
 +    List<MRoleMap> roleMaps = listMRoles(principalName, principalType);
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        MRole mrole = roleMap.getRole();
 +        Role role = new Role(mrole.getRoleName(), mrole.getCreateTime(), mrole.getOwnerName());
 +        result.add(role);
 +      }
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public List<RolePrincipalGrant> listRolesWithGrants(String principalName,
 +                                                      PrincipalType principalType) {
 +    List<RolePrincipalGrant> result = new ArrayList<RolePrincipalGrant>();
 +    List<MRoleMap> roleMaps = listMRoles(principalName, principalType);
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        RolePrincipalGrant rolePrinGrant = new RolePrincipalGrant(
 +            roleMap.getRole().getRoleName(),
 +            roleMap.getPrincipalName(),
 +            PrincipalType.valueOf(roleMap.getPrincipalType()),
 +            roleMap.getGrantOption(),
 +            roleMap.getAddTime(),
 +            roleMap.getGrantor(),
 +            // no grantor type for public role, hence the null check
 +            roleMap.getGrantorType() == null ? null
 +                : PrincipalType.valueOf(roleMap.getGrantorType())
 +        );
 +        result.add(rolePrinGrant);
 +      }
 +    }
 +    return result;
 +  }
 +
    @SuppressWarnings("unchecked")
    private List<MRoleMap> listMSecurityPrincipalMembershipRole(final String roleName,
-       final PrincipalType principalType) {
+       final PrincipalType principalType,
+       QueryWrapper queryWrapper) {
      boolean success = false;
      List<MRoleMap> mRoleMemebership = null;
      try {
@@@ -4305,22 -4448,25 +4411,24 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MRoleMap> listRoleMembers(String roleName) {
 +  public List<MRoleMap> listMRoleMembers(String roleName) {
      boolean success = false;
-     List<MRoleMap> mRoleMemeberList = null;
+     Query query = null;
+     List<MRoleMap> mRoleMemeberList = new ArrayList<MRoleMap>();
      try {
+       LOG.debug("Executing listRoleMembers");
+ 
        openTransaction();
-       LOG.debug("Executing listMSecurityUserRoleMember");
-       Query query = pm.newQuery(MRoleMap.class,
-           "role.roleName == t1");
+       query = pm.newQuery(MRoleMap.class, "role.roleName == t1");
        query.declareParameters("java.lang.String t1");
        query.setUnique(false);
-       mRoleMemeberList = (List<MRoleMap>) query.execute(
-           roleName);
-       LOG.debug("Done executing query for listMSecurityUserRoleMember");
-       pm.retrieveAll(mRoleMemeberList);
+       List<MRoleMap> mRoles = (List<MRoleMap>) query.execute(roleName);
+       pm.retrieveAll(mRoles);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityUserRoleMember");
+ 
+       mRoleMemeberList.addAll(mRoles);
+ 
+       LOG.debug("Done retrieving all objects for listRoleMembers");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -4329,48 -4478,27 +4440,51 @@@
      return mRoleMemeberList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MGlobalPrivilege> listPrincipalGlobalGrants(String principalName,
 -      PrincipalType principalType) {
 +  public List<RolePrincipalGrant> listRoleMembers(String roleName) {
 +    List<MRoleMap> roleMaps = listMRoleMembers(roleName);
 +    List<RolePrincipalGrant> rolePrinGrantList = new ArrayList<RolePrincipalGrant>();
 +
 +    if (roleMaps != null) {
 +      for (MRoleMap roleMap : roleMaps) {
 +        RolePrincipalGrant rolePrinGrant = new RolePrincipalGrant(
 +            roleMap.getRole().getRoleName(),
 +            roleMap.getPrincipalName(),
 +            PrincipalType.valueOf(roleMap.getPrincipalType()),
 +            roleMap.getGrantOption(),
 +            roleMap.getAddTime(),
 +            roleMap.getGrantor(),
 +            // no grantor type for public role, hence the null check
 +            roleMap.getGrantorType() == null ? null
 +                : PrincipalType.valueOf(roleMap.getGrantorType())
 +        );
 +        rolePrinGrantList.add(rolePrinGrant);
 +
 +      }
 +    }
 +    return rolePrinGrantList;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MGlobalPrivilege> listPrincipalMGlobalGrants(String principalName,
 +                                                           PrincipalType principalType) {
      boolean commited = false;
-     List<MGlobalPrivilege> userNameDbPriv = null;
+     Query query = null;
+     List<MGlobalPrivilege> userNameDbPriv = new ArrayList<MGlobalPrivilege>();
      try {
+       List<MGlobalPrivilege> mPrivs = null;
        openTransaction();
        if (principalName != null) {
-         Query query = pm.newQuery(MGlobalPrivilege.class,
-             "principalName == t1 && principalType == t2 ");
-         query.declareParameters(
-             "java.lang.String t1, java.lang.String t2");
-         userNameDbPriv = (List<MGlobalPrivilege>) query
-             .executeWithArray(principalName, principalType.toString());
-         pm.retrieveAll(userNameDbPriv);
+         query = pm.newQuery(MGlobalPrivilege.class, "principalName == t1 && principalType == t2 ");
+         query.declareParameters("java.lang.String t1, java.lang.String t2");
+         mPrivs = (List<MGlobalPrivilege>) query
+                 .executeWithArray(principalName, principalType.toString());
+         pm.retrieveAll(mPrivs);
        }
        commited = commitTransaction();
+       if (mPrivs != null) {
+         userNameDbPriv.addAll(mPrivs);
+       }
      } finally {
        if (!commited) {
          rollbackTransaction();
@@@ -4380,34 -4511,12 +4497,35 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalGlobalGrants(String principalName,
 +                                                             PrincipalType principalType) {
 +    List<MGlobalPrivilege> mUsers =
 +        listPrincipalMGlobalGrants(principalName, principalType);
 +    if (mUsers.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mUsers.size(); i++) {
 +      MGlobalPrivilege sUsr = mUsers.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.GLOBAL, null, null, null, null);
 +      HiveObjectPrivilege secUser = new HiveObjectPrivilege(
 +          objectRef, sUsr.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sUsr.getPrivilege(), sUsr
 +              .getCreateTime(), sUsr.getGrantor(), PrincipalType
 +              .valueOf(sUsr.getGrantorType()), sUsr.getGrantOption()));
 +      result.add(secUser);
 +    }
 +    return result;
 +  }
 +
 +  @Override
    public List<HiveObjectPrivilege> listGlobalGrantsAll() {
      boolean commited = false;
+     Query query = null;
      try {
        openTransaction();
-       Query query = pm.newQuery(MGlobalPrivilege.class);
+       query = pm.newQuery(MGlobalPrivilege.class);
        List<MGlobalPrivilege> userNameDbPriv = (List<MGlobalPrivilege>) query.execute();
        pm.retrieveAll(userNameDbPriv);
        commited = commitTransaction();
@@@ -4435,23 -4547,28 +4556,27 @@@
    }
  
    @SuppressWarnings("unchecked")
 -  @Override
 -  public List<MDBPrivilege> listPrincipalDBGrants(String principalName,
 +  public List<MDBPrivilege> listPrincipalMDBGrants(String principalName,
        PrincipalType principalType, String dbName) {
      boolean success = false;
-     List<MDBPrivilege> mSecurityDBList = null;
+     Query query = null;
+     List<MDBPrivilege> mSecurityDBList = new ArrayList<MDBPrivilege>();
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
      try {
-       openTransaction();
        LOG.debug("Executing listPrincipalDBGrants");
-         Query query = pm.newQuery(MDBPrivilege.class,
-             "principalName == t1 && principalType == t2 && database.name == t3");
-         query
-             .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
-         mSecurityDBList = (List<MDBPrivilege>) query.executeWithArray(principalName, principalType.toString(), dbName);
-       LOG.debug("Done executing query for listPrincipalDBGrants");
-       pm.retrieveAll(mSecurityDBList);
+ 
+       openTransaction();
+       query =
+           pm.newQuery(MDBPrivilege.class,
+               "principalName == t1 && principalType == t2 && database.name == t3");
+       query.declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3");
+       List<MDBPrivilege> mPrivs =
+           (List<MDBPrivilege>) query.executeWithArray(principalName, principalType.toString(),
+               dbName);
+       pm.retrieveAll(mPrivs);
        success = commitTransaction();
+ 
+       mSecurityDBList.addAll(mPrivs);
        LOG.debug("Done retrieving all objects for listPrincipalDBGrants");
      } finally {
        if (!success) {
@@@ -4462,32 -4582,14 +4590,37 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalDBGrants(String principalName,
 +                                                         PrincipalType principalType,
 +                                                         String dbName) {
 +    List<MDBPrivilege> mDbs = listPrincipalMDBGrants(principalName, principalType, dbName);
 +    if (mDbs.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege>emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mDbs.size(); i++) {
 +      MDBPrivilege sDB = mDbs.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.DATABASE, dbName, null, null, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sDB.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sDB.getPrivilege(), sDB
 +              .getCreateTime(), sDB.getGrantor(), PrincipalType
 +              .valueOf(sDB.getGrantorType()), sDB.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @Override
    public List<HiveObjectPrivilege> listPrincipalDBGrantsAll(
        String principalName, PrincipalType principalType) {
-     return convertDB(listPrincipalAllDBGrant(principalName, principalType));
+     QueryWrapper queryWrapper = new QueryWrapper();
+     try {
+       return convertDB(listPrincipalAllDBGrant(principalName, principalType, queryWrapper));
+     } finally {
+       queryWrapper.close();
+     }
    }
  
    @Override
@@@ -4781,15 -4904,15 +4935,15 @@@
      return new ObjectPair<Query, Object[]>(query, params);
    }
  
 -  @Override
    @SuppressWarnings("unchecked")
 -  public List<MTablePrivilege> listAllTableGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName) {
 +  public List<MTablePrivilege> listAllMTableGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName) {
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
      boolean success = false;
-     List<MTablePrivilege> mSecurityTabPartList = null;
+     Query query = null;
+     List<MTablePrivilege> mSecurityTabPartList = new ArrayList<MTablePrivilege>();
      try {
        openTransaction();
        LOG.debug("Executing listAllTableGrants");
@@@ -4813,57 -4941,35 +4972,60 @@@
      return mSecurityTabPartList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MPartitionPrivilege> listPrincipalPartitionGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String partName) {
 +  public List<HiveObjectPrivilege> listAllTableGrants(String principalName,
 +                                                      PrincipalType principalType,
 +                                                      String dbName,
 +                                                      String tableName) {
 +    List<MTablePrivilege> mTbls =
 +        listAllMTableGrants(principalName, principalType, dbName, tableName);
 +    if (mTbls.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mTbls.size(); i++) {
 +      MTablePrivilege sTbl = mTbls.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.TABLE, dbName, tableName, null, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sTbl.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sTbl.getPrivilege(), sTbl.getCreateTime(), sTbl
 +              .getGrantor(), PrincipalType.valueOf(sTbl
 +              .getGrantorType()), sTbl.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MPartitionPrivilege> listPrincipalMPartitionGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String partName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
- 
-     List<MPartitionPrivilege> mSecurityTabPartList = null;
+     List<MPartitionPrivilege> mSecurityTabPartList = new ArrayList<MPartitionPrivilege>();
      try {
-       openTransaction();
-       LOG.debug("Executing listMSecurityPrincipalPartitionGrant");
-       Query query = pm.newQuery(
-           MPartitionPrivilege.class,
-               "principalName == t1 && principalType == t2 && partition.table.tableName == t3 " +
-               "&& partition.table.database.name == t4 && partition.partitionName == t5");
-       query.declareParameters(
-           "java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, " +
-           "java.lang.String t5");
-       mSecurityTabPartList = (List<MPartitionPrivilege>) query
-           .executeWithArray(principalName, principalType.toString(), tableName, dbName, partName);
-       LOG.debug("Done executing query for listMSecurityPrincipalPartitionGrant");
+       LOG.debug("Executing listPrincipalPartitionGrants");
  
-       pm.retrieveAll(mSecurityTabPartList);
+       openTransaction();
+       query =
+           pm.newQuery(MPartitionPrivilege.class,
+               "principalName == t1 && principalType == t2 && partition.table.tableName == t3 "
+                   + "&& partition.table.database.name == t4 && partition.partitionName == t5");
+       query
+           .declareParameters("java.lang.String t1, java.lang.String t2, java.lang.String t3, java.lang.String t4, "
+               + "java.lang.String t5");
+       List<MPartitionPrivilege> mPrivs =
+           (List<MPartitionPrivilege>) query.executeWithArray(principalName,
+               principalType.toString(), tableName, dbName, partName);
+       pm.retrieveAll(mPrivs);
        success = commitTransaction();
-       LOG.debug("Done retrieving all objects for listMSecurityPrincipalPartitionGrant");
+ 
+       mSecurityTabPartList.addAll(mPrivs);
+ 
+       LOG.debug("Done retrieving all objects for listPrincipalPartitionGrants");
      } finally {
        if (!success) {
          rollbackTransaction();
@@@ -4872,40 -4981,12 +5037,41 @@@
      return mSecurityTabPartList;
    }
  
 -  @SuppressWarnings("unchecked")
    @Override
 -  public List<MTableColumnPrivilege> listPrincipalTableColumnGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String columnName) {
 +  public List<HiveObjectPrivilege> listPrincipalPartitionGrants(String principalName,
 +                                                                PrincipalType principalType,
 +                                                                String dbName,
 +                                                                String tableName,
 +                                                                List<String> partValues,
 +                                                                String partName) {
 +    List<MPartitionPrivilege> mParts = listPrincipalMPartitionGrants(principalName,
 +        principalType, dbName, tableName, partName);
 +    if (mParts.isEmpty()) {
 +      return Collections.<HiveObjectPrivilege> emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mParts.size(); i++) {
 +      MPartitionPrivilege sPart = mParts.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.PARTITION, dbName, tableName, partValues, null);
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sPart.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sPart.getPrivilege(), sPart
 +              .getCreateTime(), sPart.getGrantor(), PrincipalType
 +              .valueOf(sPart.getGrantorType()), sPart
 +              .getGrantOption()));
 +
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @SuppressWarnings("unchecked")
 +  public List<MTableColumnPrivilege> listPrincipalMTableColumnGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String columnName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
      columnName = HiveStringUtils.normalizeIdentifier(columnName);
@@@ -4935,37 -5022,12 +5107,38 @@@
    }
  
    @Override
 +  public List<HiveObjectPrivilege> listPrincipalTableColumnGrants(String principalName,
 +                                                                  PrincipalType principalType,
 +                                                                  String dbName,
 +                                                                  String tableName,
 +                                                                  String columnName) {
 +    List<MTableColumnPrivilege> mTableCols =
 +        listPrincipalMTableColumnGrants(principalName, principalType, dbName, tableName, columnName);
 +    if (mTableCols.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mTableCols.size(); i++) {
 +      MTableColumnPrivilege sCol = mTableCols.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.COLUMN, dbName, tableName, null, sCol.getColumnName());
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(
 +          objectRef, sCol.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sCol.getPrivilege(), sCol
 +              .getCreateTime(), sCol.getGrantor(), PrincipalType
 +              .valueOf(sCol.getGrantorType()), sCol
 +              .getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
    @SuppressWarnings("unchecked")
 -  public List<MPartitionColumnPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 -      PrincipalType principalType, String dbName, String tableName, String partitionName,
 -      String columnName) {
 +  public List<MPartitionColumnPrivilege> listPrincipalMPartitionColumnGrants(
 +      String principalName, PrincipalType principalType, String dbName,
 +      String tableName, String partitionName, String columnName) {
      boolean success = false;
+     Query query = null;
      tableName = HiveStringUtils.normalizeIdentifier(tableName);
      dbName = HiveStringUtils.normalizeIdentifier(dbName);
      columnName = HiveStringUtils.normalizeIdentifier(columnName);
@@@ -5001,38 -5063,10 +5174,39 @@@
    }
  
    @Override
 -  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(String principalName,
 -      PrincipalType principalType) {
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrants(String principalName,
 +                                                                      PrincipalType principalType,
 +                                                                      String dbName,
 +                                                                      String tableName,
 +                                                                      List<String> partValues,
 +                                                                      String partitionName,
 +                                                                      String columnName) {
 +    List<MPartitionColumnPrivilege> mPartitionCols =
 +        listPrincipalMPartitionColumnGrants(principalName, principalType, dbName, tableName,
 +            partitionName, columnName);
 +    if (mPartitionCols.isEmpty()) {
 +      return Collections.emptyList();
 +    }
 +    List<HiveObjectPrivilege> result = new ArrayList<HiveObjectPrivilege>();
 +    for (int i = 0; i < mPartitionCols.size(); i++) {
 +      MPartitionColumnPrivilege sCol = mPartitionCols.get(i);
 +      HiveObjectRef objectRef = new HiveObjectRef(
 +          HiveObjectType.COLUMN, dbName, tableName, partValues, sCol.getColumnName());
 +      HiveObjectPrivilege secObj = new HiveObjectPrivilege(objectRef,
 +          sCol.getPrincipalName(), principalType,
 +          new PrivilegeGrantInfo(sCol.getPrivilege(), sCol
 +              .getCreateTime(), sCol.getGrantor(), PrincipalType
 +              .valueOf(sCol.getGrantorType()), sCol.getGrantOption()));
 +      result.add(secObj);
 +    }
 +    return result;
 +  }
 +
 +  @Override
 +  public List<HiveObjectPrivilege> listPrincipalPartitionColumnGrantsAll(
 +      String principalName, PrincipalType principalType) {
      boolean success = false;
+     Query query = null;
      try {
        openTransaction();
        LOG.debug("Executing listPrincipalPartitionColumnGrantsAll");
@@@ -6294,16 -6400,11 +6540,16 @@@
      }.run(true);
    }
  
 -  private List<MPartitionColumnStatistics> getMPartitionColumnStatistics(Table table,
 -      List<String> partNames, List<String> colNames,
 -      QueryWrapper queryWrapper) throws NoSuchObjectException, MetaException {
 +  @Override
 +  public void flushCache() {
 +    // NOP as there's no caching
 +  }
 +
 +  private List<MPartitionColumnStatistics> getMPartitionColumnStatistics(
-       Table table, List<String> partNames, List<String> colNames)
++      Table table, List<String> partNames, List<String> colNames, QueryWrapper queryWrapper)
 +          throws NoSuchObjectException, MetaException {
      boolean committed = false;
-     MPartitionColumnStatistics mStatsObj = null;
+ 
      try {
        openTransaction();
        // We are not going to verify SD for each partition. Just verify for the table.

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/pom.xml
----------------------------------------------------------------------
diff --cc pom.xml
index 7d763fe,1abf738..e9c054a
--- a/pom.xml
+++ b/pom.xml
@@@ -172,7 -174,7 +174,8 @@@
      <felix.version>2.4.0</felix.version>
      <curator.version>2.6.0</curator.version>
      <jsr305.version>3.0.0</jsr305.version>
 +    <tephra.version>0.4.0</tephra.version>
+     <gson.version>2.2.4</gson.version>
    </properties>
  
    <repositories>

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/Driver.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/hive/blob/61db7b80/ql/src/java/org/apache/hadoop/hive/ql/metadata/Partition.java
----------------------------------------------------------------------