HDFS-6056. Clean up NFS config settings. Contributed by Brandon Li
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1598782 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
12cf17555f
commit
42391d260d
@ -423,7 +423,9 @@ Map<String, String> getReverseDeprecatedKeyMap() {
|
||||
new DeprecationDelta("fs.default.name",
|
||||
CommonConfigurationKeys.FS_DEFAULT_NAME_KEY),
|
||||
new DeprecationDelta("dfs.umaskmode",
|
||||
CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY)
|
||||
CommonConfigurationKeys.FS_PERMISSIONS_UMASK_KEY),
|
||||
new DeprecationDelta("dfs.nfs.exports.allowed.hosts",
|
||||
CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY)
|
||||
};
|
||||
|
||||
/**
|
||||
|
@ -265,4 +265,10 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
|
||||
public static final boolean RPC_METRICS_QUANTILE_ENABLE_DEFAULT = false;
|
||||
public static final String RPC_METRICS_PERCENTILES_INTERVALS_KEY =
|
||||
"rpc.metrics.percentiles.intervals";
|
||||
|
||||
/** Allowed hosts for nfs exports */
|
||||
public static final String NFS_EXPORTS_ALLOWED_HOSTS_SEPARATOR = ";";
|
||||
public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY = "nfs.exports.allowed.hosts";
|
||||
public static final String NFS_EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT = "* rw";
|
||||
|
||||
}
|
||||
|
@ -1309,18 +1309,17 @@
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs3.server.port</name>
|
||||
<value>2049</value>
|
||||
<name>nfs.exports.allowed.hosts</name>
|
||||
<value>* rw</value>
|
||||
<description>
|
||||
Specify the port number used by Hadoop NFS.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs3.mountd.port</name>
|
||||
<value>4242</value>
|
||||
<description>
|
||||
Specify the port number used by Hadoop mount daemon.
|
||||
By default, the export can be mounted by any client. The value string
|
||||
contains machine name and access privilege, separated by whitespace
|
||||
characters. The machine name format can be a single host, a Java regular
|
||||
expression, or an IPv4 address. The access privilege uses rw or ro to
|
||||
specify read/write or read-only access of the machines to exports. If the
|
||||
access privilege is not provided, the default is read-only. Entries are separated by ";".
|
||||
For example: "192.168.0.0/22 rw ; host.*\.example\.com ; host1.test.org ro;".
|
||||
Only the NFS gateway needs to restart after this property is updated.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
|
@ -27,6 +27,7 @@
|
||||
import org.apache.commons.net.util.SubnetUtils;
|
||||
import org.apache.commons.net.util.SubnetUtils.SubnetInfo;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.apache.hadoop.util.LightWeightCache;
|
||||
import org.apache.hadoop.util.LightWeightGSet;
|
||||
@ -44,13 +45,14 @@ public class NfsExports {
|
||||
|
||||
public static synchronized NfsExports getInstance(Configuration conf) {
|
||||
if (exports == null) {
|
||||
String matchHosts = conf.get(Nfs3Constant.EXPORTS_ALLOWED_HOSTS_KEY,
|
||||
Nfs3Constant.EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT);
|
||||
int cacheSize = conf.getInt(Nfs3Constant.EXPORTS_CACHE_SIZE_KEY,
|
||||
Nfs3Constant.EXPORTS_CACHE_SIZE_DEFAULT);
|
||||
String matchHosts = conf.get(
|
||||
CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY,
|
||||
CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT);
|
||||
int cacheSize = conf.getInt(Nfs3Constant.NFS_EXPORTS_CACHE_SIZE_KEY,
|
||||
Nfs3Constant.NFS_EXPORTS_CACHE_SIZE_DEFAULT);
|
||||
long expirationPeriodNano = conf.getLong(
|
||||
Nfs3Constant.EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY,
|
||||
Nfs3Constant.EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT) * 1000 * 1000;
|
||||
Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY,
|
||||
Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT) * 1000 * 1000;
|
||||
exports = new NfsExports(cacheSize, expirationPeriodNano, matchHosts);
|
||||
}
|
||||
return exports;
|
||||
@ -140,7 +142,7 @@ public long getExpirationTime() {
|
||||
accessCache = new LightWeightCache<AccessCacheEntry, AccessCacheEntry>(
|
||||
cacheSize, cacheSize, expirationPeriodNano, 0);
|
||||
String[] matchStrings = matchHosts.split(
|
||||
Nfs3Constant.EXPORTS_ALLOWED_HOSTS_SEPARATOR);
|
||||
CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_SEPARATOR);
|
||||
mMatches = new ArrayList<Match>(matchStrings.length);
|
||||
for(String mStr : matchStrings) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -50,9 +50,6 @@ public class IdUserGroup {
|
||||
static final String MAC_GET_ALL_USERS_CMD = "dscl . -list /Users UniqueID";
|
||||
static final String MAC_GET_ALL_GROUPS_CMD = "dscl . -list /Groups PrimaryGroupID";
|
||||
|
||||
// Used for finding the configured static mapping file.
|
||||
static final String NFS_STATIC_MAPPING_FILE_KEY = "dfs.nfs.static.mapping.file";
|
||||
private static final String NFS_STATIC_MAPPING_FILE_DEFAULT = "/etc/nfs.map";
|
||||
private final File staticMappingFile;
|
||||
|
||||
// Used for parsing the static mapping file.
|
||||
@ -61,11 +58,7 @@ public class IdUserGroup {
|
||||
private static final Pattern MAPPING_LINE =
|
||||
Pattern.compile("^(uid|gid)\\s+(\\d+)\\s+(\\d+)\\s*(#.*)?$");
|
||||
|
||||
// Do update every 15 minutes by default
|
||||
final static long TIMEOUT_DEFAULT = 15 * 60 * 1000; // ms
|
||||
final static long TIMEOUT_MIN = 1 * 60 * 1000; // ms
|
||||
final private long timeout;
|
||||
final static String NFS_USERUPDATE_MILLY = "hadoop.nfs.userupdate.milly";
|
||||
|
||||
// Maps for id to name map. Guarded by this object monitor lock
|
||||
private BiMap<Integer, String> uidNameMap = HashBiMap.create();
|
||||
@ -73,25 +66,21 @@ public class IdUserGroup {
|
||||
|
||||
private long lastUpdateTime = 0; // Last time maps were updated
|
||||
|
||||
public IdUserGroup() throws IOException {
|
||||
timeout = TIMEOUT_DEFAULT;
|
||||
staticMappingFile = new File(NFS_STATIC_MAPPING_FILE_DEFAULT);
|
||||
updateMaps();
|
||||
}
|
||||
|
||||
public IdUserGroup(Configuration conf) throws IOException {
|
||||
long updateTime = conf.getLong(NFS_USERUPDATE_MILLY, TIMEOUT_DEFAULT);
|
||||
long updateTime = conf.getLong(
|
||||
Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY,
|
||||
Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_DEFAULT);
|
||||
// Minimal interval is 1 minute
|
||||
if (updateTime < TIMEOUT_MIN) {
|
||||
if (updateTime < Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_MIN) {
|
||||
LOG.info("User configured user account update time is less"
|
||||
+ " than 1 minute. Use 1 minute instead.");
|
||||
timeout = TIMEOUT_MIN;
|
||||
timeout = Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_MIN;
|
||||
} else {
|
||||
timeout = updateTime;
|
||||
}
|
||||
|
||||
String staticFilePath = conf.get(NFS_STATIC_MAPPING_FILE_KEY,
|
||||
NFS_STATIC_MAPPING_FILE_DEFAULT);
|
||||
String staticFilePath = conf.get(Nfs3Constant.NFS_STATIC_MAPPING_FILE_KEY,
|
||||
Nfs3Constant.NFS_STATIC_MAPPING_FILE_DEFAULT);
|
||||
staticMappingFile = new File(staticFilePath);
|
||||
|
||||
updateMaps();
|
||||
|
@ -33,7 +33,6 @@
|
||||
public abstract class Nfs3Base {
|
||||
public static final Log LOG = LogFactory.getLog(Nfs3Base.class);
|
||||
private final RpcProgram rpcProgram;
|
||||
private final int nfsPort;
|
||||
private int nfsBoundPort; // Will set after server starts
|
||||
|
||||
public RpcProgram getRpcProgram() {
|
||||
@ -42,9 +41,7 @@ public RpcProgram getRpcProgram() {
|
||||
|
||||
protected Nfs3Base(RpcProgram rpcProgram, Configuration conf) {
|
||||
this.rpcProgram = rpcProgram;
|
||||
this.nfsPort = conf.getInt(Nfs3Constant.NFS3_SERVER_PORT,
|
||||
Nfs3Constant.NFS3_SERVER_PORT_DEFAULT);
|
||||
LOG.info("NFS server port set to: " + nfsPort);
|
||||
LOG.info("NFS server port set to: " + rpcProgram.getPort());
|
||||
}
|
||||
|
||||
public void start(boolean register) {
|
||||
@ -58,7 +55,7 @@ public void start(boolean register) {
|
||||
}
|
||||
|
||||
private void startTCPServer() {
|
||||
SimpleTcpServer tcpServer = new SimpleTcpServer(nfsPort,
|
||||
SimpleTcpServer tcpServer = new SimpleTcpServer(rpcProgram.getPort(),
|
||||
rpcProgram, 0);
|
||||
rpcProgram.startDaemons();
|
||||
tcpServer.run();
|
||||
|
@ -25,10 +25,6 @@ public class Nfs3Constant {
|
||||
// The local rpcbind/portmapper port.
|
||||
public final static int SUN_RPCBIND = 111;
|
||||
|
||||
// The IP port number for NFS.
|
||||
public final static String NFS3_SERVER_PORT = "nfs3.server.port";
|
||||
public final static int NFS3_SERVER_PORT_DEFAULT = 2049;
|
||||
|
||||
// The RPC program number for NFS.
|
||||
public final static int PROGRAM = 100003;
|
||||
|
||||
@ -191,36 +187,22 @@ public static WriteStableHow fromValue(int id) {
|
||||
public final static int CREATE_GUARDED = 1;
|
||||
public final static int CREATE_EXCLUSIVE = 2;
|
||||
|
||||
public static final String EXPORTS_ALLOWED_HOSTS_SEPARATOR = ";";
|
||||
/** Allowed hosts for nfs exports */
|
||||
public static final String EXPORTS_ALLOWED_HOSTS_KEY = "dfs.nfs.exports.allowed.hosts";
|
||||
public static final String EXPORTS_ALLOWED_HOSTS_KEY_DEFAULT = "* rw";
|
||||
/** Size for nfs exports cache */
|
||||
public static final String EXPORTS_CACHE_SIZE_KEY = "dfs.nfs.exports.cache.size";
|
||||
public static final int EXPORTS_CACHE_SIZE_DEFAULT = 512;
|
||||
public static final String NFS_EXPORTS_CACHE_SIZE_KEY = "nfs.exports.cache.size";
|
||||
public static final int NFS_EXPORTS_CACHE_SIZE_DEFAULT = 512;
|
||||
/** Expiration time for nfs exports cache entry */
|
||||
public static final String EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY = "dfs.nfs.exports.cache.expirytime.millis";
|
||||
public static final long EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 15 * 60 * 1000; // 15 min
|
||||
public static final String NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY = "nfs.exports.cache.expirytime.millis";
|
||||
public static final long NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT = 15 * 60 * 1000; // 15 min
|
||||
|
||||
public static final String FILE_DUMP_DIR_KEY = "dfs.nfs3.dump.dir";
|
||||
public static final String FILE_DUMP_DIR_DEFAULT = "/tmp/.hdfs-nfs";
|
||||
public static final String ENABLE_FILE_DUMP_KEY = "dfs.nfs3.enableDump";
|
||||
public static final boolean ENABLE_FILE_DUMP_DEFAULT = true;
|
||||
public static final String MAX_READ_TRANSFER_SIZE_KEY = "dfs.nfs.rtmax";
|
||||
public static final int MAX_READ_TRANSFER_SIZE_DEFAULT = 1024 * 1024;
|
||||
public static final String MAX_WRITE_TRANSFER_SIZE_KEY = "dfs.nfs.wtmax";
|
||||
public static final int MAX_WRITE_TRANSFER_SIZE_DEFAULT = 1024 * 1024;
|
||||
public static final String MAX_READDIR_TRANSFER_SIZE_KEY = "dfs.nfs.dtmax";
|
||||
public static final int MAX_READDIR_TRANSFER_SIZE_DEFAULT = 64 * 1024;
|
||||
public static final String MAX_OPEN_FILES = "dfs.nfs3.max.open.files";
|
||||
public static final int MAX_OPEN_FILES_DEFAULT = 256;
|
||||
public static final String OUTPUT_STREAM_TIMEOUT = "dfs.nfs3.stream.timeout";
|
||||
public static final long OUTPUT_STREAM_TIMEOUT_DEFAULT = 10 * 60 * 1000; // 10 minutes
|
||||
public static final long OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT = 10 * 1000; //10 seconds
|
||||
/** Do user/group update every 15 minutes by default, minimum 1 minute */
|
||||
public final static String NFS_USERGROUP_UPDATE_MILLIS_KEY = "nfs.usergroup.update.millis";
|
||||
public final static long NFS_USERGROUP_UPDATE_MILLIS_DEFAULT = 15 * 60 * 1000; // ms
|
||||
final static long NFS_USERGROUP_UPDATE_MILLIS_MIN = 1 * 60 * 1000; // ms
|
||||
|
||||
public final static String UNKNOWN_USER = "nobody";
|
||||
public final static String UNKNOWN_GROUP = "nobody";
|
||||
|
||||
public final static String EXPORT_POINT = "dfs.nfs3.export.point";
|
||||
public final static String EXPORT_POINT_DEFAULT = "/";
|
||||
// Used for finding the configured static mapping file.
|
||||
public static final String NFS_STATIC_MAPPING_FILE_KEY = "nfs.static.mapping.file";
|
||||
public static final String NFS_STATIC_MAPPING_FILE_DEFAULT = "/etc/nfs.map";
|
||||
}
|
||||
|
@ -17,11 +17,8 @@
|
||||
*/
|
||||
package org.apache.hadoop.nfs;
|
||||
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.apache.hadoop.nfs.AccessPrivilege;
|
||||
import org.apache.hadoop.nfs.NfsExports;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestNfsExports {
|
||||
@ -32,9 +29,9 @@ public class TestNfsExports {
|
||||
private final String hostname2 = "a.b.org";
|
||||
|
||||
private static final long ExpirationPeriod =
|
||||
Nfs3Constant.EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT * 1000 * 1000;
|
||||
Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_DEFAULT * 1000 * 1000;
|
||||
|
||||
private static final int CacheSize = Nfs3Constant.EXPORTS_CACHE_SIZE_DEFAULT;
|
||||
private static final int CacheSize = Nfs3Constant.NFS_EXPORTS_CACHE_SIZE_DEFAULT;
|
||||
private static final long NanosPerMillis = 1000000;
|
||||
|
||||
@Test
|
||||
|
@ -199,17 +199,19 @@ public void testIdOutOfIntegerRange() throws IOException {
|
||||
|
||||
@Test
|
||||
public void testUserUpdateSetting() throws IOException {
|
||||
IdUserGroup iug = new IdUserGroup();
|
||||
assertEquals(iug.getTimeout(), IdUserGroup.TIMEOUT_DEFAULT);
|
||||
IdUserGroup iug = new IdUserGroup(new Configuration());
|
||||
assertEquals(iug.getTimeout(),
|
||||
Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_DEFAULT);
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
conf.setLong(IdUserGroup.NFS_USERUPDATE_MILLY, 0);
|
||||
conf.setLong(Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY, 0);
|
||||
iug = new IdUserGroup(conf);
|
||||
assertEquals(iug.getTimeout(), IdUserGroup.TIMEOUT_MIN);
|
||||
assertEquals(iug.getTimeout(), Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_MIN);
|
||||
|
||||
conf.setLong(IdUserGroup.NFS_USERUPDATE_MILLY,
|
||||
IdUserGroup.TIMEOUT_DEFAULT * 2);
|
||||
conf.setLong(Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY,
|
||||
Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_DEFAULT * 2);
|
||||
iug = new IdUserGroup(conf);
|
||||
assertEquals(iug.getTimeout(), IdUserGroup.TIMEOUT_DEFAULT * 2);
|
||||
assertEquals(iug.getTimeout(),
|
||||
Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_DEFAULT * 2);
|
||||
}
|
||||
}
|
||||
|
@ -0,0 +1,57 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.nfs.conf;
|
||||
|
||||
public class NfsConfigKeys {
|
||||
|
||||
// The IP port number for NFS and mountd.
|
||||
public final static String DFS_NFS_SERVER_PORT_KEY = "nfs.server.port";
|
||||
public final static int DFS_NFS_SERVER_PORT_DEFAULT = 2049;
|
||||
public final static String DFS_NFS_MOUNTD_PORT_KEY = "nfs.mountd.port";
|
||||
public final static int DFS_NFS_MOUNTD_PORT_DEFAULT = 4242;
|
||||
|
||||
public static final String DFS_NFS_FILE_DUMP_KEY = "nfs.file.dump";
|
||||
public static final boolean DFS_NFS_FILE_DUMP_DEFAULT = true;
|
||||
public static final String DFS_NFS_FILE_DUMP_DIR_KEY = "nfs.file.dump.dir";
|
||||
public static final String DFS_NFS_FILE_DUMP_DIR_DEFAULT = "/tmp/.hdfs-nfs";
|
||||
|
||||
public static final String DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY = "nfs.rtmax";
|
||||
public static final int DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT = 1024 * 1024;
|
||||
public static final String DFS_NFS_MAX_WRITE_TRANSFER_SIZE_KEY = "nfs.wtmax";
|
||||
public static final int DFS_NFS_MAX_WRITE_TRANSFER_SIZE_DEFAULT = 1024 * 1024;
|
||||
public static final String DFS_NFS_MAX_READDIR_TRANSFER_SIZE_KEY = "nfs.dtmax";
|
||||
public static final int DFS_NFS_MAX_READDIR_TRANSFER_SIZE_DEFAULT = 64 * 1024;
|
||||
|
||||
public static final String DFS_NFS_MAX_OPEN_FILES_KEY = "nfs.max.open.files";
|
||||
public static final int DFS_NFS_MAX_OPEN_FILES_DEFAULT = 256;
|
||||
|
||||
public static final String DFS_NFS_STREAM_TIMEOUT_KEY = "nfs.stream.timeout";
|
||||
public static final long DFS_NFS_STREAM_TIMEOUT_DEFAULT = 10 * 60 * 1000; // 10 minutes
|
||||
public static final long DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT = 10 * 1000; // 10 seconds
|
||||
|
||||
public final static String DFS_NFS_EXPORT_POINT_KEY = "nfs.export.point";
|
||||
public final static String DFS_NFS_EXPORT_POINT_DEFAULT = "/";
|
||||
|
||||
public static final String DFS_NFS_KEYTAB_FILE_KEY = "nfs.keytab.file";
|
||||
public static final String DFS_NFS_KERBEROS_PRINCIPAL_KEY = "nfs.kerberos.principal";
|
||||
public static final String DFS_NFS_REGISTRATION_PORT_KEY = "nfs.registration.port";
|
||||
public static final int DFS_NFS_REGISTRATION_PORT_DEFAULT = 40; // Currently unassigned.
|
||||
public static final String DFS_NFS_ALLOW_INSECURE_PORTS_KEY = "nfs.allow.insecure.ports";
|
||||
public static final boolean DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT = true;
|
||||
}
|
@ -0,0 +1,54 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.hdfs.nfs.conf;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
|
||||
/**
|
||||
* Adds deprecated keys into the configuration.
|
||||
*/
|
||||
public class NfsConfiguration extends HdfsConfiguration {
|
||||
static {
|
||||
addDeprecatedKeys();
|
||||
}
|
||||
|
||||
private static void addDeprecatedKeys() {
|
||||
Configuration.addDeprecations(new DeprecationDelta[] {
|
||||
new DeprecationDelta("nfs3.server.port",
|
||||
NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY),
|
||||
new DeprecationDelta("nfs3.mountd.port",
|
||||
NfsConfigKeys.DFS_NFS_MOUNTD_PORT_KEY),
|
||||
new DeprecationDelta("dfs.nfs.exports.cache.expirytime.millis",
|
||||
Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY),
|
||||
new DeprecationDelta("hadoop.nfs.userupdate.milly",
|
||||
Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY),
|
||||
new DeprecationDelta("dfs.nfs3.enableDump",
|
||||
NfsConfigKeys.DFS_NFS_FILE_DUMP_KEY),
|
||||
new DeprecationDelta("dfs.nfs3.dump.dir",
|
||||
NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY),
|
||||
new DeprecationDelta("dfs.nfs3.max.open.files",
|
||||
NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY),
|
||||
new DeprecationDelta("dfs.nfs3.stream.timeout",
|
||||
NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY),
|
||||
new DeprecationDelta("dfs.nfs3.export.point",
|
||||
NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY) });
|
||||
}
|
||||
}
|
@ -20,7 +20,7 @@
|
||||
import java.io.IOException;
|
||||
import java.net.DatagramSocket;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.mount.MountdBase;
|
||||
|
||||
/**
|
||||
@ -32,13 +32,13 @@
|
||||
*/
|
||||
public class Mountd extends MountdBase {
|
||||
|
||||
public Mountd(Configuration config, DatagramSocket registrationSocket,
|
||||
public Mountd(NfsConfiguration config, DatagramSocket registrationSocket,
|
||||
boolean allowInsecurePorts) throws IOException {
|
||||
super(new RpcProgramMountd(config, registrationSocket, allowInsecurePorts));
|
||||
}
|
||||
|
||||
public static void main(String[] args) throws IOException {
|
||||
Configuration config = new Configuration();
|
||||
NfsConfiguration config = new NfsConfiguration();
|
||||
Mountd mountd = new Mountd(config, null, true);
|
||||
mountd.start(true);
|
||||
}
|
||||
|
@ -16,9 +16,6 @@
|
||||
* limitations under the License.
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.nfs.mount;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KEYTAB_FILE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.DatagramSocket;
|
||||
import java.net.InetAddress;
|
||||
@ -29,8 +26,9 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.mount.MountEntry;
|
||||
@ -39,7 +37,6 @@
|
||||
import org.apache.hadoop.nfs.AccessPrivilege;
|
||||
import org.apache.hadoop.nfs.NfsExports;
|
||||
import org.apache.hadoop.nfs.nfs3.FileHandle;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Status;
|
||||
import org.apache.hadoop.oncrpc.RpcAcceptedReply;
|
||||
import org.apache.hadoop.oncrpc.RpcCall;
|
||||
@ -66,9 +63,7 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
|
||||
public static final int VERSION_1 = 1;
|
||||
public static final int VERSION_2 = 2;
|
||||
public static final int VERSION_3 = 3;
|
||||
public static final int PORT = 4242;
|
||||
|
||||
// Need DFSClient for branch-1 to get ExtendedHdfsFileStatus
|
||||
private final DFSClient dfsClient;
|
||||
|
||||
/** Synchronized list */
|
||||
@ -79,19 +74,22 @@ public class RpcProgramMountd extends RpcProgram implements MountInterface {
|
||||
|
||||
private final NfsExports hostsMatcher;
|
||||
|
||||
public RpcProgramMountd(Configuration config, DatagramSocket registrationSocket,
|
||||
boolean allowInsecurePorts) throws IOException {
|
||||
public RpcProgramMountd(NfsConfiguration config,
|
||||
DatagramSocket registrationSocket, boolean allowInsecurePorts)
|
||||
throws IOException {
|
||||
// Note that RPC cache is not enabled
|
||||
super("mountd", "localhost", config.getInt("nfs3.mountd.port", PORT),
|
||||
PROGRAM, VERSION_1, VERSION_3, registrationSocket, allowInsecurePorts);
|
||||
super("mountd", "localhost", config.getInt(
|
||||
NfsConfigKeys.DFS_NFS_MOUNTD_PORT_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MOUNTD_PORT_DEFAULT), PROGRAM, VERSION_1,
|
||||
VERSION_3, registrationSocket, allowInsecurePorts);
|
||||
exports = new ArrayList<String>();
|
||||
exports.add(config.get(Nfs3Constant.EXPORT_POINT,
|
||||
Nfs3Constant.EXPORT_POINT_DEFAULT));
|
||||
exports.add(config.get(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY,
|
||||
NfsConfigKeys.DFS_NFS_EXPORT_POINT_DEFAULT));
|
||||
this.hostsMatcher = NfsExports.getInstance(config);
|
||||
this.mounts = Collections.synchronizedList(new ArrayList<MountEntry>());
|
||||
UserGroupInformation.setConfiguration(config);
|
||||
SecurityUtil.login(config, DFS_NFS_KEYTAB_FILE_KEY,
|
||||
DFS_NFS_KERBEROS_PRINCIPAL_KEY);
|
||||
SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
|
||||
this.dfsClient = new DFSClient(NameNode.getAddress(config), config);
|
||||
}
|
||||
|
||||
|
@ -30,10 +30,10 @@
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSInputStream;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.io.MultipleIOException;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
@ -72,7 +72,7 @@ class DFSClientCache {
|
||||
final static int DEFAULT_DFS_INPUTSTREAM_CACHE_SIZE = 1024;
|
||||
final static int DEFAULT_DFS_INPUTSTREAM_CACHE_TTL = 10 * 60;
|
||||
|
||||
private final Configuration config;
|
||||
private final NfsConfiguration config;
|
||||
|
||||
private static class DFSInputStreamCaheKey {
|
||||
final String userId;
|
||||
@ -99,11 +99,11 @@ public int hashCode() {
|
||||
}
|
||||
}
|
||||
|
||||
DFSClientCache(Configuration config) {
|
||||
DFSClientCache(NfsConfiguration config) {
|
||||
this(config, DEFAULT_DFS_CLIENT_CACHE_SIZE);
|
||||
}
|
||||
|
||||
DFSClientCache(Configuration config, int clientCache) {
|
||||
DFSClientCache(NfsConfiguration config, int clientCache) {
|
||||
this.config = config;
|
||||
this.clientCache = CacheBuilder.newBuilder()
|
||||
.maximumSize(clientCache)
|
||||
|
@ -20,8 +20,8 @@
|
||||
import java.io.IOException;
|
||||
import java.net.DatagramSocket;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.mount.Mountd;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Base;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
@ -36,16 +36,11 @@
|
||||
public class Nfs3 extends Nfs3Base {
|
||||
private Mountd mountd;
|
||||
|
||||
static {
|
||||
Configuration.addDefaultResource("hdfs-default.xml");
|
||||
Configuration.addDefaultResource("hdfs-site.xml");
|
||||
}
|
||||
|
||||
public Nfs3(Configuration conf) throws IOException {
|
||||
public Nfs3(NfsConfiguration conf) throws IOException {
|
||||
this(conf, null, true);
|
||||
}
|
||||
|
||||
public Nfs3(Configuration conf, DatagramSocket registrationSocket,
|
||||
public Nfs3(NfsConfiguration conf, DatagramSocket registrationSocket,
|
||||
boolean allowInsecurePorts) throws IOException {
|
||||
super(new RpcProgramNfs3(conf, registrationSocket, allowInsecurePorts), conf);
|
||||
mountd = new Mountd(conf, registrationSocket, allowInsecurePorts);
|
||||
@ -64,11 +59,11 @@ public void startServiceInternal(boolean register) throws IOException {
|
||||
static void startService(String[] args,
|
||||
DatagramSocket registrationSocket) throws IOException {
|
||||
StringUtils.startupShutdownMessage(Nfs3.class, args, LOG);
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
boolean allowInsecurePorts = conf.getBoolean(
|
||||
DFSConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_KEY,
|
||||
DFSConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT);
|
||||
final Nfs3 nfsServer = new Nfs3(new Configuration(), registrationSocket,
|
||||
NfsConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_KEY,
|
||||
NfsConfigKeys.DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT);
|
||||
final Nfs3 nfsServer = new Nfs3(conf, registrationSocket,
|
||||
allowInsecurePorts);
|
||||
nfsServer.startServiceInternal(true);
|
||||
}
|
||||
|
@ -37,6 +37,7 @@
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.WriteCtx.DataState;
|
||||
import org.apache.hadoop.io.BytesWritable.Comparator;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
@ -823,7 +824,7 @@ private void addWrite(WriteCtx writeCtx) {
|
||||
*/
|
||||
public synchronized boolean streamCleanup(long fileId, long streamTimeout) {
|
||||
Preconditions
|
||||
.checkState(streamTimeout >= Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
.checkState(streamTimeout >= NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
if (!activeState) {
|
||||
return true;
|
||||
}
|
||||
|
@ -24,9 +24,9 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.nfs.nfs3.FileHandle;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
@ -48,9 +48,9 @@ class OpenFileCtxCache {
|
||||
private final long streamTimeout;
|
||||
private final StreamMonitor streamMonitor;
|
||||
|
||||
OpenFileCtxCache(Configuration config, long streamTimeout) {
|
||||
maxStreams = config.getInt(Nfs3Constant.MAX_OPEN_FILES,
|
||||
Nfs3Constant.MAX_OPEN_FILES_DEFAULT);
|
||||
OpenFileCtxCache(NfsConfiguration config, long streamTimeout) {
|
||||
maxStreams = config.getInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_DEFAULT);
|
||||
LOG.info("Maximum open streams is " + maxStreams);
|
||||
this.streamTimeout = streamTimeout;
|
||||
streamMonitor = new StreamMonitor();
|
||||
@ -102,7 +102,7 @@ Entry<FileHandle, OpenFileCtx> getEntryToEvict() {
|
||||
} else {
|
||||
long idleTime = Time.monotonicNow()
|
||||
- idlest.getValue().getLastAccessTime();
|
||||
if (idleTime < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
|
||||
if (idleTime < NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("idlest stream's idle time:" + idleTime);
|
||||
}
|
||||
|
@ -21,9 +21,8 @@
|
||||
|
||||
import org.apache.commons.daemon.Daemon;
|
||||
import org.apache.commons.daemon.DaemonContext;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
|
||||
/**
|
||||
* This class is used to allow the initial registration of the NFS gateway with
|
||||
@ -42,12 +41,12 @@ public class PrivilegedNfsGatewayStarter implements Daemon {
|
||||
@Override
|
||||
public void init(DaemonContext context) throws Exception {
|
||||
System.err.println("Initializing privileged NFS client socket...");
|
||||
Configuration conf = new HdfsConfiguration();
|
||||
int clientPort = conf.getInt(DFSConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY,
|
||||
DFSConfigKeys.DFS_NFS_REGISTRATION_PORT_DEFAULT);
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
int clientPort = conf.getInt(NfsConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY,
|
||||
NfsConfigKeys.DFS_NFS_REGISTRATION_PORT_DEFAULT);
|
||||
if (clientPort < 1 || clientPort > 1023) {
|
||||
throw new RuntimeException("Must start privileged NFS server with '" +
|
||||
DFSConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY + "' configured to a " +
|
||||
NfsConfigKeys.DFS_NFS_REGISTRATION_PORT_KEY + "' configured to a " +
|
||||
"privileged port.");
|
||||
}
|
||||
registrationSocket = new DatagramSocket(
|
||||
|
@ -28,7 +28,6 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException;
|
||||
@ -41,6 +40,8 @@
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
@ -131,9 +132,6 @@
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KEYTAB_FILE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY;
|
||||
|
||||
/**
|
||||
* RPC program corresponding to nfs daemon. See {@link Nfs3}.
|
||||
*/
|
||||
@ -144,7 +142,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
||||
|
||||
static final Log LOG = LogFactory.getLog(RpcProgramNfs3.class);
|
||||
|
||||
private final Configuration config = new Configuration();
|
||||
private final NfsConfiguration config;
|
||||
private final WriteManager writeManager;
|
||||
private final IdUserGroup iug;
|
||||
private final DFSClientCache clientCache;
|
||||
@ -159,15 +157,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
||||
|
||||
private final RpcCallCache rpcCallCache;
|
||||
|
||||
public RpcProgramNfs3(Configuration config, DatagramSocket registrationSocket,
|
||||
public RpcProgramNfs3(NfsConfiguration config, DatagramSocket registrationSocket,
|
||||
boolean allowInsecurePorts) throws IOException {
|
||||
super("NFS3", "localhost", config.getInt(Nfs3Constant.NFS3_SERVER_PORT,
|
||||
Nfs3Constant.NFS3_SERVER_PORT_DEFAULT), Nfs3Constant.PROGRAM,
|
||||
super("NFS3", "localhost", config.getInt(
|
||||
NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY,
|
||||
NfsConfigKeys.DFS_NFS_SERVER_PORT_DEFAULT), Nfs3Constant.PROGRAM,
|
||||
Nfs3Constant.VERSION, Nfs3Constant.VERSION, registrationSocket,
|
||||
allowInsecurePorts);
|
||||
|
||||
this.config = config;
|
||||
config.set(FsPermission.UMASK_LABEL, "000");
|
||||
iug = new IdUserGroup();
|
||||
iug = new IdUserGroup(config);
|
||||
|
||||
exports = NfsExports.getInstance(config);
|
||||
writeManager = new WriteManager(iug, config);
|
||||
@ -180,13 +180,13 @@ public RpcProgramNfs3(Configuration config, DatagramSocket registrationSocket,
|
||||
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY,
|
||||
CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT);
|
||||
|
||||
writeDumpDir = config.get(Nfs3Constant.FILE_DUMP_DIR_KEY,
|
||||
Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
|
||||
boolean enableDump = config.getBoolean(Nfs3Constant.ENABLE_FILE_DUMP_KEY,
|
||||
Nfs3Constant.ENABLE_FILE_DUMP_DEFAULT);
|
||||
writeDumpDir = config.get(NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY,
|
||||
NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_DEFAULT);
|
||||
boolean enableDump = config.getBoolean(NfsConfigKeys.DFS_NFS_FILE_DUMP_KEY,
|
||||
NfsConfigKeys.DFS_NFS_FILE_DUMP_DEFAULT);
|
||||
UserGroupInformation.setConfiguration(config);
|
||||
SecurityUtil.login(config, DFS_NFS_KEYTAB_FILE_KEY,
|
||||
DFS_NFS_KERBEROS_PRINCIPAL_KEY);
|
||||
SecurityUtil.login(config, NfsConfigKeys.DFS_NFS_KEYTAB_FILE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_KERBEROS_PRINCIPAL_KEY);
|
||||
|
||||
if (!enableDump) {
|
||||
writeDumpDir = null;
|
||||
@ -567,8 +567,8 @@ public READLINK3Response readlink(XDR xdr, SecurityHandler securityHandler,
|
||||
+ handle.getFileId());
|
||||
return new READLINK3Response(Nfs3Status.NFS3ERR_SERVERFAULT);
|
||||
}
|
||||
int rtmax = config.getInt(Nfs3Constant.MAX_READ_TRANSFER_SIZE_KEY,
|
||||
Nfs3Constant.MAX_READ_TRANSFER_SIZE_DEFAULT);
|
||||
int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
|
||||
if (rtmax < target.getBytes().length) {
|
||||
LOG.error("Link size: " + target.getBytes().length
|
||||
+ " is larger than max transfer size: " + rtmax);
|
||||
@ -665,8 +665,8 @@ public READ3Response read(XDR xdr, SecurityHandler securityHandler,
|
||||
}
|
||||
|
||||
try {
|
||||
int rtmax = config.getInt(Nfs3Constant.MAX_READ_TRANSFER_SIZE_KEY,
|
||||
Nfs3Constant.MAX_READ_TRANSFER_SIZE_DEFAULT);
|
||||
int rtmax = config.getInt(NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
|
||||
int buffSize = Math.min(rtmax, count);
|
||||
byte[] readbuffer = new byte[buffSize];
|
||||
|
||||
@ -1740,12 +1740,15 @@ public FSINFO3Response fsinfo(XDR xdr, SecurityHandler securityHandler,
|
||||
}
|
||||
|
||||
try {
|
||||
int rtmax = config.getInt(Nfs3Constant.MAX_READ_TRANSFER_SIZE_KEY,
|
||||
Nfs3Constant.MAX_READ_TRANSFER_SIZE_DEFAULT);
|
||||
int wtmax = config.getInt(Nfs3Constant.MAX_WRITE_TRANSFER_SIZE_KEY,
|
||||
Nfs3Constant.MAX_WRITE_TRANSFER_SIZE_DEFAULT);
|
||||
int dtperf = config.getInt(Nfs3Constant.MAX_READDIR_TRANSFER_SIZE_KEY,
|
||||
Nfs3Constant.MAX_READDIR_TRANSFER_SIZE_DEFAULT);
|
||||
int rtmax = config.getInt(
|
||||
NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_READ_TRANSFER_SIZE_DEFAULT);
|
||||
int wtmax = config.getInt(
|
||||
NfsConfigKeys.DFS_NFS_MAX_WRITE_TRANSFER_SIZE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_WRITE_TRANSFER_SIZE_DEFAULT);
|
||||
int dtperf = config.getInt(
|
||||
NfsConfigKeys.DFS_NFS_MAX_READDIR_TRANSFER_SIZE_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_READDIR_TRANSFER_SIZE_DEFAULT);
|
||||
|
||||
Nfs3FileAttributes attrs = Nfs3Utils.getFileAttr(dfsClient,
|
||||
Nfs3Utils.getFileIdPath(handle), iug);
|
||||
|
@ -21,10 +21,11 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
|
||||
import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
@ -50,7 +51,7 @@
|
||||
public class WriteManager {
|
||||
public static final Log LOG = LogFactory.getLog(WriteManager.class);
|
||||
|
||||
private final Configuration config;
|
||||
private final NfsConfiguration config;
|
||||
private final IdUserGroup iug;
|
||||
|
||||
private AsyncDataService asyncDataService;
|
||||
@ -78,19 +79,19 @@ boolean addOpenFileStream(FileHandle h, OpenFileCtx ctx) {
|
||||
return fileContextCache.put(h, ctx);
|
||||
}
|
||||
|
||||
WriteManager(IdUserGroup iug, final Configuration config) {
|
||||
WriteManager(IdUserGroup iug, final NfsConfiguration config) {
|
||||
this.iug = iug;
|
||||
this.config = config;
|
||||
streamTimeout = config.getLong(Nfs3Constant.OUTPUT_STREAM_TIMEOUT,
|
||||
Nfs3Constant.OUTPUT_STREAM_TIMEOUT_DEFAULT);
|
||||
streamTimeout = config.getLong(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY,
|
||||
NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_DEFAULT);
|
||||
LOG.info("Stream timeout is " + streamTimeout + "ms.");
|
||||
if (streamTimeout < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
|
||||
if (streamTimeout < NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT) {
|
||||
LOG.info("Reset stream timeout to minimum value "
|
||||
+ Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT + "ms.");
|
||||
streamTimeout = Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT;
|
||||
+ NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT + "ms.");
|
||||
streamTimeout = NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT;
|
||||
}
|
||||
maxStreams = config.getInt(Nfs3Constant.MAX_OPEN_FILES,
|
||||
Nfs3Constant.MAX_OPEN_FILES_DEFAULT);
|
||||
maxStreams = config.getInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY,
|
||||
NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_DEFAULT);
|
||||
LOG.info("Maximum open streams is "+ maxStreams);
|
||||
this.fileContextCache = new OpenFileCtxCache(config, streamTimeout);
|
||||
}
|
||||
@ -171,8 +172,8 @@ void handleWrite(DFSClient dfsClient, WRITE3Request request, Channel channel,
|
||||
}
|
||||
|
||||
// Add open stream
|
||||
String writeDumpDir = config.get(Nfs3Constant.FILE_DUMP_DIR_KEY,
|
||||
Nfs3Constant.FILE_DUMP_DIR_DEFAULT);
|
||||
String writeDumpDir = config.get(NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY,
|
||||
NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_DEFAULT);
|
||||
openFileCtx = new OpenFileCtx(fos, latestAttr, writeDumpDir + "/"
|
||||
+ fileHandle.getFileId(), dfsClient, iug);
|
||||
|
||||
|
@ -23,8 +23,8 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3;
|
||||
@ -38,7 +38,7 @@ public class TestMountd {
|
||||
@Test
|
||||
public void testStart() throws IOException {
|
||||
// Start minicluster
|
||||
Configuration config = new Configuration();
|
||||
NfsConfiguration config = new NfsConfiguration();
|
||||
MiniDFSCluster cluster = new MiniDFSCluster.Builder(config).numDataNodes(1)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
@ -23,7 +23,8 @@
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3Utils;
|
||||
import org.apache.hadoop.nfs.nfs3.FileHandle;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
@ -155,10 +156,10 @@ public static void main(String[] args) throws InterruptedException {
|
||||
Arrays.fill(data3, (byte) 9);
|
||||
|
||||
// NFS3 Create request
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
WriteClient client = new WriteClient("localhost", conf.getInt(
|
||||
Nfs3Constant.NFS3_SERVER_PORT, Nfs3Constant.NFS3_SERVER_PORT_DEFAULT),
|
||||
create(), false);
|
||||
NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY,
|
||||
NfsConfigKeys.DFS_NFS_SERVER_PORT_DEFAULT), create(), false);
|
||||
client.run();
|
||||
|
||||
while (handle == null) {
|
||||
|
@ -24,11 +24,11 @@
|
||||
import java.net.InetAddress;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.DFSTestUtil;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.Nfs3;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.RpcProgramNfs3;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
@ -53,7 +53,7 @@
|
||||
*/
|
||||
public class TestReaddir {
|
||||
|
||||
static Configuration config = new Configuration();
|
||||
static NfsConfiguration config = new NfsConfiguration();
|
||||
static MiniDFSCluster cluster = null;
|
||||
static DistributedFileSystem hdfs;
|
||||
static NameNode nn;
|
||||
|
@ -18,24 +18,24 @@
|
||||
package org.apache.hadoop.hdfs.nfs.nfs3;
|
||||
|
||||
import static org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod.KERBEROS;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertFalse;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
import static org.junit.Assert.assertThat;
|
||||
import static org.hamcrest.core.Is.is;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestDFSClientCache {
|
||||
@Test
|
||||
public void testEviction() throws IOException {
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost");
|
||||
|
||||
// Only one entry will be in the cache
|
||||
@ -59,7 +59,7 @@ public void testGetUserGroupInformationSecure() throws IOException {
|
||||
String currentUser = "test-user";
|
||||
|
||||
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
UserGroupInformation currentUserUgi
|
||||
= UserGroupInformation.createRemoteUser(currentUser);
|
||||
currentUserUgi.setAuthenticationMethod(KERBEROS);
|
||||
@ -83,7 +83,7 @@ public void testGetUserGroupInformation() throws IOException {
|
||||
|
||||
UserGroupInformation currentUserUgi = UserGroupInformation
|
||||
.createUserForTesting(currentUser, new String[0]);
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
conf.set(FileSystem.FS_DEFAULT_NAME_KEY, "hdfs://localhost");
|
||||
DFSClientCache cache = new DFSClientCache(conf);
|
||||
UserGroupInformation ugiResult
|
||||
|
@ -21,22 +21,22 @@
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.mount.Mountd;
|
||||
import org.apache.hadoop.hdfs.nfs.mount.RpcProgramMountd;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestExportsTable {
|
||||
|
||||
@Test
|
||||
public void testExportPoint() throws IOException {
|
||||
Configuration config = new Configuration();
|
||||
NfsConfiguration config = new NfsConfiguration();
|
||||
MiniDFSCluster cluster = null;
|
||||
|
||||
String exportPoint = "/myexport1";
|
||||
config.setStrings(Nfs3Constant.EXPORT_POINT, exportPoint);
|
||||
config.setStrings(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY, exportPoint);
|
||||
// Use emphral port in case tests are running in parallel
|
||||
config.setInt("nfs3.mountd.port", 0);
|
||||
config.setInt("nfs3.server.port", 0);
|
||||
|
@ -22,13 +22,13 @@
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.CommitCtx;
|
||||
import org.apache.hadoop.nfs.nfs3.FileHandle;
|
||||
import org.apache.hadoop.nfs.nfs3.IdUserGroup;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3FileAttributes;
|
||||
import org.junit.Test;
|
||||
import org.mockito.Mockito;
|
||||
@ -38,10 +38,10 @@ public class TestOpenFileCtxCache {
|
||||
|
||||
@Test
|
||||
public void testEviction() throws IOException, InterruptedException {
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
|
||||
// Only two entries will be in the cache
|
||||
conf.setInt(Nfs3Constant.MAX_OPEN_FILES, 2);
|
||||
conf.setInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY, 2);
|
||||
|
||||
DFSClient dfsClient = Mockito.mock(DFSClient.class);
|
||||
Nfs3FileAttributes attr = new Nfs3FileAttributes();
|
||||
@ -49,15 +49,15 @@ public void testEviction() throws IOException, InterruptedException {
|
||||
Mockito.when(fos.getPos()).thenReturn((long) 0);
|
||||
|
||||
OpenFileCtx context1 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context2 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context3 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context4 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context5 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
|
||||
OpenFileCtxCache cache = new OpenFileCtxCache(conf, 10 * 60 * 100);
|
||||
|
||||
@ -71,7 +71,7 @@ public void testEviction() throws IOException, InterruptedException {
|
||||
assertTrue(cache.size() == 2);
|
||||
|
||||
// Wait for the oldest stream to be evict-able, insert again
|
||||
Thread.sleep(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
Thread.sleep(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
assertTrue(cache.size() == 2);
|
||||
|
||||
ret = cache.put(new FileHandle(3), context3);
|
||||
@ -90,17 +90,17 @@ public void testEviction() throws IOException, InterruptedException {
|
||||
new WriteCtx(null, 0, 0, 0, null, null, null, 0, false, null));
|
||||
context4.getPendingCommitsForTest().put(new Long(100),
|
||||
new CommitCtx(0, null, 0, attr));
|
||||
Thread.sleep(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
Thread.sleep(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
ret = cache.put(new FileHandle(5), context5);
|
||||
assertFalse(ret);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testScan() throws IOException, InterruptedException {
|
||||
Configuration conf = new Configuration();
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
|
||||
// Only two entries will be in the cache
|
||||
conf.setInt(Nfs3Constant.MAX_OPEN_FILES, 2);
|
||||
conf.setInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY, 2);
|
||||
|
||||
DFSClient dfsClient = Mockito.mock(DFSClient.class);
|
||||
Nfs3FileAttributes attr = new Nfs3FileAttributes();
|
||||
@ -108,13 +108,13 @@ public void testScan() throws IOException, InterruptedException {
|
||||
Mockito.when(fos.getPos()).thenReturn((long) 0);
|
||||
|
||||
OpenFileCtx context1 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context2 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context3 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
OpenFileCtx context4 = new OpenFileCtx(fos, attr, "/dumpFilePath",
|
||||
dfsClient, new IdUserGroup());
|
||||
dfsClient, new IdUserGroup(new NfsConfiguration()));
|
||||
|
||||
OpenFileCtxCache cache = new OpenFileCtxCache(conf, 10 * 60 * 100);
|
||||
|
||||
@ -123,8 +123,8 @@ public void testScan() throws IOException, InterruptedException {
|
||||
assertTrue(ret);
|
||||
ret = cache.put(new FileHandle(2), context2);
|
||||
assertTrue(ret);
|
||||
Thread.sleep(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT + 1);
|
||||
cache.scan(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
Thread.sleep(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT + 1);
|
||||
cache.scan(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_MIN_DEFAULT);
|
||||
assertTrue(cache.size() == 0);
|
||||
|
||||
// Test cleaning inactive entry
|
||||
@ -133,7 +133,7 @@ public void testScan() throws IOException, InterruptedException {
|
||||
ret = cache.put(new FileHandle(4), context4);
|
||||
assertTrue(ret);
|
||||
context3.setActiveStatusForTest(false);
|
||||
cache.scan(Nfs3Constant.OUTPUT_STREAM_TIMEOUT_DEFAULT);
|
||||
cache.scan(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_DEFAULT);
|
||||
assertTrue(cache.size() == 1);
|
||||
assertTrue(cache.get(new FileHandle(3)) == null);
|
||||
assertTrue(cache.get(new FileHandle(4)) != null);
|
||||
|
@ -17,6 +17,11 @@
|
||||
*/
|
||||
package org.apache.hadoop.hdfs.nfs.nfs3;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfigKeys;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.nfs.nfs3.Nfs3Constant;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
@ -63,4 +68,41 @@ public void testIdempotent() {
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDeprecatedKeys() {
|
||||
NfsConfiguration conf = new NfsConfiguration();
|
||||
conf.setInt("nfs3.server.port", 998);
|
||||
assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_SERVER_PORT_KEY, 0) == 998);
|
||||
|
||||
conf.setInt("nfs3.mountd.port", 999);
|
||||
assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_MOUNTD_PORT_KEY, 0) == 999);
|
||||
|
||||
conf.set("dfs.nfs.exports.allowed.hosts", "host1");
|
||||
assertTrue(conf.get(CommonConfigurationKeys.NFS_EXPORTS_ALLOWED_HOSTS_KEY)
|
||||
.equals("host1"));
|
||||
|
||||
conf.setInt("dfs.nfs.exports.cache.expirytime.millis", 1000);
|
||||
assertTrue(conf.getInt(
|
||||
Nfs3Constant.NFS_EXPORTS_CACHE_EXPIRYTIME_MILLIS_KEY, 0) == 1000);
|
||||
|
||||
conf.setInt("hadoop.nfs.userupdate.milly", 10);
|
||||
assertTrue(conf.getInt(Nfs3Constant.NFS_USERGROUP_UPDATE_MILLIS_KEY, 0) == 10);
|
||||
|
||||
conf.set("dfs.nfs3.dump.dir", "/nfs/tmp");
|
||||
assertTrue(conf.get(NfsConfigKeys.DFS_NFS_FILE_DUMP_DIR_KEY).equals(
|
||||
"/nfs/tmp"));
|
||||
|
||||
conf.setBoolean("dfs.nfs3.enableDump", false);
|
||||
assertTrue(conf.getBoolean(NfsConfigKeys.DFS_NFS_FILE_DUMP_KEY, true) == false);
|
||||
|
||||
conf.setInt("dfs.nfs3.max.open.files", 500);
|
||||
assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_MAX_OPEN_FILES_KEY, 0) == 500);
|
||||
|
||||
conf.setInt("dfs.nfs3.stream.timeout", 6000);
|
||||
assertTrue(conf.getInt(NfsConfigKeys.DFS_NFS_STREAM_TIMEOUT_KEY, 0) == 6000);
|
||||
|
||||
conf.set("dfs.nfs3.export.point", "/dir1");
|
||||
assertTrue(conf.get(NfsConfigKeys.DFS_NFS_EXPORT_POINT_KEY).equals("/dir1"));
|
||||
}
|
||||
}
|
||||
|
@ -27,11 +27,10 @@
|
||||
import java.util.Arrays;
|
||||
import java.util.concurrent.ConcurrentNavigableMap;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSClient;
|
||||
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
||||
import org.apache.hadoop.hdfs.nfs.conf.NfsConfiguration;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.COMMIT_STATUS;
|
||||
import org.apache.hadoop.hdfs.nfs.nfs3.OpenFileCtx.CommitCtx;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
@ -139,7 +138,7 @@ public void testCheckCommit() throws IOException {
|
||||
Mockito.when(fos.getPos()).thenReturn((long) 0);
|
||||
|
||||
OpenFileCtx ctx = new OpenFileCtx(fos, attr, "/dumpFilePath", dfsClient,
|
||||
new IdUserGroup());
|
||||
new IdUserGroup(new NfsConfiguration()));
|
||||
|
||||
COMMIT_STATUS ret;
|
||||
|
||||
@ -201,13 +200,14 @@ public void testCheckCommitFromRead() throws IOException {
|
||||
Nfs3FileAttributes attr = new Nfs3FileAttributes();
|
||||
HdfsDataOutputStream fos = Mockito.mock(HdfsDataOutputStream.class);
|
||||
Mockito.when(fos.getPos()).thenReturn((long) 0);
|
||||
NfsConfiguration config = new NfsConfiguration();
|
||||
|
||||
OpenFileCtx ctx = new OpenFileCtx(fos, attr, "/dumpFilePath", dfsClient,
|
||||
new IdUserGroup());
|
||||
new IdUserGroup(config));
|
||||
|
||||
FileHandle h = new FileHandle(1); // fake handle for "/dumpFilePath"
|
||||
COMMIT_STATUS ret;
|
||||
WriteManager wm = new WriteManager(new IdUserGroup(), new Configuration());
|
||||
WriteManager wm = new WriteManager(new IdUserGroup(config), config);
|
||||
assertTrue(wm.addOpenFileStream(h, ctx));
|
||||
|
||||
// Test inactive open file context
|
||||
@ -280,7 +280,7 @@ private void waitWrite(RpcProgramNfs3 nfsd, FileHandle handle, int maxWaitTime)
|
||||
|
||||
@Test
|
||||
public void testWriteStableHow() throws IOException, InterruptedException {
|
||||
HdfsConfiguration config = new HdfsConfiguration();
|
||||
NfsConfiguration config = new NfsConfiguration();
|
||||
DFSClient client = null;
|
||||
MiniDFSCluster cluster = null;
|
||||
RpcProgramNfs3 nfsd;
|
||||
|
@ -18,12 +18,12 @@
|
||||
|
||||
<configuration>
|
||||
<property>
|
||||
<name>nfs3.server.port</name>
|
||||
<name>nfs.server.port</name>
|
||||
<value>2079</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs3.mountd.port</name>
|
||||
<name>nfs.mountd.port</name>
|
||||
<value>4272</value>
|
||||
</property>
|
||||
</configuration>
|
||||
|
@ -459,6 +459,8 @@ Release 2.5.0 - UNRELEASED
|
||||
|
||||
HDFS-6472. fix typo in webapps/hdfs/explorer.js. (Juan Yu via wang)
|
||||
|
||||
HDFS-6056. Clean up NFS config settings (brandonli)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)
|
||||
|
@ -639,12 +639,6 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
||||
public static final String DFS_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE =
|
||||
"dfs.client.hedged.read.threadpool.size";
|
||||
public static final int DEFAULT_DFSCLIENT_HEDGED_READ_THREADPOOL_SIZE = 0;
|
||||
public static final String DFS_NFS_KEYTAB_FILE_KEY = "dfs.nfs.keytab.file";
|
||||
public static final String DFS_NFS_KERBEROS_PRINCIPAL_KEY = "dfs.nfs.kerberos.principal";
|
||||
public static final String DFS_NFS_REGISTRATION_PORT_KEY = "dfs.nfs.registration.port";
|
||||
public static final int DFS_NFS_REGISTRATION_PORT_DEFAULT = 40; // Currently unassigned.
|
||||
public static final String DFS_NFS_ALLOW_INSECURE_PORTS_KEY = "dfs.nfs.allow.insecure.ports";
|
||||
public static final boolean DFS_NFS_ALLOW_INSECURE_PORTS_DEFAULT = true;
|
||||
|
||||
// Slow io warning log threshold settings for dfsclient and datanode.
|
||||
public static final String DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY =
|
||||
|
@ -1296,8 +1296,56 @@
|
||||
non-zero integer.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.nfs.keytab.file</name>
|
||||
<name>nfs.server.port</name>
|
||||
<value>2049</value>
|
||||
<description>
|
||||
Specify the port number used by Hadoop NFS.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs.mountd.port</name>
|
||||
<value>4242</value>
|
||||
<description>
|
||||
Specify the port number used by Hadoop mount daemon.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs.dump.dir</name>
|
||||
<value>/tmp/.hdfs-nfs</value>
|
||||
<description>
|
||||
This directory is used to temporarily save out-of-order writes before
|
||||
writing to HDFS. For each file, the out-of-order writes are dumped after
|
||||
they are accumulated to exceed certain threshold (e.g., 1MB) in memory.
|
||||
One needs to make sure the directory has enough space.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs.rtmax</name>
|
||||
<value>1048576</value>
|
||||
<description>This is the maximum size in bytes of a READ request
|
||||
supported by the NFS gateway. If you change this, make sure you
|
||||
also update the nfs mount's rsize(add rsize= # of bytes to the
|
||||
mount directive).
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs.wtmax</name>
|
||||
<value>1048576</value>
|
||||
<description>This is the maximum size in bytes of a WRITE request
|
||||
supported by the NFS gateway. If you change this, make sure you
|
||||
also update the nfs mount's wsize(add wsize= # of bytes to the
|
||||
mount directive).
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>nfs.keytab.file</name>
|
||||
<value></value>
|
||||
<description>
|
||||
*Note*: Advanced property. Change with caution.
|
||||
@ -1307,7 +1355,7 @@
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.nfs.kerberos.principal</name>
|
||||
<name>nfs.kerberos.principal</name>
|
||||
<value></value>
|
||||
<description>
|
||||
*Note*: Advanced property. Change with caution.
|
||||
@ -1318,7 +1366,7 @@
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.nfs.allow.insecure.ports</name>
|
||||
<name>nfs.allow.insecure.ports</name>
|
||||
<value>true</value>
|
||||
<description>
|
||||
When set to false, client connections originating from unprivileged ports
|
||||
|
@ -76,14 +76,14 @@ HDFS NFS Gateway
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs.keytab.file</name>
|
||||
<name>nfs.keytab.file</name>
|
||||
<value>/etc/hadoop/conf/nfsserver.keytab</value> <!-- path to the nfs gateway keytab -->
|
||||
</property>
|
||||
----
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs.kerberos.principal</name>
|
||||
<name>nfs.kerberos.principal</name>
|
||||
<value>nfsserver/_HOST@YOUR-REALM.COM</value>
|
||||
</property>
|
||||
----
|
||||
@ -121,7 +121,7 @@ HDFS NFS Gateway
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs3.dump.dir</name>
|
||||
<name>nfs.dump.dir</name>
|
||||
<value>/tmp/.hdfs-nfs</value>
|
||||
</property>
|
||||
----
|
||||
@ -134,7 +134,7 @@ HDFS NFS Gateway
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs.rtmax</name>
|
||||
<name>nfs.rtmax</name>
|
||||
<value>1048576</value>
|
||||
<description>This is the maximum size in bytes of a READ request
|
||||
supported by the NFS gateway. If you change this, make sure you
|
||||
@ -146,7 +146,7 @@ HDFS NFS Gateway
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs.wtmax</name>
|
||||
<name>nfs.wtmax</name>
|
||||
<value>65536</value>
|
||||
<description>This is the maximum size in bytes of a WRITE request
|
||||
supported by the NFS gateway. If you change this, make sure you
|
||||
@ -167,7 +167,7 @@ HDFS NFS Gateway
|
||||
|
||||
----
|
||||
<property>
|
||||
<name>dfs.nfs.exports.allowed.hosts</name>
|
||||
<name>nfs.exports.allowed.hosts</name>
|
||||
<value>* rw</value>
|
||||
</property>
|
||||
----
|
||||
@ -345,7 +345,7 @@ HDFS NFS Gateway
|
||||
file in the event one wishes to access the HDFS NFS Gateway from a system with
|
||||
a completely disparate set of UIDs/GIDs. By default this file is located at
|
||||
"/etc/nfs.map", but a custom location can be configured by setting the
|
||||
"dfs.nfs.static.mapping.file" property to the path of the static mapping file.
|
||||
"nfs.static.mapping.file" property to the path of the static mapping file.
|
||||
The format of the static mapping file is similar to what is described in the
|
||||
exports(5) manual page, but roughly it is:
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user