diff --git hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java index 3f5deafb2f..cb047d8cb2 100644 --- hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java +++ hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/FileOutputCommitterContainer.java @@ -367,22 +367,12 @@ private Partition constructPartition( partPath = new Path(finalLocn); } else { partPath = new Path(partLocnRoot); - int i = 0; 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); - } 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); - } // Set the location in the StorageDescriptor if (dynamicPartitioningUsed) { @@ -400,26 +390,6 @@ private Partition constructPartition( 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 partKVs, OutputJobInfo jobInfo) { Path partPath = new Path(table.getTTable().getSd().getLocation()); @@ -946,13 +916,6 @@ private void registerPartitions(JobContext context) throws IOException{ partitionsAdded = partitionsToAdd; } } - - // 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); - } - } } catch (Exception e) { if (partitionsAdded.size() > 0) {