hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Olivier Sallou (Updated) (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (HDFS-3118) wiki and hadoop templates provides wrong superusergroup property instead of supergroup
Date Tue, 20 Mar 2012 09:07:39 GMT

     [ https://issues.apache.org/jira/browse/HDFS-3118?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]

Olivier Sallou updated HDFS-3118:
---------------------------------

    Status: Patch Available  (was: Open)

--- src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java.orig	2012-03-20 09:54:33.000000000
+0100
+++ src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java	2012-03-20 09:55:13.000000000
+0100
@@ -473,7 +473,7 @@
     fsOwner = UserGroupInformation.getCurrentUser();
     LOG.info("fsOwner=" + fsOwner);
 
-    this.supergroup = conf.get("dfs.permissions.supergroup", "supergroup");
+    this.supergroup = conf.get(DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY, "supergroup");
     this.isPermissionEnabled = conf.getBoolean("dfs.permissions", true);
     LOG.info("supergroup=" + supergroup);
     LOG.info("isPermissionEnabled=" + isPermissionEnabled);
--- src/test/org/apache/hadoop/mapred/TestMapredSystemDir.java.orig	2012-03-20 09:56:37.000000000
+0100
+++ src/test/org/apache/hadoop/mapred/TestMapredSystemDir.java	2012-03-20 09:58:14.000000000
+0100
@@ -30,6 +30,7 @@
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.*;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 
 /**
  * Test if JobTracker is resilient to garbage in mapred.system.dir.
@@ -49,7 +50,7 @@
     MiniMRCluster mr = null;
     try {
       // start dfs
-      conf.set("dfs.permissions.supergroup", "supergroup");
+      conf.set(DFSConfigKeys.DFS_PERMISSIONS_SUPERUSERGROUP_KEY, "supergroup");
       conf.set("mapred.system.dir", "/mapred");
       dfs = new MiniDFSCluster(conf, 1, true, null);
       FileSystem fs = dfs.getFileSystem();
@@ -120,4 +121,4 @@
       if (mr != null) { mr.shutdown();}
     }
   }
-}
\ No newline at end of file
+}

--- src/hdfs/hdfs-default.xml.orig	2012-03-20 10:00:53.000000000 +0100
+++ src/hdfs/hdfs-default.xml	2012-03-20 10:01:04.000000000 +0100
@@ -184,7 +184,7 @@
 </property>
 
 <property>
-  <name>dfs.permissions.supergroup</name>
+  <name>dfs.permissions.superusergroup</name>
   <value>supergroup</value>
   <description>The name of the group of super-users.</description>
 </property>
--- src/docs/src/documentation/content/xdocs/hdfs_permissions_guide.xml.orig	2012-03-20 10:02:01.000000000
+0100
+++ src/docs/src/documentation/content/xdocs/hdfs_permissions_guide.xml	2012-03-20 10:02:14.000000000
+0100
@@ -227,7 +227,7 @@
 		only those things visible using "other" permissions. Additional groups may be added to
the comma-separated list.
     </li>
     
-	<li><code>dfs.permissions.supergroup = supergroup</code>
+	<li><code>dfs.permissions.superusergroup = supergroup</code>
 	<br />The name of the group of super-users.
 	</li>

--- src/docs/cn/src/documentation/content/xdocs/hdfs_permissions_guide.xml.orig	2012-03-20
10:03:17.000000000 +0100
+++ src/docs/cn/src/documentation/content/xdocs/hdfs_permissions_guide.xml	2012-03-20 10:03:30.000000000
+0100
@@ -170,7 +170,7 @@
 	<dd>
 	Web服务器使用的用户名。如果将这个参数设置为超级用户的名称,则所有Web客户就可以看到所有的信息。如果将这个参数设置为一个不使用的用户,则Web客户就只能访问到“other”权限可访问的资源了。额外的组可以加在后面,形成一个用逗号分隔的列表。
 	</dd>
-	<dt><code>dfs.permissions.supergroup = supergroup</code></dt>
+	<dt><code>dfs.permissions.superusergroup = supergroup</code></dt>
 	<dd>
 	超级用户的组名。
 	</dd>

                
> wiki and hadoop templates provides wrong superusergroup property instead of supergroup
> --------------------------------------------------------------------------------------
>
>                 Key: HDFS-3118
>                 URL: https://issues.apache.org/jira/browse/HDFS-3118
>             Project: Hadoop HDFS
>          Issue Type: Bug
>    Affects Versions: 1.0.1, 1.0.0
>         Environment: Used Debian package install
>            Reporter: Olivier Sallou
>            Priority: Minor
>
> The hdfs-site template and the wiki: http://hadoop.apache.org/hdfs/docs/current/hdfs_permissions_guide.html#The+Super-User
> refers to property dfs.permissions.superusergroup to define the group of superuser.
> However we must use the property dfs.permissions.supergroup, and not superusergroup,
to make it work.
> In file src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java, supergroup
is extracted from:
> this.supergroup = conf.get("dfs.permissions.supergroup", "supergroup");
> It does not make use of DFS_PERMISSIONS_SUPERUSERGROUP_KEY

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

       

Mime
View raw message