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 2018/06/21 17:53:13 UTC
[03/12] hive git commit: HIVE-19870: HCatalog dynamic partition query
can fail,
if the table path is managed by Sentry (Peter Vary via Marta Kuczora)
HIVE-19870: HCatalog dynamic partition query can fail, if the table path is managed by Sentry (Peter Vary via Marta Kuczora)
Project: http://git-wip-us.apache.org/repos/asf/hive/repo
Commit: http://git-wip-us.apache.org/repos/asf/hive/commit/2394e409
Tree: http://git-wip-us.apache.org/repos/asf/hive/tree/2394e409
Diff: http://git-wip-us.apache.org/repos/asf/hive/diff/2394e409
Branch: refs/heads/master-txnstats
Commit: 2394e409f87b3a857cc00d9041b354cd47c9a923
Parents: d5cb88d
Author: Peter Vary <pv...@cloudera.com>
Authored: Tue Jun 19 09:54:19 2018 +0200
Committer: Marta Kuczora <ku...@cloudera.com>
Committed: Tue Jun 19 09:54:19 2018 +0200
----------------------------------------------------------------------
.../mapreduce/FileOutputCommitterContainer.java | 60 ++++++--------------
1 file changed, 18 insertions(+), 42 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/hive/blob/2394e409/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java
----------------------------------------------------------------------
diff --git a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java
index 3f5deaf..4a76010 100644
--- a/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java
+++ b/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java
@@ -33,10 +33,10 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hive.common.FileUtils;
import org.apache.hadoop.hive.common.StatsSetupConst;
import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.io.HdfsUtils;
import org.apache.hadoop.hive.metastore.IMetaStoreClient;
import org.apache.hadoop.hive.metastore.Warehouse;
import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -323,18 +323,18 @@ class FileOutputCommitterContainer extends OutputCommitterContainer {
* @param params The parameters to store inside the partition
* @param table The Table metadata object under which this Partition will reside
* @param fs FileSystem object to operate on the underlying filesystem
- * @param grpName Group name that owns the table dir
- * @param perms FsPermission that's the default permission of the table dir.
+ * @param conf HiveConf used to access FS
+ * @param status Permission that's the default permission of the table dir.
* @return Constructed Partition metadata object
* @throws java.io.IOException
*/
private Partition constructPartition(
- JobContext context, OutputJobInfo jobInfo,
- String partLocnRoot, String dynPartPath, Map<String, String> partKVs,
- HCatSchema outputSchema, Map<String, String> params,
- Table table, FileSystem fs,
- String grpName, FsPermission perms) throws IOException {
+ JobContext context, OutputJobInfo jobInfo,
+ String partLocnRoot, String dynPartPath, Map<String, String> partKVs,
+ HCatSchema outputSchema, Map<String, String> params,
+ Table table, FileSystem fs, HiveConf conf,
+ HdfsUtils.HadoopFileStatus status) throws IOException {
Partition partition = new Partition();
partition.setDbName(table.getDbName());
@@ -371,18 +371,16 @@ class FileOutputCommitterContainer extends OutputCommitterContainer {
for (FieldSchema partKey : table.getPartitionKeys()) {
if (i++ != 0) {
fs.mkdirs(partPath); // Attempt to make the path in case it does not exist before we check
- applyGroupAndPerms(fs, partPath, perms, grpName, false);
+ HdfsUtils.setFullFileStatus(conf, status, status.getFileStatus().getGroup(), fs,
+ partPath, false);
}
partPath = constructPartialPartPath(partPath, partKey.getName().toLowerCase(), partKVs);
}
}
- // Apply the group and permissions to the leaf partition and files.
- // Need not bother in case of HDFS as permission is taken care of by setting UMask
- fs.mkdirs(partPath); // Attempt to make the path in case it does not exist before we check
- if (!ShimLoader.getHadoopShims().getHCatShim().isFileInHDFS(fs, partPath)) {
- applyGroupAndPerms(fs, partPath, perms, grpName, true);
- }
+ // Do not need to set the status on the partition directory. We will do it later recursively.
+ // See: end of the registerPartitions method
+ fs.mkdirs(partPath);
// Set the location in the StorageDescriptor
if (dynamicPartitioningUsed) {
@@ -400,26 +398,6 @@ class FileOutputCommitterContainer extends OutputCommitterContainer {
return partition;
}
- private void applyGroupAndPerms(FileSystem fs, Path dir, FsPermission permission,
- String group, boolean recursive)
- throws IOException {
- if(LOG.isDebugEnabled()) {
- LOG.debug("applyGroupAndPerms : " + dir +
- " perms: " + permission +
- " group: " + group + " recursive: " + recursive);
- }
- fs.setPermission(dir, permission);
- if (recursive) {
- for (FileStatus fileStatus : fs.listStatus(dir)) {
- if (fileStatus.isDir()) {
- applyGroupAndPerms(fs, fileStatus.getPath(), permission, group, true);
- } else {
- fs.setPermission(fileStatus.getPath(), permission);
- }
- }
- }
- }
-
private String getFinalDynamicPartitionDestination(Table table, Map<String, String> partKVs,
OutputJobInfo jobInfo) {
Path partPath = new Path(table.getTTable().getSd().getLocation());
@@ -794,21 +772,19 @@ class FileOutputCommitterContainer extends OutputCommitterContainer {
StorerInfo storer = InternalUtil.extractStorerInfo(table.getTTable().getSd(),
table.getParameters());
- FileStatus tblStat = fs.getFileStatus(tblPath);
- String grpName = tblStat.getGroup();
- FsPermission perms = tblStat.getPermission();
+ HdfsUtils.HadoopFileStatus status = new HdfsUtils.HadoopFileStatus(conf, fs, tblPath);
List<Partition> partitionsToAdd = new ArrayList<Partition>();
if (!dynamicPartitioningUsed) {
partitionsToAdd.add(constructPartition(context, jobInfo, tblPath.toString(), null,
jobInfo.getPartitionValues(), jobInfo.getOutputSchema(), getStorerParameterMap(storer),
- table, fs, grpName, perms));
+ table, fs, hiveConf, status));
} else {
for (Entry<String, Map<String, String>> entry : partitionsDiscoveredByPath.entrySet()) {
partitionsToAdd.add(constructPartition(context, jobInfo,
getPartitionRootLocation(entry.getKey(), entry.getValue().size()), entry.getKey(),
entry.getValue(), jobInfo.getOutputSchema(), getStorerParameterMap(storer), table,
- fs, grpName, perms));
+ fs, hiveConf, status));
}
}
@@ -950,9 +926,9 @@ class FileOutputCommitterContainer extends OutputCommitterContainer {
// Set permissions appropriately for each of the partitions we just created
// so as to have their permissions mimic the table permissions
for (Partition p : partitionsAdded){
- applyGroupAndPerms(fs,new Path(p.getSd().getLocation()),tblStat.getPermission(),tblStat.getGroup(),true);
+ HdfsUtils.setFullFileStatus(conf, status, status.getFileStatus().getGroup(), fs,
+ new Path(p.getSd().getLocation()), true);
}
-
}
} catch (Exception e) {
if (partitionsAdded.size() > 0) {