HDFS-6488. Support HDFS superuser in NFSv3 gateway. Contributed by Brandon Li

This commit is contained in:
Brandon Li 2015-03-06 15:19:45 -08:00
parent 27e8ea820f
commit 0f8ecb1d0c
4 changed files with 64 additions and 8 deletions

View File

@ -73,4 +73,18 @@ public class NfsConfigKeys {
public static final String NFS_METRICS_PERCENTILES_INTERVALS_KEY = "nfs.metrics.percentiles.intervals";
public static final String NFS_METRICS_PERCENTILES_INTERVALS_DEFAULT = "";
/*
* HDFS super-user is the user with the same identity as NameNode process
* itself and the super-user can do anything in that permissions checks never
* fail for the super-user. If the following property is configured, the
* superuser on NFS client can access any file on HDFS. By default, the super
* user is not configured in the gateway. Note that, even the the superuser is
* configured, "nfs.exports.allowed.hosts" still takes effect. For example,
* the superuser will not have write access to HDFS files through the gateway
* if the NFS client host is not allowed to have write access in
* "nfs.exports.allowed.hosts".
*/
public static final String NFS_SUPERUSER_KEY = "nfs.superuser";
public static final String NFS_SUPERUSER_DEFAULT = "";
}

View File

@ -34,7 +34,6 @@ import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.CreateFlag;
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.FsStatus;
import org.apache.hadoop.fs.Options;
@ -166,6 +165,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
private JvmPauseMonitor pauseMonitor;
private Nfs3HttpServer infoServer = null;
static Nfs3Metrics metrics;
private String superuser;
public RpcProgramNfs3(NfsConfiguration config, DatagramSocket registrationSocket,
boolean allowInsecurePorts) throws IOException {
@ -200,6 +200,9 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
UserGroupInformation.setConfiguration(config);
SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
superuser = config.get(NfsConfigKeys.NFS_SUPERUSER_KEY,
NfsConfigKeys.NFS_SUPERUSER_DEFAULT);
LOG.info("Configured HDFS superuser is " + superuser);
if (!enableDump) {
writeDumpDir = null;
@ -583,13 +586,18 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
}
try {
// HDFS-5804 removed supserUserClient access
attrs = writeManager.getFileAttr(dfsClient, handle, iug);
if (attrs == null) {
LOG.error("Can't get path for fileId: " + handle.getFileId());
return new ACCESS3Response(Nfs3Status.NFS3ERR_STALE);
}
if(iug.getUserName(securityHandler.getUid(), "unknown").equals(superuser)) {
int access = Nfs3Constant.ACCESS3_LOOKUP | Nfs3Constant.ACCESS3_DELETE
| Nfs3Constant.ACCESS3_EXECUTE | Nfs3Constant.ACCESS3_EXTEND
| Nfs3Constant.ACCESS3_MODIFY | Nfs3Constant.ACCESS3_READ;
return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access);
}
int access = Nfs3Utils.getAccessRightsForUserGroup(
securityHandler.getUid(), securityHandler.getGid(),
securityHandler.getAuxGids(), attrs);

View File

@ -348,6 +348,8 @@ Release 2.7.0 - UNRELEASED
HDFS-7656. Expose truncate API for HDFS httpfs. (yliu)
HDFS-6488. Support HDFS superuser in NFS gateway. (brandonli)
IMPROVEMENTS
HDFS-7752. Improve description for

View File

@ -80,14 +80,33 @@ The above are the only required configuration for the NFS gateway in non-secure
The rest of the NFS gateway configurations are optional for both secure and non-secure mode.
The AIX NFS client has a [few known issues](https://issues.apache.org/jira/browse/HDFS-6549) that prevent it from working correctly by default with the HDFS NFS Gateway. If you want to be able to access the HDFS NFS Gateway from AIX, you should set the following configuration setting to enable work-arounds for these issues:
* The AIX NFS client has a [few known issues](https://issues.apache.org/jira/browse/HDFS-6549)
that prevent it from working correctly by default with the HDFS NFS Gateway. If you want to
be able to access the HDFS NFS Gateway from AIX, you should set the following configuration
setting to enable work-arounds for these issues:
<property>
<name>nfs.aix.compatibility.mode.enabled</name>
<value>true</value>
</property>
Note that regular, non-AIX clients should NOT enable AIX compatibility mode. The work-arounds implemented by AIX compatibility mode effectively disable safeguards to ensure that listing of directory contents via NFS returns consistent results, and that all data sent to the NFS server can be assured to have been committed.
Note that regular, non-AIX clients should NOT enable AIX compatibility mode. The work-arounds
implemented by AIX compatibility mode effectively disable safeguards to ensure that listing
of directory contents via NFS returns consistent results, and that all data sent to the NFS
server can be assured to have been committed.
* HDFS super-user is the user with the same identity as NameNode process itself and
the super-user can do anything in that permissions checks never fail for the super-user.
If the following property is configured, the superuser on NFS client can access any file
on HDFS. By default, the super user is not configured in the gateway.
Note that, even the the superuser is configured, "nfs.exports.allowed.hosts" still takes effect.
For example, the superuser will not have write access to HDFS files through the gateway if
the NFS client host is not allowed to have write access in "nfs.exports.allowed.hosts".
<property>
<name>nfs.superuser</name>
<value>the_name_of_hdfs_superuser</value>
</property>
It's strongly recommended for the users to update a few configuration properties based on their use cases. All the following configuration properties can be added or updated in hdfs-site.xml.
@ -135,6 +154,19 @@ It's strongly recommended for the users to update a few configuration properties
<value>* rw</value>
</property>
* HDFS super-user is the user with the same identity as NameNode process itself and
the super-user can do anything in that permissions checks never fail for the super-user.
If the following property is configured, the superuser on NFS client can access any file
on HDFS. By default, the super user is not configured in the gateway.
Note that, even the the superuser is configured, "nfs.exports.allowed.hosts" still takes effect.
For example, the superuser will not have write access to HDFS files through the gateway if
the NFS client host is not allowed to have write access in "nfs.exports.allowed.hosts".
<property>
<name>nfs.superuser</name>
<value>the_name_of_hdfs_superuser</value>
</property>
* JVM and log settings. You can export JVM settings (e.g., heap size and GC log) in
HADOOP\_NFS3\_OPTS. More NFS related settings can be found in hadoop-env.sh.
To get NFS debug trace, you can edit the log4j.property file