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..4a76010904 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 @@ -33,10 +33,10 @@ 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 @@ private String getPartitionRootLocation(String ptnLocn, int numPtnKeys) { * @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 partKVs, - HCatSchema outputSchema, Map params, - Table table, FileSystem fs, - String grpName, FsPermission perms) throws IOException { + JobContext context, OutputJobInfo jobInfo, + String partLocnRoot, String dynPartPath, Map partKVs, + HCatSchema outputSchema, Map params, + Table table, FileSystem fs, HiveConf conf, + HdfsUtils.HadoopFileStatus status) throws IOException { Partition partition = new Partition(); partition.setDbName(table.getDbName()); @@ -371,18 +371,16 @@ private Partition constructPartition( 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 @@ 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()); @@ -794,21 +772,19 @@ private void registerPartitions(JobContext context) throws IOException{ 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 partitionsToAdd = new ArrayList(); 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> 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 @@ private void registerPartitions(JobContext context) throws IOException{ // 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) {