merge from trunk r1598430
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/fs-encryption@1598435 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
333562b767
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop Auth, Java HTTP SPNEGO ${project.version} - Building It
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* Requirements
|
||||
|
||||
* Java 6+
|
||||
|
@ -70,6 +68,3 @@ $ mvn package -Pdocs
|
|||
|
||||
The generated documentation is available at
|
||||
<<<hadoop-auth/target/site/>>>.
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
|
|
|
@ -20,8 +20,6 @@
|
|||
Hadoop Auth, Java HTTP SPNEGO ${project.version} - Server Side
|
||||
Configuration
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* Server Side Configuration Setup
|
||||
|
||||
The AuthenticationFilter filter is Hadoop Auth's server side component.
|
||||
|
@ -241,5 +239,3 @@ Configuration
|
|||
...
|
||||
</web-app>
|
||||
+---+
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop Auth, Java HTTP SPNEGO ${project.version} - Examples
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* Accessing a Hadoop Auth protected URL Using a browser
|
||||
|
||||
<<IMPORTANT:>> The browser must support HTTP Kerberos SPNEGO. For example,
|
||||
|
@ -133,5 +131,3 @@ You are: user[tucu] principal[tucu@LOCALHOST]
|
|||
....
|
||||
|
||||
+---+
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
|
|
@ -334,6 +334,9 @@ Trunk (Unreleased)
|
|||
|
||||
HADOOP-10586. KeyShell doesn't allow setting Options via CLI. (clamb via tucu)
|
||||
|
||||
HADOOP-10625. Trim configuration names when putting/getting them
|
||||
to properties. (Wangda Tan via xgong)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HADOOP-7761. Improve the performance of raw comparisons. (todd)
|
||||
|
@ -413,6 +416,12 @@ Release 2.5.0 - UNRELEASED
|
|||
HADOOP-10614. CBZip2InputStream is not threadsafe (Xiangrui Meng via
|
||||
Sandy Ryza)
|
||||
|
||||
HADOOP-10618. Remove SingleNodeSetup.apt.vm. (Akira Ajisaka via
|
||||
Arpit Agarwal)
|
||||
|
||||
HADOOP-10448. Support pluggable mechanism to specify proxy user settings.
|
||||
(Benoy Antony via Arpit Agarwal)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -504,6 +513,12 @@ Release 2.5.0 - UNRELEASED
|
|||
HADOOP-10489. UserGroupInformation#getTokens and UserGroupInformation
|
||||
#addToken can lead to ConcurrentModificationException (Robert Kanter via atm)
|
||||
|
||||
HADOOP-10602. Documentation has broken "Go Back" hyperlinks.
|
||||
(Akira AJISAKA via cnauroth)
|
||||
|
||||
HADOOP-10639. FileBasedKeyStoresFactory initialization is not using default
|
||||
for SSL_REQUIRE_CLIENT_CERT_KEY. (tucu)
|
||||
|
||||
Release 2.4.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -566,6 +566,9 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
*/
|
||||
private String[] handleDeprecation(DeprecationContext deprecations,
|
||||
String name) {
|
||||
if (null != name) {
|
||||
name = name.trim();
|
||||
}
|
||||
ArrayList<String > names = new ArrayList<String>();
|
||||
if (isDeprecated(name)) {
|
||||
DeprecatedKeyInfo keyInfo = deprecations.getDeprecatedKeyMap().get(name);
|
||||
|
@ -843,12 +846,12 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
/**
|
||||
* Get the value of the <code>name</code> property, <code>null</code> if
|
||||
* no such property exists. If the key is deprecated, it returns the value of
|
||||
* the first key which replaces the deprecated key and is not null
|
||||
* the first key which replaces the deprecated key and is not null.
|
||||
*
|
||||
* Values are processed for <a href="#VariableExpansion">variable expansion</a>
|
||||
* before being returned.
|
||||
*
|
||||
* @param name the property name.
|
||||
* @param name the property name, will be trimmed before get value.
|
||||
* @return the value of the <code>name</code> or its replacing property,
|
||||
* or null if no such property exists.
|
||||
*/
|
||||
|
@ -952,7 +955,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
/**
|
||||
* Set the <code>value</code> of the <code>name</code> property. If
|
||||
* <code>name</code> is deprecated or there is a deprecated name associated to it,
|
||||
* it sets the value to both names.
|
||||
* it sets the value to both names. Name will be trimmed before put into
|
||||
* configuration.
|
||||
*
|
||||
* @param name property name.
|
||||
* @param value property value.
|
||||
|
@ -964,7 +968,8 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
/**
|
||||
* Set the <code>value</code> of the <code>name</code> property. If
|
||||
* <code>name</code> is deprecated, it also sets the <code>value</code> to
|
||||
* the keys that replace the deprecated key.
|
||||
* the keys that replace the deprecated key. Name will be trimmed before put
|
||||
* into configuration.
|
||||
*
|
||||
* @param name property name.
|
||||
* @param value property value.
|
||||
|
@ -979,6 +984,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
Preconditions.checkArgument(
|
||||
value != null,
|
||||
"The value of property " + name + " must not be null");
|
||||
name = name.trim();
|
||||
DeprecationContext deprecations = deprecationContext.get();
|
||||
if (deprecations.getDeprecatedKeyMap().isEmpty()) {
|
||||
getProps();
|
||||
|
@ -1064,7 +1070,7 @@ public class Configuration implements Iterable<Map.Entry<String,String>>,
|
|||
* If no such property exists,
|
||||
* then <code>defaultValue</code> is returned.
|
||||
*
|
||||
* @param name property name.
|
||||
* @param name property name, will be trimmed before get value.
|
||||
* @param defaultValue default value.
|
||||
* @return property value, or <code>defaultValue</code> if the property
|
||||
* doesn't exist.
|
||||
|
|
|
@ -293,5 +293,8 @@ public class CommonConfigurationKeysPublic {
|
|||
"hadoop.security.crypto.buffer.size";
|
||||
/** Defalt value for HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_KEY */
|
||||
public static final int HADOOP_SECURITY_CRYPTO_BUFFER_SIZE_DEFAULT = 8192;
|
||||
/** Class to override Impersonation provider */
|
||||
public static final String HADOOP_SECURITY_IMPERSONATION_PROVIDER_CLASS =
|
||||
"hadoop.security.impersonation.provider.class";
|
||||
}
|
||||
|
||||
|
|
|
@ -2140,9 +2140,21 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
* in the corresponding FileSystem.
|
||||
*/
|
||||
public FileChecksum getFileChecksum(Path f) throws IOException {
|
||||
return getFileChecksum(f, Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the checksum of a file, from the beginning of the file till the
|
||||
* specific length.
|
||||
* @param f The file path
|
||||
* @param length The length of the file range for checksum calculation
|
||||
* @return The file checksum.
|
||||
*/
|
||||
public FileChecksum getFileChecksum(Path f, final long length)
|
||||
throws IOException {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Set the verify checksum flag. This is only applicable if the
|
||||
* corresponding FileSystem supports checksum. By default doesn't do anything.
|
||||
|
|
|
@ -31,7 +31,6 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.fs.permission.AclStatus;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.fs.ContentSummary;
|
||||
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
@ -428,7 +427,12 @@ public class FilterFileSystem extends FileSystem {
|
|||
public FileChecksum getFileChecksum(Path f) throws IOException {
|
||||
return fs.getFileChecksum(f);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public FileChecksum getFileChecksum(Path f, long length) throws IOException {
|
||||
return fs.getFileChecksum(f, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setVerifyChecksum(boolean verifyChecksum) {
|
||||
fs.setVerifyChecksum(verifyChecksum);
|
||||
|
|
|
@ -687,7 +687,7 @@ public class HarFileSystem extends FileSystem {
|
|||
* @return null since no checksum algorithm is implemented.
|
||||
*/
|
||||
@Override
|
||||
public FileChecksum getFileChecksum(Path f) {
|
||||
public FileChecksum getFileChecksum(Path f, long length) {
|
||||
return null;
|
||||
}
|
||||
|
||||
|
|
|
@ -0,0 +1,210 @@
|
|||
/**
|
||||
* 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.security.authorize;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.security.Groups;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
public class DefaultImpersonationProvider implements ImpersonationProvider {
|
||||
private static final String CONF_HOSTS = ".hosts";
|
||||
private static final String CONF_USERS = ".users";
|
||||
private static final String CONF_GROUPS = ".groups";
|
||||
private static final String CONF_HADOOP_PROXYUSER = "hadoop.proxyuser.";
|
||||
private static final String CONF_HADOOP_PROXYUSER_RE = "hadoop\\.proxyuser\\.";
|
||||
// list of users, groups and hosts per proxyuser
|
||||
private Map<String, Collection<String>> proxyUsers =
|
||||
new HashMap<String, Collection<String>>();
|
||||
private Map<String, Collection<String>> proxyGroups =
|
||||
new HashMap<String, Collection<String>>();
|
||||
private Map<String, Collection<String>> proxyHosts =
|
||||
new HashMap<String, Collection<String>>();
|
||||
private Configuration conf;
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
this.conf = conf;
|
||||
|
||||
// get all the new keys for users
|
||||
String regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_USERS;
|
||||
Map<String,String> allMatchKeys = conf.getValByRegex(regex);
|
||||
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
|
||||
Collection<String> users = StringUtils.getTrimmedStringCollection(entry.getValue());
|
||||
proxyUsers.put(entry.getKey(), users);
|
||||
}
|
||||
|
||||
// get all the new keys for groups
|
||||
regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_GROUPS;
|
||||
allMatchKeys = conf.getValByRegex(regex);
|
||||
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
|
||||
Collection<String> groups = StringUtils.getTrimmedStringCollection(entry.getValue());
|
||||
proxyGroups.put(entry.getKey(), groups);
|
||||
//cache the groups. This is needed for NetGroups
|
||||
Groups.getUserToGroupsMappingService(conf).cacheGroupsAdd(
|
||||
new ArrayList<String>(groups));
|
||||
}
|
||||
|
||||
// now hosts
|
||||
regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_HOSTS;
|
||||
allMatchKeys = conf.getValByRegex(regex);
|
||||
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
|
||||
proxyHosts.put(entry.getKey(),
|
||||
StringUtils.getTrimmedStringCollection(entry.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return conf;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void authorize(UserGroupInformation user,
|
||||
String remoteAddress) throws AuthorizationException {
|
||||
|
||||
if (user.getRealUser() == null) {
|
||||
return;
|
||||
}
|
||||
boolean userAuthorized = false;
|
||||
boolean ipAuthorized = false;
|
||||
UserGroupInformation superUser = user.getRealUser();
|
||||
|
||||
Collection<String> allowedUsers = proxyUsers.get(
|
||||
getProxySuperuserUserConfKey(superUser.getShortUserName()));
|
||||
|
||||
if (isWildcardList(allowedUsers)) {
|
||||
userAuthorized = true;
|
||||
} else if (allowedUsers != null && !allowedUsers.isEmpty()) {
|
||||
if (allowedUsers.contains(user.getShortUserName())) {
|
||||
userAuthorized = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!userAuthorized){
|
||||
Collection<String> allowedUserGroups = proxyGroups.get(
|
||||
getProxySuperuserGroupConfKey(superUser.getShortUserName()));
|
||||
|
||||
if (isWildcardList(allowedUserGroups)) {
|
||||
userAuthorized = true;
|
||||
} else if (allowedUserGroups != null && !allowedUserGroups.isEmpty()) {
|
||||
for (String group : user.getGroupNames()) {
|
||||
if (allowedUserGroups.contains(group)) {
|
||||
userAuthorized = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!userAuthorized) {
|
||||
throw new AuthorizationException("User: " + superUser.getUserName()
|
||||
+ " is not allowed to impersonate " + user.getUserName());
|
||||
}
|
||||
}
|
||||
|
||||
Collection<String> ipList = proxyHosts.get(
|
||||
getProxySuperuserIpConfKey(superUser.getShortUserName()));
|
||||
|
||||
if (isWildcardList(ipList)) {
|
||||
ipAuthorized = true;
|
||||
} else if (ipList != null && !ipList.isEmpty()) {
|
||||
for (String allowedHost : ipList) {
|
||||
InetAddress hostAddr;
|
||||
try {
|
||||
hostAddr = InetAddress.getByName(allowedHost);
|
||||
} catch (UnknownHostException e) {
|
||||
continue;
|
||||
}
|
||||
if (hostAddr.getHostAddress().equals(remoteAddress)) {
|
||||
// Authorization is successful
|
||||
ipAuthorized = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
if(!ipAuthorized) {
|
||||
throw new AuthorizationException("Unauthorized connection for super-user: "
|
||||
+ superUser.getUserName() + " from IP " + remoteAddress);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if the configuration specifies the special configuration value
|
||||
* "*", indicating that any group or host list is allowed to use this configuration.
|
||||
*/
|
||||
private boolean isWildcardList(Collection<String> list) {
|
||||
return (list != null) &&
|
||||
(list.size() == 1) &&
|
||||
(list.contains("*"));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns configuration key for effective usergroups allowed for a superuser
|
||||
*
|
||||
* @param userName name of the superuser
|
||||
* @return configuration key for superuser usergroups
|
||||
*/
|
||||
public static String getProxySuperuserUserConfKey(String userName) {
|
||||
return CONF_HADOOP_PROXYUSER+userName+CONF_USERS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns configuration key for effective groups allowed for a superuser
|
||||
*
|
||||
* @param userName name of the superuser
|
||||
* @return configuration key for superuser groups
|
||||
*/
|
||||
public static String getProxySuperuserGroupConfKey(String userName) {
|
||||
return CONF_HADOOP_PROXYUSER+userName+CONF_GROUPS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return configuration key for superuser ip addresses
|
||||
*
|
||||
* @param userName name of the superuser
|
||||
* @return configuration key for superuser ip-addresses
|
||||
*/
|
||||
public static String getProxySuperuserIpConfKey(String userName) {
|
||||
return CONF_HADOOP_PROXYUSER+userName+CONF_HOSTS;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Map<String, Collection<String>> getProxyUsers() {
|
||||
return proxyUsers;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Map<String, Collection<String>> getProxyGroups() {
|
||||
return proxyGroups;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public Map<String, Collection<String>> getProxyHosts() {
|
||||
return proxyHosts;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,34 @@
|
|||
/**
|
||||
* 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.security.authorize;
|
||||
|
||||
import org.apache.hadoop.conf.Configurable;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
public interface ImpersonationProvider extends Configurable {
|
||||
/**
|
||||
* Authorize the superuser which is doing doAs
|
||||
*
|
||||
* @param user ugi of the effective or proxy user which contains a real user
|
||||
* @param remoteAddress the ip address of client
|
||||
* @throws AuthorizationException
|
||||
*/
|
||||
public void authorize(UserGroupInformation user, String remoteAddress)
|
||||
throws AuthorizationException;
|
||||
}
|
|
@ -18,42 +18,35 @@
|
|||
|
||||
package org.apache.hadoop.security.authorize;
|
||||
|
||||
import java.net.InetAddress;
|
||||
import java.net.UnknownHostException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.Map.Entry;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.security.Groups;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce", "HBase", "Hive"})
|
||||
public class ProxyUsers {
|
||||
|
||||
private static final String CONF_HOSTS = ".hosts";
|
||||
private static final String CONF_USERS = ".users";
|
||||
private static final String CONF_GROUPS = ".groups";
|
||||
private static final String CONF_HADOOP_PROXYUSER = "hadoop.proxyuser.";
|
||||
private static final String CONF_HADOOP_PROXYUSER_RE = "hadoop\\.proxyuser\\.";
|
||||
|
||||
private static boolean init = false;
|
||||
//list of users, groups and hosts per proxyuser
|
||||
private static Map<String, Collection<String>> proxyUsers =
|
||||
new HashMap<String, Collection<String>>();
|
||||
private static Map<String, Collection<String>> proxyGroups =
|
||||
new HashMap<String, Collection<String>>();
|
||||
private static Map<String, Collection<String>> proxyHosts =
|
||||
new HashMap<String, Collection<String>>();
|
||||
private static volatile ImpersonationProvider sip ;
|
||||
|
||||
/**
|
||||
* reread the conf and get new values for "hadoop.proxyuser.*.groups/users/hosts"
|
||||
* Returns an instance of ImpersonationProvider.
|
||||
* Looks up the configuration to see if there is custom class specified.
|
||||
* @param conf
|
||||
* @return ImpersonationProvider
|
||||
*/
|
||||
private static ImpersonationProvider getInstance(Configuration conf) {
|
||||
Class<? extends ImpersonationProvider> clazz =
|
||||
conf.getClass(
|
||||
CommonConfigurationKeysPublic.HADOOP_SECURITY_IMPERSONATION_PROVIDER_CLASS,
|
||||
DefaultImpersonationProvider.class, ImpersonationProvider.class);
|
||||
return ReflectionUtils.newInstance(clazz, conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* refresh Impersonation rules
|
||||
*/
|
||||
public static void refreshSuperUserGroupsConfiguration() {
|
||||
//load server side configuration;
|
||||
|
@ -64,73 +57,13 @@ public class ProxyUsers {
|
|||
* refresh configuration
|
||||
* @param conf
|
||||
*/
|
||||
public static synchronized void refreshSuperUserGroupsConfiguration(Configuration conf) {
|
||||
|
||||
// remove all existing stuff
|
||||
proxyGroups.clear();
|
||||
proxyHosts.clear();
|
||||
proxyUsers.clear();
|
||||
|
||||
// get all the new keys for users
|
||||
String regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_USERS;
|
||||
Map<String,String> allMatchKeys = conf.getValByRegex(regex);
|
||||
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
|
||||
Collection<String> users = StringUtils.getTrimmedStringCollection(entry.getValue());
|
||||
proxyUsers.put(entry.getKey(), users);
|
||||
}
|
||||
|
||||
// get all the new keys for groups
|
||||
regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_GROUPS;
|
||||
allMatchKeys = conf.getValByRegex(regex);
|
||||
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
|
||||
Collection<String> groups = StringUtils.getTrimmedStringCollection(entry.getValue());
|
||||
proxyGroups.put(entry.getKey(), groups );
|
||||
//cache the groups. This is needed for NetGroups
|
||||
Groups.getUserToGroupsMappingService(conf).cacheGroupsAdd(
|
||||
new ArrayList<String>(groups));
|
||||
}
|
||||
|
||||
// now hosts
|
||||
regex = CONF_HADOOP_PROXYUSER_RE+"[^.]*\\"+CONF_HOSTS;
|
||||
allMatchKeys = conf.getValByRegex(regex);
|
||||
for(Entry<String, String> entry : allMatchKeys.entrySet()) {
|
||||
proxyHosts.put(entry.getKey(),
|
||||
StringUtils.getTrimmedStringCollection(entry.getValue()));
|
||||
}
|
||||
init = true;
|
||||
public static void refreshSuperUserGroupsConfiguration(Configuration conf) {
|
||||
// sip is volatile. Any assignment to it as well as the object's state
|
||||
// will be visible to all the other threads.
|
||||
sip = getInstance(conf);
|
||||
ProxyServers.refresh(conf);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns configuration key for effective users allowed for a superuser
|
||||
*
|
||||
* @param userName name of the superuser
|
||||
* @return configuration key for superuser users
|
||||
*/
|
||||
public static String getProxySuperuserUserConfKey(String userName) {
|
||||
return ProxyUsers.CONF_HADOOP_PROXYUSER+userName+ProxyUsers.CONF_USERS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns configuration key for effective user groups allowed for a superuser
|
||||
*
|
||||
* @param userName name of the superuser
|
||||
* @return configuration key for superuser groups
|
||||
*/
|
||||
public static String getProxySuperuserGroupConfKey(String userName) {
|
||||
return ProxyUsers.CONF_HADOOP_PROXYUSER+userName+ProxyUsers.CONF_GROUPS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return configuration key for superuser ip addresses
|
||||
*
|
||||
* @param userName name of the superuser
|
||||
* @return configuration key for superuser ip-addresses
|
||||
*/
|
||||
public static String getProxySuperuserIpConfKey(String userName) {
|
||||
return ProxyUsers.CONF_HADOOP_PROXYUSER+userName+ProxyUsers.CONF_HOSTS;
|
||||
}
|
||||
|
||||
/**
|
||||
* Authorize the superuser which is doing doAs
|
||||
*
|
||||
|
@ -138,75 +71,14 @@ public class ProxyUsers {
|
|||
* @param remoteAddress the ip address of client
|
||||
* @throws AuthorizationException
|
||||
*/
|
||||
public static synchronized void authorize(UserGroupInformation user,
|
||||
public static void authorize(UserGroupInformation user,
|
||||
String remoteAddress) throws AuthorizationException {
|
||||
|
||||
if(!init) {
|
||||
if (sip==null) {
|
||||
// In a race situation, It is possible for multiple threads to satisfy this condition.
|
||||
// The last assignment will prevail.
|
||||
refreshSuperUserGroupsConfiguration();
|
||||
}
|
||||
|
||||
if (user.getRealUser() == null) {
|
||||
return;
|
||||
}
|
||||
boolean userAuthorized = false;
|
||||
boolean ipAuthorized = false;
|
||||
UserGroupInformation superUser = user.getRealUser();
|
||||
|
||||
Collection<String> allowedUsers = proxyUsers.get(
|
||||
getProxySuperuserUserConfKey(superUser.getShortUserName()));
|
||||
|
||||
if (isWildcardList(allowedUsers)) {
|
||||
userAuthorized = true;
|
||||
} else if (allowedUsers != null && !allowedUsers.isEmpty()) {
|
||||
if (allowedUsers.contains(user.getShortUserName())) {
|
||||
userAuthorized = true;
|
||||
}
|
||||
}
|
||||
|
||||
if (!userAuthorized) {
|
||||
Collection<String> allowedUserGroups = proxyGroups.get(
|
||||
getProxySuperuserGroupConfKey(superUser.getShortUserName()));
|
||||
|
||||
if (isWildcardList(allowedUserGroups)) {
|
||||
userAuthorized = true;
|
||||
} else if (allowedUserGroups != null && !allowedUserGroups.isEmpty()) {
|
||||
for (String group : user.getGroupNames()) {
|
||||
if (allowedUserGroups.contains(group)) {
|
||||
userAuthorized = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (!userAuthorized) {
|
||||
throw new AuthorizationException("User: " + superUser.getUserName()
|
||||
+ " is not allowed to impersonate " + user.getUserName());
|
||||
}
|
||||
}
|
||||
|
||||
Collection<String> ipList = proxyHosts.get(
|
||||
getProxySuperuserIpConfKey(superUser.getShortUserName()));
|
||||
|
||||
if (isWildcardList(ipList)) {
|
||||
ipAuthorized = true;
|
||||
} else if (ipList != null && !ipList.isEmpty()) {
|
||||
for (String allowedHost : ipList) {
|
||||
InetAddress hostAddr;
|
||||
try {
|
||||
hostAddr = InetAddress.getByName(allowedHost);
|
||||
} catch (UnknownHostException e) {
|
||||
continue;
|
||||
}
|
||||
if (hostAddr.getHostAddress().equals(remoteAddress)) {
|
||||
// Authorization is successful
|
||||
ipAuthorized = true;
|
||||
}
|
||||
}
|
||||
}
|
||||
if (!ipAuthorized) {
|
||||
throw new AuthorizationException("Unauthorized connection for super-user: "
|
||||
+ superUser.getUserName() + " from IP " + remoteAddress);
|
||||
}
|
||||
sip.authorize(user, remoteAddress);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -218,33 +90,14 @@ public class ProxyUsers {
|
|||
* @deprecated use {@link #authorize(UserGroupInformation, String) instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public static synchronized void authorize(UserGroupInformation user,
|
||||
public static void authorize(UserGroupInformation user,
|
||||
String remoteAddress, Configuration conf) throws AuthorizationException {
|
||||
authorize(user,remoteAddress);
|
||||
}
|
||||
|
||||
/**
|
||||
* Return true if the configuration specifies the special configuration value
|
||||
* "*", indicating that any group or host list is allowed to use this configuration.
|
||||
*/
|
||||
private static boolean isWildcardList(Collection<String> list) {
|
||||
return (list != null) &&
|
||||
(list.size() == 1) &&
|
||||
(list.contains("*"));
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static Map<String, Collection<String>> getProxyUsers() {
|
||||
return proxyUsers;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static Map<String, Collection<String>> getProxyGroups() {
|
||||
return proxyGroups;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static Map<String, Collection<String>> getProxyHosts() {
|
||||
return proxyHosts;
|
||||
|
||||
@VisibleForTesting
|
||||
public static DefaultImpersonationProvider getDefaultImpersonationProvider() {
|
||||
return ((DefaultImpersonationProvider)sip);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -131,7 +131,8 @@ public class FileBasedKeyStoresFactory implements KeyStoresFactory {
|
|||
throws IOException, GeneralSecurityException {
|
||||
|
||||
boolean requireClientCert =
|
||||
conf.getBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY, true);
|
||||
conf.getBoolean(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY,
|
||||
SSLFactory.DEFAULT_SSL_REQUIRE_CLIENT_CERT);
|
||||
|
||||
// certificate store
|
||||
String keystoreType =
|
||||
|
|
|
@ -722,6 +722,17 @@
|
|||
|
||||
<!-- Proxy Configuration -->
|
||||
|
||||
<property>
|
||||
<name>hadoop.security.impersonation.provider.class</name>
|
||||
<value></value>
|
||||
<description>A class which implements ImpersonationProvider interface, used to
|
||||
authorize whether one user can impersonate a specific user.
|
||||
If not specified, the DefaultImpersonationProvider will be used.
|
||||
If a class is specified, then that class will be used to determine
|
||||
the impersonation capability.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>hadoop.rpc.socket.factory.class.default</name>
|
||||
<value>org.apache.hadoop.net.StandardSocketFactory</value>
|
||||
|
|
|
@ -18,210 +18,7 @@
|
|||
|
||||
Single Node Setup
|
||||
|
||||
%{toc|section=1|fromDepth=0}
|
||||
This page will be removed in the next major release.
|
||||
|
||||
* Purpose
|
||||
|
||||
This document describes how to set up and configure a single-node
|
||||
Hadoop installation so that you can quickly perform simple operations
|
||||
using Hadoop MapReduce and the Hadoop Distributed File System (HDFS).
|
||||
|
||||
* Prerequisites
|
||||
|
||||
** Supported Platforms
|
||||
|
||||
* GNU/Linux is supported as a development and production platform.
|
||||
Hadoop has been demonstrated on GNU/Linux clusters with 2000 nodes.
|
||||
|
||||
* Windows is also a supported platform.
|
||||
|
||||
** Required Software
|
||||
|
||||
Required software for Linux and Windows include:
|
||||
|
||||
[[1]] Java^TM 1.6.x, preferably from Sun, must be installed.
|
||||
|
||||
[[2]] ssh must be installed and sshd must be running to use the Hadoop
|
||||
scripts that manage remote Hadoop daemons.
|
||||
|
||||
** Installing Software
|
||||
|
||||
If your cluster doesn't have the requisite software you will need to
|
||||
install it.
|
||||
|
||||
For example on Ubuntu Linux:
|
||||
|
||||
----
|
||||
$ sudo apt-get install ssh
|
||||
$ sudo apt-get install rsync
|
||||
----
|
||||
|
||||
* Download
|
||||
|
||||
To get a Hadoop distribution, download a recent stable release from one
|
||||
of the Apache Download Mirrors.
|
||||
|
||||
* Prepare to Start the Hadoop Cluster
|
||||
|
||||
Unpack the downloaded Hadoop distribution. In the distribution, edit
|
||||
the file <<<conf/hadoop-env.sh>>> to define at least <<<JAVA_HOME>>> to be the root
|
||||
of your Java installation.
|
||||
|
||||
Try the following command:
|
||||
|
||||
----
|
||||
$ bin/hadoop
|
||||
----
|
||||
|
||||
This will display the usage documentation for the hadoop script.
|
||||
|
||||
Now you are ready to start your Hadoop cluster in one of the three
|
||||
supported modes:
|
||||
|
||||
* Local (Standalone) Mode
|
||||
|
||||
* Pseudo-Distributed Mode
|
||||
|
||||
* Fully-Distributed Mode
|
||||
|
||||
* Standalone Operation
|
||||
|
||||
By default, Hadoop is configured to run in a non-distributed mode, as a
|
||||
single Java process. This is useful for debugging.
|
||||
|
||||
The following example copies the unpacked conf directory to use as
|
||||
input and then finds and displays every match of the given regular
|
||||
expression. Output is written to the given output directory.
|
||||
|
||||
----
|
||||
$ mkdir input
|
||||
$ cp conf/*.xml input
|
||||
$ bin/hadoop jar hadoop-*-examples.jar grep input output 'dfs[a-z.]+'
|
||||
$ cat output/*
|
||||
---
|
||||
|
||||
* Pseudo-Distributed Operation
|
||||
|
||||
Hadoop can also be run on a single-node in a pseudo-distributed mode
|
||||
where each Hadoop daemon runs in a separate Java process.
|
||||
|
||||
** Configuration
|
||||
|
||||
Use the following:
|
||||
|
||||
conf/core-site.xml:
|
||||
|
||||
----
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.defaultFS</name>
|
||||
<value>hdfs://localhost:9000</value>
|
||||
</property>
|
||||
</configuration>
|
||||
----
|
||||
|
||||
conf/hdfs-site.xml:
|
||||
|
||||
----
|
||||
<configuration>
|
||||
<property>
|
||||
<name>dfs.replication</name>
|
||||
<value>1</value>
|
||||
</property>
|
||||
</configuration>
|
||||
----
|
||||
|
||||
conf/mapred-site.xml:
|
||||
|
||||
----
|
||||
<configuration>
|
||||
<property>
|
||||
<name>mapred.job.tracker</name>
|
||||
<value>localhost:9001</value>
|
||||
</property>
|
||||
</configuration>
|
||||
----
|
||||
|
||||
** Setup passphraseless ssh
|
||||
|
||||
Now check that you can ssh to the localhost without a passphrase:
|
||||
|
||||
----
|
||||
$ ssh localhost
|
||||
----
|
||||
|
||||
If you cannot ssh to localhost without a passphrase, execute the
|
||||
following commands:
|
||||
|
||||
----
|
||||
$ ssh-keygen -t dsa -P '' -f ~/.ssh/id_dsa
|
||||
$ cat ~/.ssh/id_dsa.pub >> ~/.ssh/authorized_keys
|
||||
----
|
||||
|
||||
** Execution
|
||||
|
||||
Format a new distributed-filesystem:
|
||||
|
||||
----
|
||||
$ bin/hadoop namenode -format
|
||||
----
|
||||
|
||||
Start the hadoop daemons:
|
||||
|
||||
----
|
||||
$ bin/start-all.sh
|
||||
----
|
||||
|
||||
The hadoop daemon log output is written to the <<<${HADOOP_LOG_DIR}>>>
|
||||
directory (defaults to <<<${HADOOP_PREFIX}/logs>>>).
|
||||
|
||||
Browse the web interface for the NameNode and the JobTracker; by
|
||||
default they are available at:
|
||||
|
||||
* NameNode - <<<http://localhost:50070/>>>
|
||||
|
||||
* JobTracker - <<<http://localhost:50030/>>>
|
||||
|
||||
Copy the input files into the distributed filesystem:
|
||||
|
||||
----
|
||||
$ bin/hadoop fs -put conf input
|
||||
----
|
||||
|
||||
Run some of the examples provided:
|
||||
|
||||
----
|
||||
$ bin/hadoop jar hadoop-*-examples.jar grep input output 'dfs[a-z.]+'
|
||||
----
|
||||
|
||||
Examine the output files:
|
||||
|
||||
Copy the output files from the distributed filesystem to the local
|
||||
filesytem and examine them:
|
||||
|
||||
----
|
||||
$ bin/hadoop fs -get output output
|
||||
$ cat output/*
|
||||
----
|
||||
|
||||
or
|
||||
|
||||
View the output files on the distributed filesystem:
|
||||
|
||||
----
|
||||
$ bin/hadoop fs -cat output/*
|
||||
----
|
||||
|
||||
When you're done, stop the daemons with:
|
||||
|
||||
----
|
||||
$ bin/stop-all.sh
|
||||
----
|
||||
|
||||
* Fully-Distributed Operation
|
||||
|
||||
For information on setting up fully-distributed, non-trivial clusters
|
||||
see {{{./ClusterSetup.html}Cluster Setup}}.
|
||||
|
||||
Java and JNI are trademarks or registered trademarks of Sun
|
||||
Microsystems, Inc. in the United States and other countries.
|
||||
See {{{./SingleCluster.html}Single Cluster Setup}} to set up and configure a
|
||||
single-node Hadoop installation.
|
||||
|
|
|
@ -49,7 +49,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import static org.apache.hadoop.util.PlatformName.IBM_JAVA;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
public class TestConfiguration extends TestCase {
|
||||
|
||||
|
@ -1003,6 +1003,14 @@ public class TestConfiguration extends TestCase {
|
|||
String resource;
|
||||
}
|
||||
|
||||
public void testGetSetTrimmedNames() throws IOException {
|
||||
Configuration conf = new Configuration(false);
|
||||
conf.set(" name", "value");
|
||||
assertEquals("value", conf.get("name"));
|
||||
assertEquals("value", conf.get(" name"));
|
||||
assertEquals("value", conf.getRaw(" name "));
|
||||
}
|
||||
|
||||
public void testDumpConfiguration () throws IOException {
|
||||
StringWriter outWriter = new StringWriter();
|
||||
Configuration.dumpConfiguration(conf, outWriter);
|
||||
|
|
|
@ -139,6 +139,7 @@ public class TestHarFileSystem {
|
|||
public int getDefaultPort();
|
||||
public String getCanonicalServiceName();
|
||||
public Token<?> getDelegationToken(String renewer) throws IOException;
|
||||
public FileChecksum getFileChecksum(Path f) throws IOException;
|
||||
public boolean deleteOnExit(Path f) throws IOException;
|
||||
public boolean cancelDeleteOnExit(Path f) throws IOException;
|
||||
public Token<?>[] addDelegationTokens(String renewer, Credentials creds)
|
||||
|
@ -223,10 +224,16 @@ public class TestHarFileSystem {
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testFileChecksum() {
|
||||
public void testFileChecksum() throws Exception {
|
||||
final Path p = new Path("har://file-localhost/foo.har/file1");
|
||||
final HarFileSystem harfs = new HarFileSystem();
|
||||
Assert.assertEquals(null, harfs.getFileChecksum(p));
|
||||
try {
|
||||
Assert.assertEquals(null, harfs.getFileChecksum(p));
|
||||
} finally {
|
||||
if (harfs != null) {
|
||||
harfs.close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.net.NetUtils;
|
|||
import org.apache.hadoop.security.KerberosInfo;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
|
@ -326,7 +327,7 @@ public class MiniRPCBenchmark {
|
|||
String shortUserName =
|
||||
UserGroupInformation.createRemoteUser(user).getShortUserName();
|
||||
try {
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(shortUserName),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserGroupConfKey(shortUserName),
|
||||
GROUP_NAME_1);
|
||||
configureSuperUserIPAddresses(conf, shortUserName);
|
||||
// start the server
|
||||
|
@ -410,7 +411,7 @@ public class MiniRPCBenchmark {
|
|||
}
|
||||
builder.append("127.0.1.1,");
|
||||
builder.append(InetAddress.getLocalHost().getCanonicalHostName());
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserIpConfKey(superUserShortName),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserIpConfKey(superUserShortName),
|
||||
builder.toString());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,6 +35,7 @@ import org.apache.hadoop.ipc.Server;
|
|||
import org.apache.hadoop.ipc.VersionedProtocol;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenInfo;
|
||||
|
@ -100,7 +101,7 @@ public class TestDoAsEffectiveUser {
|
|||
builder.append("127.0.1.1,");
|
||||
builder.append(InetAddress.getLocalHost().getCanonicalHostName());
|
||||
LOG.info("Local Ip addresses: "+builder.toString());
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserIpConfKey(superUserShortName),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserIpConfKey(superUserShortName),
|
||||
builder.toString());
|
||||
}
|
||||
|
||||
|
@ -180,7 +181,7 @@ public class TestDoAsEffectiveUser {
|
|||
@Test(timeout=4000)
|
||||
public void testRealUserSetup() throws IOException {
|
||||
final Configuration conf = new Configuration();
|
||||
conf.setStrings(ProxyUsers
|
||||
conf.setStrings(DefaultImpersonationProvider
|
||||
.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME), "group1");
|
||||
configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
|
||||
Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
|
||||
|
@ -213,7 +214,7 @@ public class TestDoAsEffectiveUser {
|
|||
public void testRealUserAuthorizationSuccess() throws IOException {
|
||||
final Configuration conf = new Configuration();
|
||||
configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
|
||||
"group1");
|
||||
Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
|
||||
.setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
|
||||
|
@ -247,9 +248,9 @@ public class TestDoAsEffectiveUser {
|
|||
@Test
|
||||
public void testRealUserIPAuthorizationFailure() throws IOException {
|
||||
final Configuration conf = new Configuration();
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_SHORT_NAME),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_SHORT_NAME),
|
||||
"20.20.20.20"); //Authorized IP address
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
|
||||
"group1");
|
||||
Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
|
||||
.setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
|
||||
|
@ -292,7 +293,7 @@ public class TestDoAsEffectiveUser {
|
|||
@Test
|
||||
public void testRealUserIPNotSpecified() throws IOException {
|
||||
final Configuration conf = new Configuration();
|
||||
conf.setStrings(ProxyUsers
|
||||
conf.setStrings(DefaultImpersonationProvider
|
||||
.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME), "group1");
|
||||
Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
|
||||
.setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
|
||||
|
@ -376,7 +377,7 @@ public class TestDoAsEffectiveUser {
|
|||
public void testRealUserGroupAuthorizationFailure() throws IOException {
|
||||
final Configuration conf = new Configuration();
|
||||
configureSuperUserIPAddresses(conf, REAL_USER_SHORT_NAME);
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_SHORT_NAME),
|
||||
"group3");
|
||||
Server server = new RPC.Builder(conf).setProtocol(TestProtocol.class)
|
||||
.setInstance(new TestImpl()).setBindAddress(ADDRESS).setPort(0)
|
||||
|
|
|
@ -17,6 +17,9 @@
|
|||
*/
|
||||
package org.apache.hadoop.security.authorize;
|
||||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Collection;
|
||||
|
@ -25,13 +28,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.util.NativeCodeLoader;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.security.Groups;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
|
||||
import org.apache.hadoop.util.NativeCodeLoader;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
|
||||
|
||||
public class TestProxyUsers {
|
||||
|
@ -46,6 +47,8 @@ public class TestProxyUsers {
|
|||
new String[] { "@foo_group" };
|
||||
private static final String[] OTHER_GROUP_NAMES =
|
||||
new String[] { "bar_group" };
|
||||
private static final String[] SUDO_GROUP_NAMES =
|
||||
new String[] { "sudo_proxied_user" };
|
||||
private static final String PROXY_IP = "1.2.3.4";
|
||||
|
||||
/**
|
||||
|
@ -106,10 +109,10 @@ public class TestProxyUsers {
|
|||
groupMappingClassName);
|
||||
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(NETGROUP_NAMES)));
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
@ -130,10 +133,10 @@ public class TestProxyUsers {
|
|||
public void testProxyUsers() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(GROUP_NAMES)));
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
|
@ -164,11 +167,11 @@ public class TestProxyUsers {
|
|||
public void testProxyUsersWithUserConf() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserUserConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(AUTHORIZED_PROXY_USER_NAME)));
|
||||
DefaultImpersonationProvider.getProxySuperuserUserConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(AUTHORIZED_PROXY_USER_NAME)));
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
|
||||
|
@ -198,10 +201,10 @@ public class TestProxyUsers {
|
|||
public void testWildcardGroup() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
"*");
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
|
@ -232,10 +235,10 @@ public class TestProxyUsers {
|
|||
public void testWildcardUser() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserUserConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserUserConfKey(REAL_USER_NAME),
|
||||
"*");
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
|
@ -266,10 +269,10 @@ public class TestProxyUsers {
|
|||
public void testWildcardIP() {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(GROUP_NAMES)));
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
"*");
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
|
@ -297,15 +300,16 @@ public class TestProxyUsers {
|
|||
public void testWithDuplicateProxyGroups() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(GROUP_NAMES,GROUP_NAMES)));
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
PROXY_IP);
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
Collection<String> groupsToBeProxied = ProxyUsers.getProxyGroups().get(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME));
|
||||
Collection<String> groupsToBeProxied =
|
||||
ProxyUsers.getDefaultImpersonationProvider().getProxyGroups().get(
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME));
|
||||
|
||||
assertEquals (1,groupsToBeProxied.size());
|
||||
}
|
||||
|
@ -314,18 +318,51 @@ public class TestProxyUsers {
|
|||
public void testWithDuplicateProxyHosts() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(GROUP_NAMES)));
|
||||
conf.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME),
|
||||
StringUtils.join(",", Arrays.asList(PROXY_IP,PROXY_IP)));
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
Collection<String> hosts = ProxyUsers.getProxyHosts().get(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(REAL_USER_NAME));
|
||||
Collection<String> hosts =
|
||||
ProxyUsers.getDefaultImpersonationProvider().getProxyHosts().get(
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(REAL_USER_NAME));
|
||||
|
||||
assertEquals (1,hosts.size());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testProxyUsersWithProviderOverride() throws Exception {
|
||||
Configuration conf = new Configuration();
|
||||
conf.set(
|
||||
CommonConfigurationKeysPublic.HADOOP_SECURITY_IMPERSONATION_PROVIDER_CLASS,
|
||||
"org.apache.hadoop.security.authorize.TestProxyUsers$TestDummyImpersonationProvider");
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
|
||||
// First try proxying a group that's allowed
|
||||
UserGroupInformation realUserUgi = UserGroupInformation
|
||||
.createUserForTesting(REAL_USER_NAME, SUDO_GROUP_NAMES);
|
||||
UserGroupInformation proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
|
||||
PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
|
||||
|
||||
// From good IP
|
||||
assertAuthorized(proxyUserUgi, "1.2.3.4");
|
||||
// From bad IP
|
||||
assertAuthorized(proxyUserUgi, "1.2.3.5");
|
||||
|
||||
// Now try proxying a group that's not allowed
|
||||
realUserUgi = UserGroupInformation
|
||||
.createUserForTesting(REAL_USER_NAME, GROUP_NAMES);
|
||||
proxyUserUgi = UserGroupInformation.createProxyUserForTesting(
|
||||
PROXY_USER_NAME, realUserUgi, GROUP_NAMES);
|
||||
|
||||
// From good IP
|
||||
assertNotAuthorized(proxyUserUgi, "1.2.3.4");
|
||||
// From bad IP
|
||||
assertNotAuthorized(proxyUserUgi, "1.2.3.5");
|
||||
}
|
||||
|
||||
|
||||
private void assertNotAuthorized(UserGroupInformation proxyUgi, String host) {
|
||||
try {
|
||||
|
@ -343,4 +380,32 @@ public class TestProxyUsers {
|
|||
fail("Did not allow authorization of " + proxyUgi + " from " + host);
|
||||
}
|
||||
}
|
||||
|
||||
static class TestDummyImpersonationProvider implements ImpersonationProvider {
|
||||
/**
|
||||
* Authorize a user (superuser) to impersonate another user (user1) if the
|
||||
* superuser belongs to the group "sudo_user1" .
|
||||
*/
|
||||
|
||||
public void authorize(UserGroupInformation user,
|
||||
String remoteAddress) throws AuthorizationException{
|
||||
UserGroupInformation superUser = user.getRealUser();
|
||||
|
||||
String sudoGroupName = "sudo_" + user.getShortUserName();
|
||||
if (!Arrays.asList(superUser.getGroupNames()).contains(sudoGroupName)){
|
||||
throw new AuthorizationException("User: " + superUser.getUserName()
|
||||
+ " is not allowed to impersonate " + user.getUserName());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setConf(Configuration conf) {
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public Configuration getConf() {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -272,4 +272,17 @@ public class TestSSLFactory {
|
|||
sslFactory.destroy();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNoClientCertsInitialization() throws Exception {
|
||||
Configuration conf = createConfiguration(false);
|
||||
conf.unset(SSLFactory.SSL_REQUIRE_CLIENT_CERT_KEY);
|
||||
SSLFactory sslFactory = new SSLFactory(SSLFactory.Mode.CLIENT, conf);
|
||||
try {
|
||||
sslFactory.init();
|
||||
} finally {
|
||||
sslFactory.destroy();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -483,5 +483,3 @@ Content-Type: application/json
|
|||
...
|
||||
]
|
||||
+---+
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
|
|
@ -18,8 +18,14 @@
|
|||
package org.apache.hadoop.nfs.nfs3;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -44,6 +50,17 @@ 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.
|
||||
private static final Pattern EMPTY_LINE = Pattern.compile("^\\s*$");
|
||||
private static final Pattern COMMENT_LINE = Pattern.compile("^\\s*#.*$");
|
||||
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
|
||||
|
@ -58,6 +75,7 @@ public class IdUserGroup {
|
|||
|
||||
public IdUserGroup() throws IOException {
|
||||
timeout = TIMEOUT_DEFAULT;
|
||||
staticMappingFile = new File(NFS_STATIC_MAPPING_FILE_DEFAULT);
|
||||
updateMaps();
|
||||
}
|
||||
|
||||
|
@ -71,6 +89,11 @@ public class IdUserGroup {
|
|||
} else {
|
||||
timeout = updateTime;
|
||||
}
|
||||
|
||||
String staticFilePath = conf.get(NFS_STATIC_MAPPING_FILE_KEY,
|
||||
NFS_STATIC_MAPPING_FILE_DEFAULT);
|
||||
staticMappingFile = new File(staticFilePath);
|
||||
|
||||
updateMaps();
|
||||
}
|
||||
|
||||
|
@ -137,7 +160,8 @@ public class IdUserGroup {
|
|||
*/
|
||||
@VisibleForTesting
|
||||
public static void updateMapInternal(BiMap<Integer, String> map, String mapName,
|
||||
String command, String regex) throws IOException {
|
||||
String command, String regex, Map<Integer, Integer> staticMapping)
|
||||
throws IOException {
|
||||
BufferedReader br = null;
|
||||
try {
|
||||
Process process = Runtime.getRuntime().exec(
|
||||
|
@ -151,7 +175,7 @@ public class IdUserGroup {
|
|||
}
|
||||
LOG.debug("add to " + mapName + "map:" + nameId[0] + " id:" + nameId[1]);
|
||||
// HDFS can't differentiate duplicate names with simple authentication
|
||||
final Integer key = parseId(nameId[1]);
|
||||
final Integer key = staticMapping.get(parseId(nameId[1]));
|
||||
final String value = nameId[0];
|
||||
if (map.containsKey(key)) {
|
||||
final String prevValue = map.get(key);
|
||||
|
@ -173,7 +197,7 @@ public class IdUserGroup {
|
|||
}
|
||||
map.put(key, value);
|
||||
}
|
||||
LOG.info("Updated " + mapName + " map size:" + map.size());
|
||||
LOG.info("Updated " + mapName + " map size: " + map.size());
|
||||
|
||||
} catch (IOException e) {
|
||||
LOG.error("Can't update " + mapName + " map");
|
||||
|
@ -199,20 +223,115 @@ public class IdUserGroup {
|
|||
+ " 'nobody' will be used for any user and group.");
|
||||
return;
|
||||
}
|
||||
|
||||
StaticMapping staticMapping = new StaticMapping(
|
||||
new HashMap<Integer, Integer>(), new HashMap<Integer, Integer>());
|
||||
if (staticMappingFile.exists()) {
|
||||
LOG.info("Using '" + staticMappingFile + "' for static UID/GID mapping...");
|
||||
staticMapping = parseStaticMap(staticMappingFile);
|
||||
} else {
|
||||
LOG.info("Not doing static UID/GID mapping because '" + staticMappingFile
|
||||
+ "' does not exist.");
|
||||
}
|
||||
|
||||
if (OS.startsWith("Linux")) {
|
||||
updateMapInternal(uMap, "user", LINUX_GET_ALL_USERS_CMD, ":");
|
||||
updateMapInternal(gMap, "group", LINUX_GET_ALL_GROUPS_CMD, ":");
|
||||
updateMapInternal(uMap, "user", LINUX_GET_ALL_USERS_CMD, ":",
|
||||
staticMapping.uidMapping);
|
||||
updateMapInternal(gMap, "group", LINUX_GET_ALL_GROUPS_CMD, ":",
|
||||
staticMapping.gidMapping);
|
||||
} else {
|
||||
// Mac
|
||||
updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+");
|
||||
updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+");
|
||||
updateMapInternal(uMap, "user", MAC_GET_ALL_USERS_CMD, "\\s+",
|
||||
staticMapping.uidMapping);
|
||||
updateMapInternal(gMap, "group", MAC_GET_ALL_GROUPS_CMD, "\\s+",
|
||||
staticMapping.gidMapping);
|
||||
}
|
||||
|
||||
uidNameMap = uMap;
|
||||
gidNameMap = gMap;
|
||||
lastUpdateTime = Time.monotonicNow();
|
||||
}
|
||||
|
||||
@SuppressWarnings("serial")
|
||||
static final class PassThroughMap<K> extends HashMap<K, K> {
|
||||
|
||||
public PassThroughMap() {
|
||||
this(new HashMap<K, K>());
|
||||
}
|
||||
|
||||
public PassThroughMap(Map<K, K> mapping) {
|
||||
super();
|
||||
for (Map.Entry<K, K> entry : mapping.entrySet()) {
|
||||
super.put(entry.getKey(), entry.getValue());
|
||||
}
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
@Override
|
||||
public K get(Object key) {
|
||||
if (super.containsKey(key)) {
|
||||
return super.get(key);
|
||||
} else {
|
||||
return (K) key;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
static final class StaticMapping {
|
||||
final Map<Integer, Integer> uidMapping;
|
||||
final Map<Integer, Integer> gidMapping;
|
||||
|
||||
public StaticMapping(Map<Integer, Integer> uidMapping,
|
||||
Map<Integer, Integer> gidMapping) {
|
||||
this.uidMapping = new PassThroughMap<Integer>(uidMapping);
|
||||
this.gidMapping = new PassThroughMap<Integer>(gidMapping);
|
||||
}
|
||||
}
|
||||
|
||||
static StaticMapping parseStaticMap(File staticMapFile)
|
||||
throws IOException {
|
||||
|
||||
Map<Integer, Integer> uidMapping = new HashMap<Integer, Integer>();
|
||||
Map<Integer, Integer> gidMapping = new HashMap<Integer, Integer>();
|
||||
|
||||
BufferedReader in = new BufferedReader(new InputStreamReader(
|
||||
new FileInputStream(staticMapFile)));
|
||||
|
||||
try {
|
||||
String line = null;
|
||||
while ((line = in.readLine()) != null) {
|
||||
// Skip entirely empty and comment lines.
|
||||
if (EMPTY_LINE.matcher(line).matches() ||
|
||||
COMMENT_LINE.matcher(line).matches()) {
|
||||
continue;
|
||||
}
|
||||
|
||||
Matcher lineMatcher = MAPPING_LINE.matcher(line);
|
||||
if (!lineMatcher.matches()) {
|
||||
LOG.warn("Could not parse line '" + line + "'. Lines should be of " +
|
||||
"the form '[uid|gid] [remote id] [local id]'. Blank lines and " +
|
||||
"everything following a '#' on a line will be ignored.");
|
||||
continue;
|
||||
}
|
||||
|
||||
// We know the line is fine to parse without error checking like this
|
||||
// since it matched the regex above.
|
||||
String firstComponent = lineMatcher.group(1);
|
||||
int remoteId = Integer.parseInt(lineMatcher.group(2));
|
||||
int localId = Integer.parseInt(lineMatcher.group(3));
|
||||
if (firstComponent.equals("uid")) {
|
||||
uidMapping.put(localId, remoteId);
|
||||
} else {
|
||||
gidMapping.put(localId, remoteId);
|
||||
}
|
||||
}
|
||||
} finally {
|
||||
in.close();
|
||||
}
|
||||
|
||||
return new StaticMapping(uidMapping, gidMapping);
|
||||
}
|
||||
|
||||
synchronized public int getUid(String user) throws IOException {
|
||||
checkAndUpdateMaps();
|
||||
|
|
|
@ -46,10 +46,12 @@ public class ACCESS3Response extends NFS3Response {
|
|||
@Override
|
||||
public XDR writeHeaderAndResponse(XDR out, int xid, Verifier verifier) {
|
||||
super.writeHeaderAndResponse(out, xid, verifier);
|
||||
out.writeBoolean(true);
|
||||
postOpAttr.serialize(out);
|
||||
if (this.getStatus() == Nfs3Status.NFS3_OK) {
|
||||
out.writeBoolean(true);
|
||||
postOpAttr.serialize(out);
|
||||
out.writeInt(access);
|
||||
} else {
|
||||
out.writeBoolean(false);
|
||||
}
|
||||
return out;
|
||||
}
|
||||
|
|
|
@ -19,15 +19,97 @@ package org.apache.hadoop.nfs.nfs3;
|
|||
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.nfs.nfs3.IdUserGroup.PassThroughMap;
|
||||
import org.apache.hadoop.nfs.nfs3.IdUserGroup.StaticMapping;
|
||||
import org.junit.Test;
|
||||
|
||||
import com.google.common.collect.BiMap;
|
||||
import com.google.common.collect.HashBiMap;
|
||||
|
||||
public class TestIdUserGroup {
|
||||
|
||||
private static final Map<Integer, Integer> EMPTY_PASS_THROUGH_MAP =
|
||||
new PassThroughMap<Integer>();
|
||||
|
||||
@Test
|
||||
public void testStaticMapParsing() throws IOException {
|
||||
File tempStaticMapFile = File.createTempFile("nfs-", ".map");
|
||||
final String staticMapFileContents =
|
||||
"uid 10 100\n" +
|
||||
"gid 10 200\n" +
|
||||
"uid 11 201 # comment at the end of a line\n" +
|
||||
"uid 12 301\n" +
|
||||
"# Comment at the beginning of a line\n" +
|
||||
" # Comment that starts late in the line\n" +
|
||||
"uid 10000 10001# line without whitespace before comment\n" +
|
||||
"uid 13 302\n" +
|
||||
"gid\t11\t201\n" + // Tabs instead of spaces.
|
||||
"\n" + // Entirely empty line.
|
||||
"gid 12 202";
|
||||
OutputStream out = new FileOutputStream(tempStaticMapFile);
|
||||
out.write(staticMapFileContents.getBytes());
|
||||
out.close();
|
||||
StaticMapping parsedMap = IdUserGroup.parseStaticMap(tempStaticMapFile);
|
||||
|
||||
assertEquals(10, (int)parsedMap.uidMapping.get(100));
|
||||
assertEquals(11, (int)parsedMap.uidMapping.get(201));
|
||||
assertEquals(12, (int)parsedMap.uidMapping.get(301));
|
||||
assertEquals(13, (int)parsedMap.uidMapping.get(302));
|
||||
assertEquals(10, (int)parsedMap.gidMapping.get(200));
|
||||
assertEquals(11, (int)parsedMap.gidMapping.get(201));
|
||||
assertEquals(12, (int)parsedMap.gidMapping.get(202));
|
||||
assertEquals(10000, (int)parsedMap.uidMapping.get(10001));
|
||||
// Ensure pass-through of unmapped IDs works.
|
||||
assertEquals(1000, (int)parsedMap.uidMapping.get(1000));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testStaticMapping() throws IOException {
|
||||
Map<Integer, Integer> uidStaticMap = new PassThroughMap<Integer>();
|
||||
Map<Integer, Integer> gidStaticMap = new PassThroughMap<Integer>();
|
||||
|
||||
uidStaticMap.put(11501, 10);
|
||||
gidStaticMap.put(497, 200);
|
||||
|
||||
// Maps for id to name map
|
||||
BiMap<Integer, String> uMap = HashBiMap.create();
|
||||
BiMap<Integer, String> gMap = HashBiMap.create();
|
||||
|
||||
String GET_ALL_USERS_CMD =
|
||||
"echo \"atm:x:1000:1000:Aaron T. Myers,,,:/home/atm:/bin/bash\n"
|
||||
+ "hdfs:x:11501:10787:Grid Distributed File System:/home/hdfs:/bin/bash\""
|
||||
+ " | cut -d: -f1,3";
|
||||
|
||||
String GET_ALL_GROUPS_CMD = "echo \"hdfs:*:11501:hrt_hdfs\n"
|
||||
+ "mapred:x:497\n"
|
||||
+ "mapred2:x:498\""
|
||||
+ " | cut -d: -f1,3";
|
||||
|
||||
IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":",
|
||||
uidStaticMap);
|
||||
IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":",
|
||||
gidStaticMap);
|
||||
|
||||
assertEquals("hdfs", uMap.get(10));
|
||||
assertEquals(10, (int)uMap.inverse().get("hdfs"));
|
||||
assertEquals("atm", uMap.get(1000));
|
||||
assertEquals(1000, (int)uMap.inverse().get("atm"));
|
||||
|
||||
assertEquals("hdfs", gMap.get(11501));
|
||||
assertEquals(11501, (int)gMap.inverse().get("hdfs"));
|
||||
assertEquals("mapred", gMap.get(200));
|
||||
assertEquals(200, (int)gMap.inverse().get("mapred"));
|
||||
assertEquals("mapred2", gMap.get(498));
|
||||
assertEquals(498, (int)gMap.inverse().get("mapred2"));
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDuplicates() throws IOException {
|
||||
|
@ -51,15 +133,17 @@ public class TestIdUserGroup {
|
|||
BiMap<Integer, String> uMap = HashBiMap.create();
|
||||
BiMap<Integer, String> gMap = HashBiMap.create();
|
||||
|
||||
IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":");
|
||||
assertTrue(uMap.size() == 5);
|
||||
IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":",
|
||||
EMPTY_PASS_THROUGH_MAP);
|
||||
assertEquals(5, uMap.size());
|
||||
assertEquals("root", uMap.get(0));
|
||||
assertEquals("hdfs", uMap.get(11501));
|
||||
assertEquals("hdfs2",uMap.get(11502));
|
||||
assertEquals("bin", uMap.get(2));
|
||||
assertEquals("daemon", uMap.get(1));
|
||||
|
||||
IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":");
|
||||
IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":",
|
||||
EMPTY_PASS_THROUGH_MAP);
|
||||
assertTrue(gMap.size() == 3);
|
||||
assertEquals("hdfs",gMap.get(11501));
|
||||
assertEquals("mapred", gMap.get(497));
|
||||
|
@ -90,7 +174,8 @@ public class TestIdUserGroup {
|
|||
BiMap<Integer, String> uMap = HashBiMap.create();
|
||||
BiMap<Integer, String> gMap = HashBiMap.create();
|
||||
|
||||
IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":");
|
||||
IdUserGroup.updateMapInternal(uMap, "user", GET_ALL_USERS_CMD, ":",
|
||||
EMPTY_PASS_THROUGH_MAP);
|
||||
assertTrue(uMap.size() == 7);
|
||||
assertEquals("nfsnobody", uMap.get(-2));
|
||||
assertEquals("nfsnobody1", uMap.get(-1));
|
||||
|
@ -100,7 +185,8 @@ public class TestIdUserGroup {
|
|||
assertEquals("hdfs",uMap.get(11501));
|
||||
assertEquals("daemon", uMap.get(2));
|
||||
|
||||
IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":");
|
||||
IdUserGroup.updateMapInternal(gMap, "group", GET_ALL_GROUPS_CMD, ":",
|
||||
EMPTY_PASS_THROUGH_MAP);
|
||||
assertTrue(gMap.size() == 7);
|
||||
assertEquals("hdfs",gMap.get(11501));
|
||||
assertEquals("rpcuser", gMap.get(29));
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop HDFS over HTTP ${project.version} - Server Setup
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
This page explains how to quickly setup HttpFS with Pseudo authentication
|
||||
against a Hadoop cluster with Pseudo authentication.
|
||||
|
||||
|
@ -159,5 +157,3 @@ $ keytool -genkey -alias tomcat -keyalg RSA
|
|||
<<<swebhdfs://>>> scheme. Make sure the JVM is picking up the truststore
|
||||
containing the public key of the SSL certificate if using a self-signed
|
||||
certificate.
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop HDFS over HTTP ${project.version} - Using HTTP Tools
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* Security
|
||||
|
||||
Out of the box HttpFS supports both pseudo authentication and Kerberos HTTP
|
||||
|
@ -87,5 +85,3 @@ $ curl --negotiate -u foo -c ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v
|
|||
+---+
|
||||
$ curl -b ~/.httpfsauth "http://<HTTPFS_HOST>:14000/webhdfs/v1?op=liststatus"
|
||||
+---+
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
|
|
@ -54,6 +54,7 @@ import org.apache.hadoop.nfs.nfs3.response.WccData;
|
|||
import org.apache.hadoop.oncrpc.XDR;
|
||||
import org.apache.hadoop.oncrpc.security.VerifierNone;
|
||||
import org.apache.hadoop.util.Daemon;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.jboss.netty.channel.Channel;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
@ -136,7 +137,7 @@ class OpenFileCtx {
|
|||
this.channel = channel;
|
||||
this.xid = xid;
|
||||
this.preOpAttr = preOpAttr;
|
||||
this.startTime = System.currentTimeMillis();
|
||||
this.startTime = Time.monotonicNow();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -158,11 +159,11 @@ class OpenFileCtx {
|
|||
private Daemon dumpThread;
|
||||
|
||||
private void updateLastAccessTime() {
|
||||
lastAccessTime = System.currentTimeMillis();
|
||||
lastAccessTime = Time.monotonicNow();
|
||||
}
|
||||
|
||||
private boolean checkStreamTimeout(long streamTimeout) {
|
||||
return System.currentTimeMillis() - lastAccessTime > streamTimeout;
|
||||
return Time.monotonicNow() - lastAccessTime > streamTimeout;
|
||||
}
|
||||
|
||||
long getLastAccessTime() {
|
||||
|
@ -696,7 +697,7 @@ class OpenFileCtx {
|
|||
+ " updating the mtime, then return success");
|
||||
Nfs3FileAttributes postOpAttr = null;
|
||||
try {
|
||||
dfsClient.setTimes(path, System.currentTimeMillis(), -1);
|
||||
dfsClient.setTimes(path, Time.monotonicNow(), -1);
|
||||
postOpAttr = Nfs3Utils.getFileAttr(dfsClient, path, iug);
|
||||
} catch (IOException e) {
|
||||
LOG.info("Got error when processing perfect overwrite, path=" + path
|
||||
|
@ -997,7 +998,7 @@ class OpenFileCtx {
|
|||
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("FileId: " + latestAttr.getFileId() + " Service time:"
|
||||
+ (System.currentTimeMillis() - commit.getStartTime())
|
||||
+ (Time.monotonicNow() - commit.getStartTime())
|
||||
+ "ms. Sent response for commit:" + commit);
|
||||
}
|
||||
entry = pendingCommits.firstEntry();
|
||||
|
|
|
@ -28,6 +28,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
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;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
|
@ -99,7 +100,7 @@ class OpenFileCtxCache {
|
|||
LOG.warn("No eviction candidate. All streams have pending work.");
|
||||
return null;
|
||||
} else {
|
||||
long idleTime = System.currentTimeMillis()
|
||||
long idleTime = Time.monotonicNow()
|
||||
- idlest.getValue().getLastAccessTime();
|
||||
if (idleTime < Nfs3Constant.OUTPUT_STREAM_TIMEOUT_MIN_DEFAULT) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -250,7 +251,7 @@ class OpenFileCtxCache {
|
|||
|
||||
// Check if it can sleep
|
||||
try {
|
||||
long workedTime = System.currentTimeMillis() - lastWakeupTime;
|
||||
long workedTime = Time.monotonicNow() - lastWakeupTime;
|
||||
if (workedTime < rotation) {
|
||||
if (LOG.isTraceEnabled()) {
|
||||
LOG.trace("StreamMonitor can still have a sleep:"
|
||||
|
@ -258,7 +259,7 @@ class OpenFileCtxCache {
|
|||
}
|
||||
Thread.sleep(rotation - workedTime);
|
||||
}
|
||||
lastWakeupTime = System.currentTimeMillis();
|
||||
lastWakeupTime = Time.monotonicNow();
|
||||
|
||||
} catch (InterruptedException e) {
|
||||
LOG.info("StreamMonitor got interrupted");
|
||||
|
@ -267,4 +268,4 @@ class OpenFileCtxCache {
|
|||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -44,7 +44,6 @@ import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.nfs.AccessPrivilege;
|
||||
import org.apache.hadoop.nfs.NfsExports;
|
||||
|
@ -124,6 +123,7 @@ import org.apache.hadoop.oncrpc.security.VerifierNone;
|
|||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.AuthorizationException;
|
||||
import org.jboss.netty.buffer.ChannelBuffer;
|
||||
import org.jboss.netty.buffer.ChannelBuffers;
|
||||
import org.jboss.netty.channel.Channel;
|
||||
|
@ -151,13 +151,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
|
||||
private final NfsExports exports;
|
||||
|
||||
/**
|
||||
* superUserClient should always impersonate HDFS file system owner to send
|
||||
* requests which requires supergroup privilege. This requires the same user
|
||||
* to start HDFS and NFS.
|
||||
*/
|
||||
private final DFSClient superUserClient;
|
||||
|
||||
private final short replication;
|
||||
private final long blockSize;
|
||||
private final int bufferSize;
|
||||
|
@ -179,7 +172,6 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
exports = NfsExports.getInstance(config);
|
||||
writeManager = new WriteManager(iug, config);
|
||||
clientCache = new DFSClientCache(config);
|
||||
superUserClient = new DFSClient(NameNode.getAddress(config), config);
|
||||
replication = (short) config.getInt(DFSConfigKeys.DFS_REPLICATION_KEY,
|
||||
DFSConfigKeys.DFS_REPLICATION_DEFAULT);
|
||||
blockSize = config.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY,
|
||||
|
@ -270,6 +262,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
Nfs3FileAttributes attrs = null;
|
||||
try {
|
||||
attrs = writeManager.getFileAttr(dfsClient, handle, iug);
|
||||
} catch (RemoteException r) {
|
||||
LOG.warn("Exception ", r);
|
||||
IOException io = r.unwrapRemoteException();
|
||||
/**
|
||||
* AuthorizationException can be thrown if the user can't be proxy'ed.
|
||||
*/
|
||||
if (io instanceof AuthorizationException) {
|
||||
return new GETATTR3Response(Nfs3Status.NFS3ERR_ACCES);
|
||||
} else {
|
||||
return new GETATTR3Response(Nfs3Status.NFS3ERR_IO);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.info("Can't get file attribute, fileId=" + handle.getFileId(), e);
|
||||
response.setStatus(Nfs3Status.NFS3ERR_IO);
|
||||
|
@ -499,6 +502,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
securityHandler.getUid(), securityHandler.getGid(), attrs);
|
||||
|
||||
return new ACCESS3Response(Nfs3Status.NFS3_OK, attrs, access);
|
||||
} catch (RemoteException r) {
|
||||
LOG.warn("Exception ", r);
|
||||
IOException io = r.unwrapRemoteException();
|
||||
/**
|
||||
* AuthorizationException can be thrown if the user can't be proxy'ed.
|
||||
*/
|
||||
if (io instanceof AuthorizationException) {
|
||||
return new ACCESS3Response(Nfs3Status.NFS3ERR_ACCES);
|
||||
} else {
|
||||
return new ACCESS3Response(Nfs3Status.NFS3ERR_IO);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception ", e);
|
||||
return new ACCESS3Response(Nfs3Status.NFS3ERR_IO);
|
||||
|
@ -1658,8 +1672,7 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
}
|
||||
|
||||
try {
|
||||
// Use superUserClient to get file system status
|
||||
FsStatus fsStatus = superUserClient.getDiskStatus();
|
||||
FsStatus fsStatus = dfsClient.getDiskStatus();
|
||||
long totalBytes = fsStatus.getCapacity();
|
||||
long freeBytes = fsStatus.getRemaining();
|
||||
|
||||
|
@ -1680,6 +1693,17 @@ public class RpcProgramNfs3 extends RpcProgram implements Nfs3Interface {
|
|||
|
||||
return new FSSTAT3Response(Nfs3Status.NFS3_OK, attrs, totalBytes,
|
||||
freeBytes, freeBytes, maxFsObjects, maxFsObjects, maxFsObjects, 0);
|
||||
} catch (RemoteException r) {
|
||||
LOG.warn("Exception ", r);
|
||||
IOException io = r.unwrapRemoteException();
|
||||
/**
|
||||
* AuthorizationException can be thrown if the user can't be proxy'ed.
|
||||
*/
|
||||
if (io instanceof AuthorizationException) {
|
||||
return new FSSTAT3Response(Nfs3Status.NFS3ERR_ACCES);
|
||||
} else {
|
||||
return new FSSTAT3Response(Nfs3Status.NFS3ERR_IO);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception ", e);
|
||||
return new FSSTAT3Response(Nfs3Status.NFS3ERR_IO);
|
||||
|
|
|
@ -22,7 +22,6 @@ import static org.junit.Assert.assertTrue;
|
|||
|
||||
import java.io.IOException;
|
||||
import java.net.InetAddress;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
@ -41,9 +40,8 @@ import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response;
|
|||
import org.apache.hadoop.nfs.nfs3.response.READDIRPLUS3Response.EntryPlus3;
|
||||
import org.apache.hadoop.oncrpc.XDR;
|
||||
import org.apache.hadoop.oncrpc.security.SecurityHandler;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.junit.AfterClass;
|
||||
import org.junit.Before;
|
||||
import org.junit.BeforeClass;
|
||||
|
@ -67,10 +65,10 @@ public class TestReaddir {
|
|||
public static void setup() throws Exception {
|
||||
String currentUser = System.getProperty("user.name");
|
||||
config.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(currentUser),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(currentUser),
|
||||
"*");
|
||||
config.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(currentUser),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(currentUser),
|
||||
"*");
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(config);
|
||||
cluster = new MiniDFSCluster.Builder(config).numDataNodes(1).build();
|
||||
|
|
|
@ -50,6 +50,7 @@ import org.apache.hadoop.nfs.nfs3.response.CREATE3Response;
|
|||
import org.apache.hadoop.nfs.nfs3.response.READ3Response;
|
||||
import org.apache.hadoop.oncrpc.XDR;
|
||||
import org.apache.hadoop.oncrpc.security.SecurityHandler;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.jboss.netty.channel.Channel;
|
||||
import org.junit.Assert;
|
||||
|
@ -288,10 +289,10 @@ public class TestWrites {
|
|||
System.getProperty("user.name"));
|
||||
String currentUser = System.getProperty("user.name");
|
||||
config.set(
|
||||
ProxyUsers.getProxySuperuserGroupConfKey(currentUser),
|
||||
DefaultImpersonationProvider.getProxySuperuserGroupConfKey(currentUser),
|
||||
"*");
|
||||
config.set(
|
||||
ProxyUsers.getProxySuperuserIpConfKey(currentUser),
|
||||
DefaultImpersonationProvider.getProxySuperuserIpConfKey(currentUser),
|
||||
"*");
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(config);
|
||||
|
||||
|
|
|
@ -317,6 +317,9 @@ Trunk (Unreleased)
|
|||
HDFS-6374. setXAttr should require the user to be the owner of the file
|
||||
or directory (Charles Lamb via wang)
|
||||
|
||||
HDFS-6110 adding more slow action log in critical write path
|
||||
(Liang Xie via stack)
|
||||
|
||||
Release 2.5.0 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -440,6 +443,17 @@ Release 2.5.0 - UNRELEASED
|
|||
HDFS-6396. Remove support for ACL feature from INodeSymlink.
|
||||
(Charles Lamb via wang)
|
||||
|
||||
HDFS-6435. Add support for specifying a static uid/gid mapping for the NFS
|
||||
gateway. (atm via wang)
|
||||
|
||||
HDFS-6416. Use Time#monotonicNow in OpenFileCtx and OpenFileCtxCatch to
|
||||
avoid system clock bugs (Abhiraj Butala via brandonli)
|
||||
|
||||
HDFS-6356. Fix typo in DatanodeLayoutVersion. (Tulasi G via wang)
|
||||
|
||||
HDFS-6447. balancer should timestamp the completion message.
|
||||
(Juan Yu via wang).
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
HDFS-6214. Webhdfs has poor throughput for files >2GB (daryn)
|
||||
|
@ -583,6 +597,26 @@ Release 2.5.0 - UNRELEASED
|
|||
HDFS-6423. Diskspace quota usage should be updated when appending data to
|
||||
partial block. (jing9)
|
||||
|
||||
HDFS-6443. Fix MiniQJMHACluster related test failures. (Zesheng Wu via
|
||||
Arpit Agarwal)
|
||||
|
||||
HDFS-6227. ShortCircuitCache#unref should purge ShortCircuitReplicas whose
|
||||
streams have been closed by java interrupts. (Colin Patrick McCabe via jing9)
|
||||
|
||||
HDFS-6442. Fix TestEditLogAutoroll and TestStandbyCheckpoints failure
|
||||
caused by port conficts. (Zesheng Wu via Arpit Agarwal)
|
||||
|
||||
HDFS-6448. BlockReaderLocalLegacy should set socket timeout based on
|
||||
conf.socketTimeout (liangxie via cmccabe)
|
||||
|
||||
HDFS-6453. Use Time#monotonicNow to avoid system clock reset.
|
||||
(Liang Xie via wang)
|
||||
|
||||
HDFS-6461. Use Time#monotonicNow to compute duration in DataNode#shutDown.
|
||||
(James Thomas via wang)
|
||||
|
||||
HDFS-6462. NFS: fsstat request fails with the secure hdfs (brandonli)
|
||||
|
||||
Release 2.4.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
@ -666,6 +700,9 @@ Release 2.4.1 - UNRELEASED
|
|||
HDFS-6397. NN shows inconsistent value in deadnode count.
|
||||
(Mohammad Kamrul Islam via kihwal)
|
||||
|
||||
HDFS-6411. nfs-hdfs-gateway mount raises I/O error and hangs when a
|
||||
unauthorized user attempts to access it (brandonli)
|
||||
|
||||
Release 2.4.0 - 2014-04-07
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -116,7 +116,7 @@ public class Hdfs extends AbstractFileSystem {
|
|||
@Override
|
||||
public FileChecksum getFileChecksum(Path f)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
return dfs.getFileChecksum(getUriPath(f));
|
||||
return dfs.getFileChecksum(getUriPath(f), Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -187,7 +187,7 @@ class BlockReaderLocalLegacy implements BlockReader {
|
|||
userGroupInformation = UserGroupInformation.getCurrentUser();
|
||||
}
|
||||
pathinfo = getBlockPathInfo(userGroupInformation, blk, node,
|
||||
configuration, conf.hdfsTimeout, token,
|
||||
configuration, conf.socketTimeout, token,
|
||||
conf.connectToDnViaHostname);
|
||||
}
|
||||
|
||||
|
|
|
@ -276,6 +276,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
final int retryTimesForGetLastBlockLength;
|
||||
final int retryIntervalForGetLastBlockLength;
|
||||
final long datanodeRestartTimeout;
|
||||
final long dfsclientSlowIoWarningThresholdMs;
|
||||
|
||||
final boolean useLegacyBlockReader;
|
||||
final boolean useLegacyBlockReaderLocal;
|
||||
|
@ -430,6 +431,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
datanodeRestartTimeout = conf.getLong(
|
||||
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_KEY,
|
||||
DFS_CLIENT_DATANODE_RESTART_TIMEOUT_DEFAULT) * 1000;
|
||||
dfsclientSlowIoWarningThresholdMs = conf.getLong(
|
||||
DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
|
||||
}
|
||||
|
||||
public boolean isUseLegacyBlockReaderLocal() {
|
||||
|
@ -1801,15 +1805,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the checksum of a file.
|
||||
* Get the checksum of the whole file of a range of the file. Note that the
|
||||
* range always starts from the beginning of the file.
|
||||
* @param src The file path
|
||||
* @param length The length of the range
|
||||
* @return The checksum
|
||||
* @see DistributedFileSystem#getFileChecksum(Path)
|
||||
*/
|
||||
public MD5MD5CRC32FileChecksum getFileChecksum(String src) throws IOException {
|
||||
public MD5MD5CRC32FileChecksum getFileChecksum(String src, long length)
|
||||
throws IOException {
|
||||
checkOpen();
|
||||
return getFileChecksum(src, clientName, namenode, socketFactory,
|
||||
dfsClientConf.socketTimeout, getDataEncryptionKey(),
|
||||
Preconditions.checkArgument(length >= 0);
|
||||
return getFileChecksum(src, length, clientName, namenode,
|
||||
socketFactory, dfsClientConf.socketTimeout, getDataEncryptionKey(),
|
||||
dfsClientConf.connectToDnViaHostname);
|
||||
}
|
||||
|
||||
|
@ -1850,8 +1858,9 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
}
|
||||
|
||||
/**
|
||||
* Get the checksum of a file.
|
||||
* Get the checksum of the whole file or a range of the file.
|
||||
* @param src The file path
|
||||
* @param length the length of the range, i.e., the range is [0, length]
|
||||
* @param clientName the name of the client requesting the checksum.
|
||||
* @param namenode the RPC proxy for the namenode
|
||||
* @param socketFactory to create sockets to connect to DNs
|
||||
|
@ -1861,12 +1870,13 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
* @return The checksum
|
||||
*/
|
||||
private static MD5MD5CRC32FileChecksum getFileChecksum(String src,
|
||||
String clientName,
|
||||
ClientProtocol namenode, SocketFactory socketFactory, int socketTimeout,
|
||||
long length, String clientName, ClientProtocol namenode,
|
||||
SocketFactory socketFactory, int socketTimeout,
|
||||
DataEncryptionKey encryptionKey, boolean connectToDnViaHostname)
|
||||
throws IOException {
|
||||
//get all block locations
|
||||
LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0, Long.MAX_VALUE);
|
||||
//get block locations for the file range
|
||||
LocatedBlocks blockLocations = callGetBlockLocations(namenode, src, 0,
|
||||
length);
|
||||
if (null == blockLocations) {
|
||||
throw new FileNotFoundException("File does not exist: " + src);
|
||||
}
|
||||
|
@ -1878,10 +1888,11 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
boolean refetchBlocks = false;
|
||||
int lastRetriedIndex = -1;
|
||||
|
||||
//get block checksum for each block
|
||||
for(int i = 0; i < locatedblocks.size(); i++) {
|
||||
// get block checksum for each block
|
||||
long remaining = length;
|
||||
for(int i = 0; i < locatedblocks.size() && remaining > 0; i++) {
|
||||
if (refetchBlocks) { // refetch to get fresh tokens
|
||||
blockLocations = callGetBlockLocations(namenode, src, 0, Long.MAX_VALUE);
|
||||
blockLocations = callGetBlockLocations(namenode, src, 0, length);
|
||||
if (null == blockLocations) {
|
||||
throw new FileNotFoundException("File does not exist: " + src);
|
||||
}
|
||||
|
@ -1890,6 +1901,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory {
|
|||
}
|
||||
LocatedBlock lb = locatedblocks.get(i);
|
||||
final ExtendedBlock block = lb.getBlock();
|
||||
if (remaining < block.getNumBytes()) {
|
||||
block.setNumBytes(remaining);
|
||||
}
|
||||
remaining -= block.getNumBytes();
|
||||
final DatanodeInfo[] datanodes = lb.getLocations();
|
||||
|
||||
//try each datanode location of the block
|
||||
|
|
|
@ -645,5 +645,12 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
|
|||
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 =
|
||||
"dfs.client.slow.io.warning.threshold.ms";
|
||||
public static final long DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 30000;
|
||||
public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY =
|
||||
"dfs.datanode.slow.io.warning.threshold.ms";
|
||||
public static final long DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT = 300;
|
||||
}
|
||||
|
|
|
@ -122,6 +122,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
implements Syncable, CanSetDropBehind {
|
||||
private static final int MAX_PACKETS = 80; // each packet 64K, total 5MB
|
||||
private final DFSClient dfsClient;
|
||||
private final long dfsclientSlowLogThresholdMs;
|
||||
private Socket s;
|
||||
// closed is accessed by different threads under different locks.
|
||||
private volatile boolean closed = false;
|
||||
|
@ -788,11 +789,19 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
// process responses from datanodes.
|
||||
try {
|
||||
// read an ack from the pipeline
|
||||
long begin = Time.monotonicNow();
|
||||
ack.readFields(blockReplyStream);
|
||||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > dfsclientSlowLogThresholdMs
|
||||
&& ack.getSeqno() != Packet.HEART_BEAT_SEQNO) {
|
||||
DFSClient.LOG
|
||||
.warn("Slow ReadProcessor read fields took " + duration
|
||||
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms); ack: "
|
||||
+ ack + ", targets: " + Arrays.asList(targets));
|
||||
} else if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("DFSClient " + ack);
|
||||
}
|
||||
|
||||
|
||||
long seqno = ack.getSeqno();
|
||||
// processes response status from datanodes.
|
||||
for (int i = ack.getNumOfReplies()-1; i >=0 && dfsClient.clientRunning; i--) {
|
||||
|
@ -1570,6 +1579,8 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
|
||||
}
|
||||
this.checksum = checksum;
|
||||
this.dfsclientSlowLogThresholdMs =
|
||||
dfsClient.getConf().dfsclientSlowIoWarningThresholdMs;
|
||||
}
|
||||
|
||||
/** Construct a new output stream for creating a file. */
|
||||
|
@ -2001,6 +2012,7 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
if (DFSClient.LOG.isDebugEnabled()) {
|
||||
DFSClient.LOG.debug("Waiting for ack for: " + seqno);
|
||||
}
|
||||
long begin = Time.monotonicNow();
|
||||
try {
|
||||
synchronized (dataQueue) {
|
||||
while (!closed) {
|
||||
|
@ -2020,6 +2032,11 @@ public class DFSOutputStream extends FSOutputSummer
|
|||
checkClosed();
|
||||
} catch (ClosedChannelException e) {
|
||||
}
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > dfsclientSlowLogThresholdMs) {
|
||||
DFSClient.LOG.warn("Slow waitForAckedSeqno took " + duration
|
||||
+ "ms (threshold=" + dfsclientSlowLogThresholdMs + "ms)");
|
||||
}
|
||||
}
|
||||
|
||||
private synchronized void start() {
|
||||
|
|
|
@ -68,14 +68,12 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
|
|||
import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
||||
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
|
||||
import org.apache.hadoop.hdfs.protocol.LocatedBlock;
|
||||
import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
|
||||
import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
|
||||
|
@ -85,7 +83,6 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
|
@ -1142,7 +1139,7 @@ public class DistributedFileSystem extends FileSystem {
|
|||
@Override
|
||||
public FileChecksum doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
return dfs.getFileChecksum(getPathName(p));
|
||||
return dfs.getFileChecksum(getPathName(p), Long.MAX_VALUE);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1153,6 +1150,32 @@ public class DistributedFileSystem extends FileSystem {
|
|||
}.resolve(this, absF);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChecksum getFileChecksum(Path f, final long length)
|
||||
throws IOException {
|
||||
statistics.incrementReadOps(1);
|
||||
Path absF = fixRelativePart(f);
|
||||
return new FileSystemLinkResolver<FileChecksum>() {
|
||||
@Override
|
||||
public FileChecksum doCall(final Path p)
|
||||
throws IOException, UnresolvedLinkException {
|
||||
return dfs.getFileChecksum(getPathName(p), length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileChecksum next(final FileSystem fs, final Path p)
|
||||
throws IOException {
|
||||
if (fs instanceof DistributedFileSystem) {
|
||||
return ((DistributedFileSystem) fs).getFileChecksum(p, length);
|
||||
} else {
|
||||
throw new UnsupportedFileSystemException(
|
||||
"getFileChecksum(Path, long) is not supported by "
|
||||
+ fs.getClass().getSimpleName());
|
||||
}
|
||||
}
|
||||
}.resolve(this, absF);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setPermission(Path p, final FsPermission permission
|
||||
) throws IOException {
|
||||
|
|
|
@ -1552,6 +1552,7 @@ public class Balancer {
|
|||
System.out.println(e + ". Exiting ...");
|
||||
return ReturnStatus.INTERRUPTED.code;
|
||||
} finally {
|
||||
System.out.format("%-24s ", DateFormat.getDateTimeInstance().format(new Date()));
|
||||
System.out.println("Balancing took " + time2Str(Time.now()-startTime));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -69,7 +69,7 @@ class BlockReceiver implements Closeable {
|
|||
|
||||
@VisibleForTesting
|
||||
static long CACHE_DROP_LAG_BYTES = 8 * 1024 * 1024;
|
||||
|
||||
private final long datanodeSlowLogThresholdMs;
|
||||
private DataInputStream in = null; // from where data are read
|
||||
private DataChecksum clientChecksum; // checksum used by client
|
||||
private DataChecksum diskChecksum; // checksum we write to disk
|
||||
|
@ -140,7 +140,7 @@ class BlockReceiver implements Closeable {
|
|||
this.isDatanode = clientname.length() == 0;
|
||||
this.isClient = !this.isDatanode;
|
||||
this.restartBudget = datanode.getDnConf().restartReplicaExpiry;
|
||||
|
||||
this.datanodeSlowLogThresholdMs = datanode.getDnConf().datanodeSlowIoWarningThresholdMs;
|
||||
//for datanode, we have
|
||||
//1: clientName.length() == 0, and
|
||||
//2: stage == null or PIPELINE_SETUP_CREATE
|
||||
|
@ -335,6 +335,7 @@ class BlockReceiver implements Closeable {
|
|||
*/
|
||||
void flushOrSync(boolean isSync) throws IOException {
|
||||
long flushTotalNanos = 0;
|
||||
long begin = Time.monotonicNow();
|
||||
if (checksumOut != null) {
|
||||
long flushStartNanos = System.nanoTime();
|
||||
checksumOut.flush();
|
||||
|
@ -363,6 +364,12 @@ class BlockReceiver implements Closeable {
|
|||
datanode.metrics.incrFsyncCount();
|
||||
}
|
||||
}
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > datanodeSlowLogThresholdMs) {
|
||||
LOG.warn("Slow flushOrSync took " + duration + "ms (threshold="
|
||||
+ datanodeSlowLogThresholdMs + "ms), isSync:" + isSync + ", flushTotalNanos="
|
||||
+ flushTotalNanos + "ns");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -488,8 +495,14 @@ class BlockReceiver implements Closeable {
|
|||
//First write the packet to the mirror:
|
||||
if (mirrorOut != null && !mirrorError) {
|
||||
try {
|
||||
long begin = Time.monotonicNow();
|
||||
packetReceiver.mirrorPacketTo(mirrorOut);
|
||||
mirrorOut.flush();
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > datanodeSlowLogThresholdMs) {
|
||||
LOG.warn("Slow BlockReceiver write packet to mirror took " + duration
|
||||
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
|
||||
}
|
||||
} catch (IOException e) {
|
||||
handleMirrorOutError(e);
|
||||
}
|
||||
|
@ -572,7 +585,13 @@ class BlockReceiver implements Closeable {
|
|||
int numBytesToDisk = (int)(offsetInBlock-onDiskLen);
|
||||
|
||||
// Write data to disk.
|
||||
long begin = Time.monotonicNow();
|
||||
out.write(dataBuf.array(), startByteToDisk, numBytesToDisk);
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > datanodeSlowLogThresholdMs) {
|
||||
LOG.warn("Slow BlockReceiver write data to disk cost:" + duration
|
||||
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
|
||||
}
|
||||
|
||||
// If this is a partial chunk, then verify that this is the only
|
||||
// chunk in the packet. Calculate new crc for this chunk.
|
||||
|
@ -638,6 +657,7 @@ class BlockReceiver implements Closeable {
|
|||
try {
|
||||
if (outFd != null &&
|
||||
offsetInBlock > lastCacheManagementOffset + CACHE_DROP_LAG_BYTES) {
|
||||
long begin = Time.monotonicNow();
|
||||
//
|
||||
// For SYNC_FILE_RANGE_WRITE, we want to sync from
|
||||
// lastCacheManagementOffset to a position "two windows ago"
|
||||
|
@ -670,6 +690,11 @@ class BlockReceiver implements Closeable {
|
|||
NativeIO.POSIX.POSIX_FADV_DONTNEED);
|
||||
}
|
||||
lastCacheManagementOffset = offsetInBlock;
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > datanodeSlowLogThresholdMs) {
|
||||
LOG.warn("Slow manageWriterOsCache took " + duration
|
||||
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms)");
|
||||
}
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.warn("Error managing cache for writer of block " + block, t);
|
||||
|
@ -1299,9 +1324,15 @@ class BlockReceiver implements Closeable {
|
|||
replicaInfo.setBytesAcked(offsetInBlock);
|
||||
}
|
||||
// send my ack back to upstream datanode
|
||||
long begin = Time.monotonicNow();
|
||||
replyAck.write(upstreamOut);
|
||||
upstreamOut.flush();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
long duration = Time.monotonicNow() - begin;
|
||||
if (duration > datanodeSlowLogThresholdMs) {
|
||||
LOG.warn("Slow PacketResponder send ack to upstream took " + duration
|
||||
+ "ms (threshold=" + datanodeSlowLogThresholdMs + "ms), " + myString
|
||||
+ ", replyAck=" + replyAck);
|
||||
} else if (LOG.isDebugEnabled()) {
|
||||
LOG.debug(myString + ", replyAck=" + replyAck);
|
||||
}
|
||||
|
||||
|
|
|
@ -79,6 +79,8 @@ public class DNConf {
|
|||
final long deleteReportInterval;
|
||||
final long initialBlockReportDelay;
|
||||
final long cacheReportInterval;
|
||||
final long dfsclientSlowIoWarningThresholdMs;
|
||||
final long datanodeSlowIoWarningThresholdMs;
|
||||
final int writePacketSize;
|
||||
|
||||
final String minimumNameNodeVersion;
|
||||
|
@ -129,7 +131,14 @@ public class DNConf {
|
|||
DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
|
||||
this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
|
||||
DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT);
|
||||
|
||||
|
||||
this.dfsclientSlowIoWarningThresholdMs = conf.getLong(
|
||||
DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
|
||||
DFSConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
|
||||
this.datanodeSlowIoWarningThresholdMs = conf.getLong(
|
||||
DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY,
|
||||
DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
|
||||
|
||||
long initBRDelay = conf.getLong(
|
||||
DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
||||
DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT) * 1000L;
|
||||
|
@ -168,7 +177,7 @@ public class DNConf {
|
|||
DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY,
|
||||
DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT) * 1000L;
|
||||
}
|
||||
|
||||
|
||||
// We get minimumNameNodeVersion via a method so it can be mocked out in tests.
|
||||
String getMinimumNameNodeVersion() {
|
||||
return this.minimumNameNodeVersion;
|
||||
|
|
|
@ -1221,7 +1221,7 @@ public class DataNode extends Configured
|
|||
}
|
||||
|
||||
// Record the time of initial notification
|
||||
long timeNotified = Time.now();
|
||||
long timeNotified = Time.monotonicNow();
|
||||
|
||||
if (localDataXceiverServer != null) {
|
||||
((DataXceiverServer) this.localDataXceiverServer.getRunnable()).kill();
|
||||
|
@ -1253,8 +1253,9 @@ public class DataNode extends Configured
|
|||
while (true) {
|
||||
// When shutting down for restart, wait 2.5 seconds before forcing
|
||||
// termination of receiver threads.
|
||||
if (!this.shutdownForUpgrade ||
|
||||
(this.shutdownForUpgrade && (Time.now() - timeNotified > 2500))) {
|
||||
if (!this.shutdownForUpgrade ||
|
||||
(this.shutdownForUpgrade && (Time.monotonicNow() - timeNotified
|
||||
> 2500))) {
|
||||
this.threadGroup.interrupt();
|
||||
}
|
||||
LOG.info("Waiting for threadgroup to exit, active threads is " +
|
||||
|
@ -2581,7 +2582,7 @@ public class DataNode extends Configured
|
|||
return;
|
||||
}
|
||||
synchronized(checkDiskErrorMutex) {
|
||||
lastDiskErrorCheck = System.currentTimeMillis();
|
||||
lastDiskErrorCheck = Time.monotonicNow();
|
||||
}
|
||||
}
|
||||
try {
|
||||
|
|
|
@ -62,7 +62,7 @@ public class DataNodeLayoutVersion {
|
|||
* </ul>
|
||||
*/
|
||||
public static enum Feature implements LayoutFeature {
|
||||
FIRST_LAYOUT(-55, -53, "First datenode layout", false);
|
||||
FIRST_LAYOUT(-55, -53, "First datanode layout", false);
|
||||
|
||||
private final FeatureInfo info;
|
||||
|
||||
|
|
|
@ -42,6 +42,7 @@ import java.net.Socket;
|
|||
import java.net.SocketException;
|
||||
import java.net.UnknownHostException;
|
||||
import java.nio.channels.ClosedChannelException;
|
||||
import java.security.MessageDigest;
|
||||
import java.util.Arrays;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
|
@ -83,6 +84,7 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.net.InetAddresses;
|
||||
import com.google.protobuf.ByteString;
|
||||
|
||||
|
@ -802,7 +804,44 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
IOUtils.closeStream(out);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
private MD5Hash calcPartialBlockChecksum(ExtendedBlock block,
|
||||
long requestLength, DataChecksum checksum, DataInputStream checksumIn)
|
||||
throws IOException {
|
||||
final int bytesPerCRC = checksum.getBytesPerChecksum();
|
||||
final int csize = checksum.getChecksumSize();
|
||||
final byte[] buffer = new byte[4*1024];
|
||||
MessageDigest digester = MD5Hash.getDigester();
|
||||
|
||||
long remaining = requestLength / bytesPerCRC * csize;
|
||||
for (int toDigest = 0; remaining > 0; remaining -= toDigest) {
|
||||
toDigest = checksumIn.read(buffer, 0,
|
||||
(int) Math.min(remaining, buffer.length));
|
||||
if (toDigest < 0) {
|
||||
break;
|
||||
}
|
||||
digester.update(buffer, 0, toDigest);
|
||||
}
|
||||
|
||||
int partialLength = (int) (requestLength % bytesPerCRC);
|
||||
if (partialLength > 0) {
|
||||
byte[] buf = new byte[partialLength];
|
||||
final InputStream blockIn = datanode.data.getBlockInputStream(block,
|
||||
requestLength - partialLength);
|
||||
try {
|
||||
// Get the CRC of the partialLength.
|
||||
IOUtils.readFully(blockIn, buf, 0, partialLength);
|
||||
} finally {
|
||||
IOUtils.closeStream(blockIn);
|
||||
}
|
||||
checksum.update(buf, 0, partialLength);
|
||||
byte[] partialCrc = new byte[csize];
|
||||
checksum.writeValue(partialCrc, 0, true);
|
||||
digester.update(partialCrc);
|
||||
}
|
||||
return new MD5Hash(digester.digest());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void blockChecksum(final ExtendedBlock block,
|
||||
final Token<BlockTokenIdentifier> blockToken) throws IOException {
|
||||
|
@ -810,25 +849,32 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
getOutputStream());
|
||||
checkAccess(out, true, block, blockToken,
|
||||
Op.BLOCK_CHECKSUM, BlockTokenSecretManager.AccessMode.READ);
|
||||
updateCurrentThreadName("Reading metadata for block " + block);
|
||||
final LengthInputStream metadataIn =
|
||||
datanode.data.getMetaDataInputStream(block);
|
||||
final DataInputStream checksumIn = new DataInputStream(new BufferedInputStream(
|
||||
metadataIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
|
||||
// client side now can specify a range of the block for checksum
|
||||
long requestLength = block.getNumBytes();
|
||||
Preconditions.checkArgument(requestLength >= 0);
|
||||
long visibleLength = datanode.data.getReplicaVisibleLength(block);
|
||||
boolean partialBlk = requestLength < visibleLength;
|
||||
|
||||
updateCurrentThreadName("Reading metadata for block " + block);
|
||||
final LengthInputStream metadataIn = datanode.data
|
||||
.getMetaDataInputStream(block);
|
||||
|
||||
final DataInputStream checksumIn = new DataInputStream(
|
||||
new BufferedInputStream(metadataIn, HdfsConstants.IO_FILE_BUFFER_SIZE));
|
||||
updateCurrentThreadName("Getting checksum for block " + block);
|
||||
try {
|
||||
//read metadata file
|
||||
final BlockMetadataHeader header = BlockMetadataHeader.readHeader(checksumIn);
|
||||
final DataChecksum checksum = header.getChecksum();
|
||||
final BlockMetadataHeader header = BlockMetadataHeader
|
||||
.readHeader(checksumIn);
|
||||
final DataChecksum checksum = header.getChecksum();
|
||||
final int csize = checksum.getChecksumSize();
|
||||
final int bytesPerCRC = checksum.getBytesPerChecksum();
|
||||
final long crcPerBlock = checksum.getChecksumSize() > 0
|
||||
? (metadataIn.getLength() - BlockMetadataHeader.getHeaderSize())/checksum.getChecksumSize()
|
||||
: 0;
|
||||
|
||||
//compute block checksum
|
||||
final MD5Hash md5 = MD5Hash.digest(checksumIn);
|
||||
final long crcPerBlock = csize <= 0 ? 0 :
|
||||
(metadataIn.getLength() - BlockMetadataHeader.getHeaderSize()) / csize;
|
||||
|
||||
final MD5Hash md5 = partialBlk && crcPerBlock > 0 ?
|
||||
calcPartialBlockChecksum(block, requestLength, checksum, checksumIn)
|
||||
: MD5Hash.digest(checksumIn);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("block=" + block + ", bytesPerCRC=" + bytesPerCRC
|
||||
+ ", crcPerBlock=" + crcPerBlock + ", md5=" + md5);
|
||||
|
@ -841,8 +887,7 @@ class DataXceiver extends Receiver implements Runnable {
|
|||
.setBytesPerCrc(bytesPerCRC)
|
||||
.setCrcPerBlock(crcPerBlock)
|
||||
.setMd5(ByteString.copyFrom(md5.getDigest()))
|
||||
.setCrcType(PBHelper.convert(checksum.getChecksumType()))
|
||||
)
|
||||
.setCrcType(PBHelper.convert(checksum.getChecksumType())))
|
||||
.build()
|
||||
.writeDelimitedTo(out);
|
||||
out.flush();
|
||||
|
|
|
@ -23,6 +23,7 @@ import java.util.*;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.VolumeChoosingPolicy;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.apache.hadoop.util.DiskChecker.DiskErrorException;
|
||||
|
||||
class FsVolumeList {
|
||||
|
@ -97,7 +98,7 @@ class FsVolumeList {
|
|||
}
|
||||
|
||||
void getAllVolumesMap(final String bpid, final ReplicaMap volumeMap) throws IOException {
|
||||
long totalStartTime = System.currentTimeMillis();
|
||||
long totalStartTime = Time.monotonicNow();
|
||||
final List<IOException> exceptions = Collections.synchronizedList(
|
||||
new ArrayList<IOException>());
|
||||
List<Thread> replicaAddingThreads = new ArrayList<Thread>();
|
||||
|
@ -107,9 +108,9 @@ class FsVolumeList {
|
|||
try {
|
||||
FsDatasetImpl.LOG.info("Adding replicas to map for block pool " +
|
||||
bpid + " on volume " + v + "...");
|
||||
long startTime = System.currentTimeMillis();
|
||||
long startTime = Time.monotonicNow();
|
||||
v.getVolumeMap(bpid, volumeMap);
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
long timeTaken = Time.monotonicNow() - startTime;
|
||||
FsDatasetImpl.LOG.info("Time to add replicas to map for block pool"
|
||||
+ " " + bpid + " on volume " + v + ": " + timeTaken + "ms");
|
||||
} catch (IOException ioe) {
|
||||
|
@ -132,7 +133,7 @@ class FsVolumeList {
|
|||
if (!exceptions.isEmpty()) {
|
||||
throw exceptions.get(0);
|
||||
}
|
||||
long totalTimeTaken = System.currentTimeMillis() - totalStartTime;
|
||||
long totalTimeTaken = Time.monotonicNow() - totalStartTime;
|
||||
FsDatasetImpl.LOG.info("Total time to add all replicas to map: "
|
||||
+ totalTimeTaken + "ms");
|
||||
}
|
||||
|
@ -141,9 +142,9 @@ class FsVolumeList {
|
|||
throws IOException {
|
||||
FsDatasetImpl.LOG.info("Adding replicas to map for block pool " + bpid +
|
||||
" on volume " + volume + "...");
|
||||
long startTime = System.currentTimeMillis();
|
||||
long startTime = Time.monotonicNow();
|
||||
volume.getVolumeMap(bpid, volumeMap);
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
long timeTaken = Time.monotonicNow() - startTime;
|
||||
FsDatasetImpl.LOG.info("Time to add replicas to map for block pool " + bpid +
|
||||
" on volume " + volume + ": " + timeTaken + "ms");
|
||||
}
|
||||
|
@ -195,7 +196,7 @@ class FsVolumeList {
|
|||
|
||||
|
||||
void addBlockPool(final String bpid, final Configuration conf) throws IOException {
|
||||
long totalStartTime = System.currentTimeMillis();
|
||||
long totalStartTime = Time.monotonicNow();
|
||||
|
||||
final List<IOException> exceptions = Collections.synchronizedList(
|
||||
new ArrayList<IOException>());
|
||||
|
@ -206,9 +207,9 @@ class FsVolumeList {
|
|||
try {
|
||||
FsDatasetImpl.LOG.info("Scanning block pool " + bpid +
|
||||
" on volume " + v + "...");
|
||||
long startTime = System.currentTimeMillis();
|
||||
long startTime = Time.monotonicNow();
|
||||
v.addBlockPool(bpid, conf);
|
||||
long timeTaken = System.currentTimeMillis() - startTime;
|
||||
long timeTaken = Time.monotonicNow() - startTime;
|
||||
FsDatasetImpl.LOG.info("Time taken to scan block pool " + bpid +
|
||||
" on " + v + ": " + timeTaken + "ms");
|
||||
} catch (IOException ioe) {
|
||||
|
@ -232,7 +233,7 @@ class FsVolumeList {
|
|||
throw exceptions.get(0);
|
||||
}
|
||||
|
||||
long totalTimeTaken = System.currentTimeMillis() - totalStartTime;
|
||||
long totalTimeTaken = Time.monotonicNow() - totalStartTime;
|
||||
FsDatasetImpl.LOG.info("Total time to scan all replicas for block pool " +
|
||||
bpid + ": " + totalTimeTaken + "ms");
|
||||
}
|
||||
|
|
|
@ -74,7 +74,6 @@ import org.apache.hadoop.hdfs.web.resources.PutOpParam;
|
|||
import org.apache.hadoop.hdfs.web.resources.ReplicationParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.UriFsPathParam;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
@ -452,7 +451,7 @@ public class DatanodeWebHdfsMethods {
|
|||
MD5MD5CRC32FileChecksum checksum = null;
|
||||
DFSClient dfsclient = newDfsClient(nnId, conf);
|
||||
try {
|
||||
checksum = dfsclient.getFileChecksum(fullpath);
|
||||
checksum = dfsclient.getFileChecksum(fullpath, Long.MAX_VALUE);
|
||||
dfsclient.close();
|
||||
dfsclient = null;
|
||||
} finally {
|
||||
|
|
|
@ -59,6 +59,7 @@ import org.apache.hadoop.hdfs.util.MD5FileUtils;
|
|||
import org.apache.hadoop.io.MD5Hash;
|
||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.io.compress.CompressorStream;
|
||||
import org.apache.hadoop.util.Time;
|
||||
|
||||
import com.google.common.collect.Lists;
|
||||
import com.google.common.collect.Maps;
|
||||
|
@ -160,13 +161,13 @@ public final class FSImageFormatProtobuf {
|
|||
}
|
||||
|
||||
void load(File file) throws IOException {
|
||||
long start = System.currentTimeMillis();
|
||||
long start = Time.monotonicNow();
|
||||
imgDigest = MD5FileUtils.computeMd5ForFile(file);
|
||||
RandomAccessFile raFile = new RandomAccessFile(file, "r");
|
||||
FileInputStream fin = new FileInputStream(file);
|
||||
try {
|
||||
loadInternal(raFile, fin);
|
||||
long end = System.currentTimeMillis();
|
||||
long end = Time.monotonicNow();
|
||||
LOG.info("Loaded FSImage in " + (end - start) / 1000 + " seconds.");
|
||||
} finally {
|
||||
fin.close();
|
||||
|
|
|
@ -159,7 +159,7 @@ public class TransferFsImage {
|
|||
}
|
||||
}
|
||||
|
||||
final long milliTime = System.currentTimeMillis();
|
||||
final long milliTime = Time.monotonicNow();
|
||||
String tmpFileName = NNStorage.getTemporaryEditsFileName(
|
||||
log.getStartTxId(), log.getEndTxId(), milliTime);
|
||||
List<File> tmpFiles = dstStorage.getFiles(NameNodeDirType.EDITS,
|
||||
|
|
|
@ -437,11 +437,22 @@ public class ShortCircuitCache implements Closeable {
|
|||
void unref(ShortCircuitReplica replica) {
|
||||
lock.lock();
|
||||
try {
|
||||
// If the replica is stale, but we haven't purged it yet, let's do that.
|
||||
// It would be a shame to evict a non-stale replica so that we could put
|
||||
// a stale one into the cache.
|
||||
if ((!replica.purged) && replica.isStale()) {
|
||||
purge(replica);
|
||||
// If the replica is stale or unusable, but we haven't purged it yet,
|
||||
// let's do that. It would be a shame to evict a non-stale replica so
|
||||
// that we could put a stale or unusable one into the cache.
|
||||
if (!replica.purged) {
|
||||
String purgeReason = null;
|
||||
if (!replica.getDataStream().getChannel().isOpen()) {
|
||||
purgeReason = "purging replica because its data channel is closed.";
|
||||
} else if (!replica.getMetaStream().getChannel().isOpen()) {
|
||||
purgeReason = "purging replica because its meta channel is closed.";
|
||||
} else if (replica.isStale()) {
|
||||
purgeReason = "purging replica because it is stale.";
|
||||
}
|
||||
if (purgeReason != null) {
|
||||
LOG.debug(this + ": " + purgeReason);
|
||||
purge(replica);
|
||||
}
|
||||
}
|
||||
String addedString = "";
|
||||
boolean shouldTrimEvictionMaps = false;
|
||||
|
|
|
@ -1930,4 +1930,22 @@
|
|||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.client.slow.io.warning.threshold.ms</name>
|
||||
<value>30000</value>
|
||||
<description>The threshold in milliseconds at which we will log a slow
|
||||
io warning in a dfsclient. By default, this parameter is set to 30000
|
||||
milliseconds (30 seconds).
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>dfs.datanode.slow.io.warning.threshold.ms</name>
|
||||
<value>300</value>
|
||||
<description>The threshold in milliseconds at which we will log a slow
|
||||
io warning in a datanode. By default, this parameter is set to 300
|
||||
milliseconds.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Centralized Cache Management in HDFS
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=2|toDepth=4}
|
||||
|
||||
* {Overview}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Extended Attributes in HDFS
|
||||
|
||||
\[ {{{../../index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=2|toDepth=4}
|
||||
|
||||
* {Overview}
|
||||
|
|
|
@ -19,8 +19,6 @@
|
|||
|
||||
HDFS NFS Gateway
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=0}
|
||||
|
||||
* {Overview}
|
||||
|
@ -338,8 +336,21 @@ HDFS NFS Gateway
|
|||
The system administrator must ensure that the user on NFS client host has the same
|
||||
name and UID as that on the NFS gateway host. This is usually not a problem if
|
||||
the same user management system (e.g., LDAP/NIS) is used to create and deploy users on
|
||||
HDFS nodes and NFS client node. In case the user account is created manually in different hosts, one might need to
|
||||
HDFS nodes and NFS client node. In case the user account is created manually on different hosts, one might need to
|
||||
modify UID (e.g., do "usermod -u 123 myusername") on either NFS client or NFS gateway host
|
||||
in order to make it the same on both sides. More technical details of RPC AUTH_UNIX can be found
|
||||
in {{{http://tools.ietf.org/html/rfc1057}RPC specification}}.
|
||||
|
||||
Optionally, the system administrator can configure a custom static mapping
|
||||
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.
|
||||
The format of the static mapping file is similar to what is described in the
|
||||
exports(5) manual page, but roughly it is:
|
||||
|
||||
-------------------------
|
||||
# Mapping for clients accessing the NFS gateway
|
||||
uid 10 100 # Map the remote UID 10 the local UID 100
|
||||
gid 11 101 # Map the remote GID 11 to the local GID 101
|
||||
-------------------------
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
ViewFs Guide
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=0}
|
||||
|
||||
* {Introduction}
|
||||
|
|
|
@ -28,12 +28,15 @@ import static org.hamcrest.CoreMatchers.equalTo;
|
|||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.nio.channels.ClosedByInterruptException;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.Semaphore;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -409,4 +412,121 @@ public class TestBlockReaderFactory {
|
|||
getDomainSocketWatcher().isClosed());
|
||||
cluster.shutdown();
|
||||
}
|
||||
|
||||
/**
|
||||
* When an InterruptedException is sent to a thread calling
|
||||
* FileChannel#read, the FileChannel is immediately closed and the
|
||||
* thread gets an exception. This effectively means that we might have
|
||||
* someone asynchronously calling close() on the file descriptors we use
|
||||
* in BlockReaderLocal. So when unreferencing a ShortCircuitReplica in
|
||||
* ShortCircuitCache#unref, we should check if the FileChannel objects
|
||||
* are still open. If not, we should purge the replica to avoid giving
|
||||
* it out to any future readers.
|
||||
*
|
||||
* This is a regression test for HDFS-6227: Short circuit read failed
|
||||
* due to ClosedChannelException.
|
||||
*
|
||||
* Note that you may still get ClosedChannelException errors if two threads
|
||||
* are reading from the same replica and an InterruptedException is delivered
|
||||
* to one of them.
|
||||
*/
|
||||
@Test(timeout=120000)
|
||||
public void testPurgingClosedReplicas() throws Exception {
|
||||
BlockReaderTestUtil.enableBlockReaderFactoryTracing();
|
||||
final AtomicInteger replicasCreated = new AtomicInteger(0);
|
||||
final AtomicBoolean testFailed = new AtomicBoolean(false);
|
||||
DFSInputStream.tcpReadsDisabledForTesting = true;
|
||||
BlockReaderFactory.createShortCircuitReplicaInfoCallback =
|
||||
new ShortCircuitCache.ShortCircuitReplicaCreator() {
|
||||
@Override
|
||||
public ShortCircuitReplicaInfo createShortCircuitReplicaInfo() {
|
||||
replicasCreated.incrementAndGet();
|
||||
return null;
|
||||
}
|
||||
};
|
||||
TemporarySocketDirectory sockDir = new TemporarySocketDirectory();
|
||||
Configuration conf = createShortCircuitConf(
|
||||
"testPurgingClosedReplicas", sockDir);
|
||||
final MiniDFSCluster cluster =
|
||||
new MiniDFSCluster.Builder(conf).numDataNodes(1).build();
|
||||
cluster.waitActive();
|
||||
final DistributedFileSystem dfs = cluster.getFileSystem();
|
||||
final String TEST_FILE = "/test_file";
|
||||
final int TEST_FILE_LEN = 4095;
|
||||
final int SEED = 0xFADE0;
|
||||
final DistributedFileSystem fs =
|
||||
(DistributedFileSystem)FileSystem.get(cluster.getURI(0), conf);
|
||||
DFSTestUtil.createFile(fs, new Path(TEST_FILE), TEST_FILE_LEN,
|
||||
(short)1, SEED);
|
||||
|
||||
final Semaphore sem = new Semaphore(0);
|
||||
final List<LocatedBlock> locatedBlocks =
|
||||
cluster.getNameNode().getRpcServer().getBlockLocations(
|
||||
TEST_FILE, 0, TEST_FILE_LEN).getLocatedBlocks();
|
||||
final LocatedBlock lblock = locatedBlocks.get(0); // first block
|
||||
final byte[] buf = new byte[TEST_FILE_LEN];
|
||||
Runnable readerRunnable = new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
while (true) {
|
||||
BlockReader blockReader = null;
|
||||
try {
|
||||
blockReader = BlockReaderTestUtil.
|
||||
getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
|
||||
sem.release();
|
||||
try {
|
||||
blockReader.readAll(buf, 0, TEST_FILE_LEN);
|
||||
} finally {
|
||||
sem.acquireUninterruptibly();
|
||||
}
|
||||
} catch (ClosedByInterruptException e) {
|
||||
LOG.info("got the expected ClosedByInterruptException", e);
|
||||
sem.release();
|
||||
break;
|
||||
} finally {
|
||||
if (blockReader != null) blockReader.close();
|
||||
}
|
||||
LOG.info("read another " + TEST_FILE_LEN + " bytes.");
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
LOG.error("getBlockReader failure", t);
|
||||
testFailed.set(true);
|
||||
sem.release();
|
||||
}
|
||||
}
|
||||
};
|
||||
Thread thread = new Thread(readerRunnable);
|
||||
thread.start();
|
||||
|
||||
// While the thread is reading, send it interrupts.
|
||||
// These should trigger a ClosedChannelException.
|
||||
while (thread.isAlive()) {
|
||||
sem.acquireUninterruptibly();
|
||||
thread.interrupt();
|
||||
sem.release();
|
||||
}
|
||||
Assert.assertFalse(testFailed.get());
|
||||
|
||||
// We should be able to read from the file without
|
||||
// getting a ClosedChannelException.
|
||||
BlockReader blockReader = null;
|
||||
try {
|
||||
blockReader = BlockReaderTestUtil.
|
||||
getBlockReader(cluster, lblock, 0, TEST_FILE_LEN);
|
||||
blockReader.readFully(buf, 0, TEST_FILE_LEN);
|
||||
} finally {
|
||||
if (blockReader != null) blockReader.close();
|
||||
}
|
||||
byte expected[] = DFSTestUtil.
|
||||
calculateFileContentsFromSeed(SEED, TEST_FILE_LEN);
|
||||
Assert.assertTrue(Arrays.equals(buf, expected));
|
||||
|
||||
// Another ShortCircuitReplica object should have been created.
|
||||
Assert.assertEquals(2, replicasCreated.get());
|
||||
|
||||
dfs.close();
|
||||
cluster.shutdown();
|
||||
sockDir.close();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,75 @@
|
|||
/**
|
||||
* 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;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestGetFileChecksum {
|
||||
private static final int BLOCKSIZE = 1024;
|
||||
private static final short REPLICATION = 3;
|
||||
|
||||
private Configuration conf;
|
||||
private MiniDFSCluster cluster;
|
||||
private DistributedFileSystem dfs;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
conf = new Configuration();
|
||||
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
|
||||
cluster = new MiniDFSCluster.Builder(conf).numDataNodes(REPLICATION)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
dfs = cluster.getFileSystem();
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
if (cluster != null) {
|
||||
cluster.shutdown();
|
||||
}
|
||||
}
|
||||
|
||||
public void testGetFileChecksum(final Path foo, final int appendLength)
|
||||
throws Exception {
|
||||
final int appendRounds = 16;
|
||||
FileChecksum[] fc = new FileChecksum[appendRounds + 1];
|
||||
DFSTestUtil.createFile(dfs, foo, appendLength, REPLICATION, 0L);
|
||||
fc[0] = dfs.getFileChecksum(foo);
|
||||
for (int i = 0; i < appendRounds; i++) {
|
||||
DFSTestUtil.appendFile(dfs, foo, appendLength);
|
||||
fc[i + 1] = dfs.getFileChecksum(foo);
|
||||
}
|
||||
|
||||
for (int i = 0; i < appendRounds + 1; i++) {
|
||||
FileChecksum checksum = dfs.getFileChecksum(foo, appendLength * (i+1));
|
||||
Assert.assertTrue(checksum.equals(fc[i]));
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetFileChecksum() throws Exception {
|
||||
testGetFileChecksum(new Path("/foo"), BLOCKSIZE / 4);
|
||||
testGetFileChecksum(new Path("/bar"), BLOCKSIZE / 4 - 1);
|
||||
}
|
||||
}
|
|
@ -22,8 +22,12 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HA_NAMENODES_KEY_PREFIX;
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RPC_ADDRESS_KEY;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.BindException;
|
||||
import java.net.URI;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.DFSUtil;
|
||||
|
@ -37,14 +41,13 @@ public class MiniQJMHACluster {
|
|||
private MiniDFSCluster cluster;
|
||||
private MiniJournalCluster journalCluster;
|
||||
private final Configuration conf;
|
||||
private static final Log LOG = LogFactory.getLog(MiniQJMHACluster.class);
|
||||
|
||||
public static final String NAMESERVICE = "ns1";
|
||||
private static final String NN1 = "nn1";
|
||||
private static final String NN2 = "nn2";
|
||||
private static final int NN1_IPC_PORT = 10000;
|
||||
private static final int NN1_INFO_PORT = 10001;
|
||||
private static final int NN2_IPC_PORT = 10002;
|
||||
private static final int NN2_INFO_PORT = 10003;
|
||||
private static final Random RANDOM = new Random();
|
||||
private int basePort = 10000;
|
||||
|
||||
public static class Builder {
|
||||
private final Configuration conf;
|
||||
|
@ -69,51 +72,62 @@ public class MiniQJMHACluster {
|
|||
}
|
||||
}
|
||||
|
||||
public static MiniDFSNNTopology createDefaultTopology() {
|
||||
public static MiniDFSNNTopology createDefaultTopology(int basePort) {
|
||||
return new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf(NAMESERVICE).addNN(
|
||||
new MiniDFSNNTopology.NNConf("nn1").setIpcPort(NN1_IPC_PORT)
|
||||
.setHttpPort(NN1_INFO_PORT)).addNN(
|
||||
new MiniDFSNNTopology.NNConf("nn2").setIpcPort(NN2_IPC_PORT)
|
||||
.setHttpPort(NN2_INFO_PORT)));
|
||||
new MiniDFSNNTopology.NNConf("nn1").setIpcPort(basePort)
|
||||
.setHttpPort(basePort + 1)).addNN(
|
||||
new MiniDFSNNTopology.NNConf("nn2").setIpcPort(basePort + 2)
|
||||
.setHttpPort(basePort + 3)));
|
||||
}
|
||||
|
||||
|
||||
private MiniQJMHACluster(Builder builder) throws IOException {
|
||||
this.conf = builder.conf;
|
||||
// start 3 journal nodes
|
||||
journalCluster = new MiniJournalCluster.Builder(conf).format(true)
|
||||
.build();
|
||||
URI journalURI = journalCluster.getQuorumJournalURI(NAMESERVICE);
|
||||
|
||||
// start cluster with 2 NameNodes
|
||||
MiniDFSNNTopology topology = createDefaultTopology();
|
||||
|
||||
initHAConf(journalURI, builder.conf);
|
||||
|
||||
// First start up the NNs just to format the namespace. The MinIDFSCluster
|
||||
// has no way to just format the NameNodes without also starting them.
|
||||
cluster = builder.dfsBuilder.nnTopology(topology)
|
||||
.manageNameDfsSharedDirs(false).build();
|
||||
cluster.waitActive();
|
||||
cluster.shutdown();
|
||||
|
||||
// initialize the journal nodes
|
||||
Configuration confNN0 = cluster.getConfiguration(0);
|
||||
NameNode.initializeSharedEdits(confNN0, true);
|
||||
|
||||
cluster.getNameNodeInfos()[0].setStartOpt(builder.startOpt);
|
||||
cluster.getNameNodeInfos()[1].setStartOpt(builder.startOpt);
|
||||
|
||||
// restart the cluster
|
||||
cluster.restartNameNodes();
|
||||
int retryCount = 0;
|
||||
while (true) {
|
||||
try {
|
||||
basePort = 10000 + RANDOM.nextInt(1000) * 4;
|
||||
// start 3 journal nodes
|
||||
journalCluster = new MiniJournalCluster.Builder(conf).format(true)
|
||||
.build();
|
||||
URI journalURI = journalCluster.getQuorumJournalURI(NAMESERVICE);
|
||||
|
||||
// start cluster with 2 NameNodes
|
||||
MiniDFSNNTopology topology = createDefaultTopology(basePort);
|
||||
|
||||
initHAConf(journalURI, builder.conf);
|
||||
|
||||
// First start up the NNs just to format the namespace. The MinIDFSCluster
|
||||
// has no way to just format the NameNodes without also starting them.
|
||||
cluster = builder.dfsBuilder.nnTopology(topology)
|
||||
.manageNameDfsSharedDirs(false).build();
|
||||
cluster.waitActive();
|
||||
cluster.shutdown();
|
||||
|
||||
// initialize the journal nodes
|
||||
Configuration confNN0 = cluster.getConfiguration(0);
|
||||
NameNode.initializeSharedEdits(confNN0, true);
|
||||
|
||||
cluster.getNameNodeInfos()[0].setStartOpt(builder.startOpt);
|
||||
cluster.getNameNodeInfos()[1].setStartOpt(builder.startOpt);
|
||||
|
||||
// restart the cluster
|
||||
cluster.restartNameNodes();
|
||||
++retryCount;
|
||||
break;
|
||||
} catch (BindException e) {
|
||||
LOG.info("MiniQJMHACluster port conflicts, retried " +
|
||||
retryCount + " times");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private Configuration initHAConf(URI journalURI, Configuration conf) {
|
||||
conf.set(DFSConfigKeys.DFS_NAMENODE_SHARED_EDITS_DIR_KEY,
|
||||
journalURI.toString());
|
||||
|
||||
String address1 = "127.0.0.1:" + NN1_IPC_PORT;
|
||||
String address2 = "127.0.0.1:" + NN2_IPC_PORT;
|
||||
String address1 = "127.0.0.1:" + basePort;
|
||||
String address2 = "127.0.0.1:" + (basePort + 2);
|
||||
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
|
||||
NAMESERVICE, NN1), address1);
|
||||
conf.set(DFSUtil.addKeySuffixes(DFS_NAMENODE_RPC_ADDRESS_KEY,
|
||||
|
|
|
@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
|||
import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
|
||||
import org.apache.hadoop.security.TestDoAsEffectiveUser;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.junit.AfterClass;
|
||||
|
@ -88,7 +89,7 @@ public class TestDelegationTokenForProxyUser {
|
|||
builder.append("127.0.1.1,");
|
||||
builder.append(InetAddress.getLocalHost().getCanonicalHostName());
|
||||
LOG.info("Local Ip addresses: " + builder.toString());
|
||||
conf.setStrings(ProxyUsers.getProxySuperuserIpConfKey(superUserShortName),
|
||||
conf.setStrings(DefaultImpersonationProvider.getProxySuperuserIpConfKey(superUserShortName),
|
||||
builder.toString());
|
||||
}
|
||||
|
||||
|
@ -100,7 +101,7 @@ public class TestDelegationTokenForProxyUser {
|
|||
DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_MAX_LIFETIME_KEY, 10000);
|
||||
config.setLong(
|
||||
DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_RENEW_INTERVAL_KEY, 5000);
|
||||
config.setStrings(ProxyUsers.getProxySuperuserGroupConfKey(REAL_USER),
|
||||
config.setStrings(DefaultImpersonationProvider.getProxySuperuserGroupConfKey(REAL_USER),
|
||||
"group1");
|
||||
config.setBoolean(
|
||||
DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, true);
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.io.Text;
|
|||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
||||
import org.apache.hadoop.security.authorize.AuthorizationException;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyServers;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
|
@ -284,8 +285,8 @@ public class TestJspHelper {
|
|||
String user = "TheNurse";
|
||||
conf.set(DFSConfigKeys.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
|
||||
|
||||
conf.set(ProxyUsers.getProxySuperuserGroupConfKey(realUser), "*");
|
||||
conf.set(ProxyUsers.getProxySuperuserIpConfKey(realUser), "*");
|
||||
conf.set(DefaultImpersonationProvider.getProxySuperuserGroupConfKey(realUser), "*");
|
||||
conf.set(DefaultImpersonationProvider.getProxySuperuserIpConfKey(realUser), "*");
|
||||
ProxyUsers.refreshSuperUserGroupsConfiguration(conf);
|
||||
UserGroupInformation.setConfiguration(conf);
|
||||
UserGroupInformation ugi;
|
||||
|
|
|
@ -44,6 +44,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
|
|||
import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi;
|
||||
import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.Time;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
@ -215,6 +216,6 @@ public class TestDiskError {
|
|||
dataNode.checkDiskError();
|
||||
Thread.sleep(dataNode.checkDiskErrorInterval);
|
||||
long lastDiskErrorCheck = dataNode.getLastDiskErrorCheck();
|
||||
assertTrue("Disk Error check is not performed within " + dataNode.checkDiskErrorInterval + " ms", ((System.currentTimeMillis()-lastDiskErrorCheck) < (dataNode.checkDiskErrorInterval + slackTime)));
|
||||
assertTrue("Disk Error check is not performed within " + dataNode.checkDiskErrorInterval + " ms", ((Time.monotonicNow()-lastDiskErrorCheck) < (dataNode.checkDiskErrorInterval + slackTime)));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -39,8 +39,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
|
|||
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.apache.hadoop.security.authorize.ProxyServers;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
|
||||
|
|
|
@ -17,11 +17,16 @@
|
|||
*/
|
||||
package org.apache.hadoop.hdfs.server.namenode;
|
||||
|
||||
import java.net.BindException;
|
||||
import java.util.Random;
|
||||
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_PERIOD_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_CHECKPOINT_TXNS_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
@ -43,6 +48,9 @@ public class TestEditLogAutoroll {
|
|||
private NameNode nn0;
|
||||
private FileSystem fs;
|
||||
private FSEditLog editLog;
|
||||
private final Random random = new Random();
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestEditLog.class);
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
|
@ -54,24 +62,35 @@ public class TestEditLogAutoroll {
|
|||
conf.setFloat(DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD, 0.5f);
|
||||
conf.setInt(DFS_NAMENODE_EDIT_LOG_AUTOROLL_CHECK_INTERVAL_MS, 100);
|
||||
|
||||
MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10061))
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10062)));
|
||||
int retryCount = 0;
|
||||
while (true) {
|
||||
try {
|
||||
int basePort = 10060 + random.nextInt(100) * 2;
|
||||
MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(topology)
|
||||
.numDataNodes(0)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(topology)
|
||||
.numDataNodes(0)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
||||
nn0 = cluster.getNameNode(0);
|
||||
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
nn0 = cluster.getNameNode(0);
|
||||
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
|
||||
cluster.transitionToActive(0);
|
||||
cluster.transitionToActive(0);
|
||||
|
||||
fs = cluster.getFileSystem(0);
|
||||
editLog = nn0.getNamesystem().getEditLog();
|
||||
fs = cluster.getFileSystem(0);
|
||||
editLog = nn0.getNamesystem().getEditLog();
|
||||
++retryCount;
|
||||
break;
|
||||
} catch (BindException e) {
|
||||
LOG.info("Set up MiniDFSCluster failed due to port conflicts, retry "
|
||||
+ retryCount + " times");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@After
|
||||
|
|
|
@ -104,7 +104,7 @@ public class TestFailureToReadEdits {
|
|||
HAUtil.setAllowStandbyReads(conf, true);
|
||||
|
||||
if (clusterType == TestType.SHARED_DIR_HA) {
|
||||
MiniDFSNNTopology topology = MiniQJMHACluster.createDefaultTopology();
|
||||
MiniDFSNNTopology topology = MiniQJMHACluster.createDefaultTopology(10000);
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(topology)
|
||||
.numDataNodes(0)
|
||||
|
|
|
@ -25,12 +25,14 @@ import static org.junit.Assert.fail;
|
|||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.net.BindException;
|
||||
import java.lang.management.ManagementFactory;
|
||||
import java.lang.management.ThreadInfo;
|
||||
import java.lang.management.ThreadMXBean;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
|
@ -73,6 +75,7 @@ public class TestStandbyCheckpoints {
|
|||
protected MiniDFSCluster cluster;
|
||||
protected NameNode nn0, nn1;
|
||||
protected FileSystem fs;
|
||||
private final Random random = new Random();
|
||||
protected File tmpOivImgDir;
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(TestStandbyCheckpoints.class);
|
||||
|
@ -87,22 +90,33 @@ public class TestStandbyCheckpoints {
|
|||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_CHECKPOINTS_RETAINED_KEY, 1);
|
||||
conf.setInt(DFSConfigKeys.DFS_NAMENODE_NUM_EXTRA_EDITS_RETAINED_KEY, 0);
|
||||
|
||||
MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(10061))
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(10062)));
|
||||
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(topology)
|
||||
.numDataNodes(0)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
||||
nn0 = cluster.getNameNode(0);
|
||||
nn1 = cluster.getNameNode(1);
|
||||
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
int retryCount = 0;
|
||||
while (true) {
|
||||
try {
|
||||
int basePort = 10060 + random.nextInt(100) * 2;
|
||||
MiniDFSNNTopology topology = new MiniDFSNNTopology()
|
||||
.addNameservice(new MiniDFSNNTopology.NSConf("ns1")
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn1").setHttpPort(basePort))
|
||||
.addNN(new MiniDFSNNTopology.NNConf("nn2").setHttpPort(basePort + 1)));
|
||||
|
||||
cluster.transitionToActive(0);
|
||||
cluster = new MiniDFSCluster.Builder(conf)
|
||||
.nnTopology(topology)
|
||||
.numDataNodes(0)
|
||||
.build();
|
||||
cluster.waitActive();
|
||||
|
||||
nn0 = cluster.getNameNode(0);
|
||||
nn1 = cluster.getNameNode(1);
|
||||
fs = HATestUtil.configureFailoverFs(cluster, conf);
|
||||
|
||||
cluster.transitionToActive(0);
|
||||
++retryCount;
|
||||
break;
|
||||
} catch (BindException e) {
|
||||
LOG.info("Set up MiniDFSCluster failed due to port conflicts, retry "
|
||||
+ retryCount + " times");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
protected Configuration setupCommonConfig() {
|
||||
|
|
|
@ -41,6 +41,7 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.hdfs.MiniDFSCluster;
|
||||
import org.apache.hadoop.hdfs.tools.DFSAdmin;
|
||||
import org.apache.hadoop.security.authorize.AuthorizationException;
|
||||
import org.apache.hadoop.security.authorize.DefaultImpersonationProvider;
|
||||
import org.apache.hadoop.security.authorize.ProxyUsers;
|
||||
import org.junit.After;
|
||||
import org.junit.Before;
|
||||
|
@ -150,8 +151,8 @@ public class TestRefreshUserMappings {
|
|||
final String [] GROUP_NAMES2 = new String [] {"gr3" , "gr4"};
|
||||
|
||||
//keys in conf
|
||||
String userKeyGroups = ProxyUsers.getProxySuperuserGroupConfKey(SUPER_USER);
|
||||
String userKeyHosts = ProxyUsers.getProxySuperuserIpConfKey (SUPER_USER);
|
||||
String userKeyGroups = DefaultImpersonationProvider.getProxySuperuserGroupConfKey(SUPER_USER);
|
||||
String userKeyHosts = DefaultImpersonationProvider.getProxySuperuserIpConfKey (SUPER_USER);
|
||||
|
||||
config.set(userKeyGroups, "gr3,gr4,gr5"); // superuser can proxy for this group
|
||||
config.set(userKeyHosts,"127.0.0.1");
|
||||
|
|
|
@ -202,6 +202,8 @@ Release 2.5.0 - UNRELEASED
|
|||
|
||||
MAPREDUCE-5809. Enhance distcp to support preserving HDFS ACLs. (cnauroth)
|
||||
|
||||
MAPREDUCE-5899. Support incremental data copy in DistCp. (jing9)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -239,6 +241,12 @@ Release 2.5.0 - UNRELEASED
|
|||
MAPREDUCE-5309. 2.0.4 JobHistoryParser can't parse certain failed job
|
||||
history files generated by 2.0.3 history server (Rushabh S Shah via jlowe)
|
||||
|
||||
MAPREDUCE-5862. Line records longer than 2x split size aren't handled
|
||||
correctly (bc Wong via jlowe)
|
||||
|
||||
MAPREDUCE-5895. Close streams properly to avoid leakage in TaskLog.
|
||||
(Kousuke Saruta via devaraj)
|
||||
|
||||
Release 2.4.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -85,6 +85,15 @@
|
|||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.rat</groupId>
|
||||
<artifactId>apache-rat-plugin</artifactId>
|
||||
<configuration>
|
||||
<excludes>
|
||||
<exclude>src/test/resources/recordSpanningMultipleSplits.txt</exclude>
|
||||
</excludes>
|
||||
</configuration>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
||||
|
|
|
@ -184,7 +184,7 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
|
|||
private int maxBytesToConsume(long pos) {
|
||||
return isCompressedInput()
|
||||
? Integer.MAX_VALUE
|
||||
: (int) Math.min(Integer.MAX_VALUE, end - pos);
|
||||
: (int) Math.max(Math.min(Integer.MAX_VALUE, end - pos), maxLineLength);
|
||||
}
|
||||
|
||||
private long getFilePosition() throws IOException {
|
||||
|
@ -206,8 +206,7 @@ public class LineRecordReader implements RecordReader<LongWritable, Text> {
|
|||
while (getFilePosition() <= end || in.needAdditionalRecordAfterSplit()) {
|
||||
key.set(pos);
|
||||
|
||||
int newSize = in.readLine(value, maxLineLength,
|
||||
Math.max(maxBytesToConsume(pos), maxLineLength));
|
||||
int newSize = in.readLine(value, maxLineLength, maxBytesToConsume(pos));
|
||||
if (newSize == 0) {
|
||||
return false;
|
||||
}
|
||||
|
|
|
@ -199,16 +199,18 @@ public class TaskLog {
|
|||
// file first and then rename.
|
||||
File tmpIndexFile = getTmpIndexFile(currentTaskid, isCleanup);
|
||||
|
||||
BufferedOutputStream bos =
|
||||
new BufferedOutputStream(
|
||||
SecureIOUtils.createForWrite(tmpIndexFile, 0644));
|
||||
DataOutputStream dos = new DataOutputStream(bos);
|
||||
//the format of the index file is
|
||||
//LOG_DIR: <the dir where the task logs are really stored>
|
||||
//STDOUT: <start-offset in the stdout file> <length>
|
||||
//STDERR: <start-offset in the stderr file> <length>
|
||||
//SYSLOG: <start-offset in the syslog file> <length>
|
||||
BufferedOutputStream bos = null;
|
||||
DataOutputStream dos = null;
|
||||
try{
|
||||
bos = new BufferedOutputStream(
|
||||
SecureIOUtils.createForWrite(tmpIndexFile, 0644));
|
||||
dos = new DataOutputStream(bos);
|
||||
//the format of the index file is
|
||||
//LOG_DIR: <the dir where the task logs are really stored>
|
||||
//STDOUT: <start-offset in the stdout file> <length>
|
||||
//STDERR: <start-offset in the stderr file> <length>
|
||||
//SYSLOG: <start-offset in the syslog file> <length>
|
||||
|
||||
dos.writeBytes(LogFileDetail.LOCATION + logLocation + "\n"
|
||||
+ LogName.STDOUT.toString() + ":");
|
||||
dos.writeBytes(Long.toString(prevOutLength) + " ");
|
||||
|
@ -225,8 +227,10 @@ public class TaskLog {
|
|||
+ "\n");
|
||||
dos.close();
|
||||
dos = null;
|
||||
bos.close();
|
||||
bos = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(LOG, dos);
|
||||
IOUtils.cleanup(LOG, dos, bos);
|
||||
}
|
||||
|
||||
File indexFile = getIndexFile(currentTaskid, isCleanup);
|
||||
|
|
|
@ -121,7 +121,7 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
|
|||
private int maxBytesToConsume(long pos) {
|
||||
return isCompressedInput
|
||||
? Integer.MAX_VALUE
|
||||
: (int) Math.min(Integer.MAX_VALUE, end - pos);
|
||||
: (int) Math.max(Math.min(Integer.MAX_VALUE, end - pos), maxLineLength);
|
||||
}
|
||||
|
||||
private long getFilePosition() throws IOException {
|
||||
|
@ -146,8 +146,7 @@ public class LineRecordReader extends RecordReader<LongWritable, Text> {
|
|||
// We always read one extra line, which lies outside the upper
|
||||
// split limit i.e. (end - 1)
|
||||
while (getFilePosition() <= end || in.needAdditionalRecordAfterSplit()) {
|
||||
newSize = in.readLine(value, maxLineLength,
|
||||
Math.max(maxBytesToConsume(pos), maxLineLength));
|
||||
newSize = in.readLine(value, maxLineLength, maxBytesToConsume(pos));
|
||||
pos += newSize;
|
||||
if (newSize < maxLineLength) {
|
||||
break;
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop MapReduce Next Generation - Distributed Cache Deploy
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* Introduction
|
||||
|
||||
The MapReduce application framework has rudimentary support for deploying a
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop MapReduce Next Generation - Encrypted Shuffle
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* {Introduction}
|
||||
|
||||
The Encrypted Shuffle capability allows encryption of the MapReduce shuffle
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Apache Hadoop MapReduce - Migrating from Apache Hadoop 1.x to Apache Hadoop 2.x
|
||||
|
||||
\[ {{{../../hadoop-yarn/hadoop-yarn-site/index.html}Go Back}} \]
|
||||
|
||||
* {Introduction}
|
||||
|
||||
This document provides information for users to migrate their Apache Hadoop
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Hadoop MapReduce Next Generation - Pluggable Shuffle and Pluggable Sort
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
* Introduction
|
||||
|
||||
The pluggable shuffle and pluggable sort capabilities allow replacing the
|
||||
|
|
|
@ -23,9 +23,12 @@ import static org.junit.Assert.assertNotNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.io.LongWritable;
|
||||
|
@ -97,4 +100,92 @@ public class TestLineRecordReader {
|
|||
// character is a linefeed
|
||||
testSplitRecords("blockEndingInCRThenLF.txt.bz2", 136498);
|
||||
}
|
||||
|
||||
// Use the LineRecordReader to read records from the file
|
||||
public ArrayList<String> readRecords(URL testFileUrl, int splitSize)
|
||||
throws IOException {
|
||||
|
||||
// Set up context
|
||||
File testFile = new File(testFileUrl.getFile());
|
||||
long testFileSize = testFile.length();
|
||||
Path testFilePath = new Path(testFile.getAbsolutePath());
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt("io.file.buffer.size", 1);
|
||||
|
||||
// Gather the records returned by the record reader
|
||||
ArrayList<String> records = new ArrayList<String>();
|
||||
|
||||
long offset = 0;
|
||||
LongWritable key = new LongWritable();
|
||||
Text value = new Text();
|
||||
while (offset < testFileSize) {
|
||||
FileSplit split =
|
||||
new FileSplit(testFilePath, offset, splitSize, (String[]) null);
|
||||
LineRecordReader reader = new LineRecordReader(conf, split);
|
||||
|
||||
while (reader.next(key, value)) {
|
||||
records.add(value.toString());
|
||||
}
|
||||
offset += splitSize;
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
// Gather the records by just splitting on new lines
|
||||
public String[] readRecordsDirectly(URL testFileUrl, boolean bzip)
|
||||
throws IOException {
|
||||
int MAX_DATA_SIZE = 1024 * 1024;
|
||||
byte[] data = new byte[MAX_DATA_SIZE];
|
||||
FileInputStream fis = new FileInputStream(testFileUrl.getFile());
|
||||
int count;
|
||||
if (bzip) {
|
||||
BZip2CompressorInputStream bzIn = new BZip2CompressorInputStream(fis);
|
||||
count = bzIn.read(data);
|
||||
bzIn.close();
|
||||
} else {
|
||||
count = fis.read(data);
|
||||
}
|
||||
fis.close();
|
||||
assertTrue("Test file data too big for buffer", count < data.length);
|
||||
return new String(data, 0, count, "UTF-8").split("\n");
|
||||
}
|
||||
|
||||
public void checkRecordSpanningMultipleSplits(String testFile,
|
||||
int splitSize,
|
||||
boolean bzip)
|
||||
throws IOException {
|
||||
URL testFileUrl = getClass().getClassLoader().getResource(testFile);
|
||||
ArrayList<String> records = readRecords(testFileUrl, splitSize);
|
||||
String[] actuals = readRecordsDirectly(testFileUrl, bzip);
|
||||
|
||||
assertEquals("Wrong number of records", actuals.length, records.size());
|
||||
|
||||
boolean hasLargeRecord = false;
|
||||
for (int i = 0; i < actuals.length; ++i) {
|
||||
assertEquals(actuals[i], records.get(i));
|
||||
if (actuals[i].length() > 2 * splitSize) {
|
||||
hasLargeRecord = true;
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue("Invalid test data. Doesn't have a large enough record",
|
||||
hasLargeRecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordSpanningMultipleSplits()
|
||||
throws IOException {
|
||||
checkRecordSpanningMultipleSplits("recordSpanningMultipleSplits.txt",
|
||||
10, false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordSpanningMultipleSplitsCompressed()
|
||||
throws IOException {
|
||||
// The file is generated with bz2 block size of 100k. The split size
|
||||
// needs to be larger than that for the CompressedSplitLineReader to
|
||||
// work.
|
||||
checkRecordSpanningMultipleSplits("recordSpanningMultipleSplits.txt.bz2",
|
||||
200 * 1000, true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,9 +23,12 @@ import static org.junit.Assert.assertNotNull;
|
|||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
|
||||
import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.TaskAttemptContext;
|
||||
|
@ -101,4 +104,93 @@ public class TestLineRecordReader {
|
|||
// character is a linefeed
|
||||
testSplitRecords("blockEndingInCRThenLF.txt.bz2", 136498);
|
||||
}
|
||||
|
||||
// Use the LineRecordReader to read records from the file
|
||||
public ArrayList<String> readRecords(URL testFileUrl, int splitSize)
|
||||
throws IOException {
|
||||
|
||||
// Set up context
|
||||
File testFile = new File(testFileUrl.getFile());
|
||||
long testFileSize = testFile.length();
|
||||
Path testFilePath = new Path(testFile.getAbsolutePath());
|
||||
Configuration conf = new Configuration();
|
||||
conf.setInt("io.file.buffer.size", 1);
|
||||
TaskAttemptContext context = new TaskAttemptContextImpl(conf, new TaskAttemptID());
|
||||
|
||||
// Gather the records returned by the record reader
|
||||
ArrayList<String> records = new ArrayList<String>();
|
||||
|
||||
long offset = 0;
|
||||
while (offset < testFileSize) {
|
||||
FileSplit split = new FileSplit(testFilePath, offset, splitSize, null);
|
||||
LineRecordReader reader = new LineRecordReader();
|
||||
reader.initialize(split, context);
|
||||
|
||||
while (reader.nextKeyValue()) {
|
||||
records.add(reader.getCurrentValue().toString());
|
||||
}
|
||||
offset += splitSize;
|
||||
}
|
||||
return records;
|
||||
}
|
||||
|
||||
// Gather the records by just splitting on new lines
|
||||
public String[] readRecordsDirectly(URL testFileUrl, boolean bzip)
|
||||
throws IOException {
|
||||
int MAX_DATA_SIZE = 1024 * 1024;
|
||||
byte[] data = new byte[MAX_DATA_SIZE];
|
||||
FileInputStream fis = new FileInputStream(testFileUrl.getFile());
|
||||
int count;
|
||||
if (bzip) {
|
||||
BZip2CompressorInputStream bzIn = new BZip2CompressorInputStream(fis);
|
||||
count = bzIn.read(data);
|
||||
bzIn.close();
|
||||
} else {
|
||||
count = fis.read(data);
|
||||
}
|
||||
fis.close();
|
||||
assertTrue("Test file data too big for buffer", count < data.length);
|
||||
return new String(data, 0, count, "UTF-8").split("\n");
|
||||
}
|
||||
|
||||
public void checkRecordSpanningMultipleSplits(String testFile,
|
||||
int splitSize,
|
||||
boolean bzip)
|
||||
throws IOException {
|
||||
URL testFileUrl = getClass().getClassLoader().getResource(testFile);
|
||||
ArrayList<String> records = readRecords(testFileUrl, splitSize);
|
||||
String[] actuals = readRecordsDirectly(testFileUrl, bzip);
|
||||
|
||||
assertEquals("Wrong number of records", actuals.length, records.size());
|
||||
|
||||
boolean hasLargeRecord = false;
|
||||
for (int i = 0; i < actuals.length; ++i) {
|
||||
assertEquals(actuals[i], records.get(i));
|
||||
if (actuals[i].length() > 2 * splitSize) {
|
||||
hasLargeRecord = true;
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue("Invalid test data. Doesn't have a large enough record",
|
||||
hasLargeRecord);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordSpanningMultipleSplits()
|
||||
throws IOException {
|
||||
checkRecordSpanningMultipleSplits("recordSpanningMultipleSplits.txt",
|
||||
10,
|
||||
false);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRecordSpanningMultipleSplitsCompressed()
|
||||
throws IOException {
|
||||
// The file is generated with bz2 block size of 100k. The split size
|
||||
// needs to be larger than that for the CompressedSplitLineReader to
|
||||
// work.
|
||||
checkRecordSpanningMultipleSplits("recordSpanningMultipleSplits.txt.bz2",
|
||||
200 * 1000,
|
||||
true);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,4 @@
|
|||
Use with small split size,
|
||||
like 32.
|
||||
And then we give it a really really long line, which will surely span multiple splits,
|
||||
to see how it handles.
|
Binary file not shown.
|
@ -50,6 +50,7 @@ public class DistCpConstants {
|
|||
public static final String CONF_LABEL_COPY_STRATEGY = "distcp.copy.strategy";
|
||||
public static final String CONF_LABEL_SKIP_CRC = "distcp.skip.crc";
|
||||
public static final String CONF_LABEL_OVERWRITE = "distcp.copy.overwrite";
|
||||
public static final String CONF_LABEL_APPEND = "distcp.copy.append";
|
||||
public static final String CONF_LABEL_BANDWIDTH_MB = "distcp.map.bandwidth.mb";
|
||||
|
||||
public static final String CONF_LABEL_MAX_CHUNKS_TOLERABLE =
|
||||
|
|
|
@ -138,6 +138,10 @@ public enum DistCpOptionSwitch {
|
|||
new Option("overwrite", false, "Choose to overwrite target files " +
|
||||
"unconditionally, even if they exist.")),
|
||||
|
||||
APPEND(DistCpConstants.CONF_LABEL_APPEND,
|
||||
new Option("append", false,
|
||||
"Reuse existing data in target files and append new data to them if possible")),
|
||||
|
||||
/**
|
||||
* Should DisctpExecution be blocking
|
||||
*/
|
||||
|
|
|
@ -39,6 +39,7 @@ public class DistCpOptions {
|
|||
private boolean deleteMissing = false;
|
||||
private boolean ignoreFailures = false;
|
||||
private boolean overwrite = false;
|
||||
private boolean append = false;
|
||||
private boolean skipCRC = false;
|
||||
private boolean blocking = true;
|
||||
|
||||
|
@ -244,6 +245,22 @@ public class DistCpOptions {
|
|||
this.overwrite = overwrite;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether we can append new data to target files
|
||||
*/
|
||||
public boolean shouldAppend() {
|
||||
return append;
|
||||
}
|
||||
|
||||
/**
|
||||
* Set if we want to append new data to target files. This is valid only with
|
||||
* update option and CRC is not skipped.
|
||||
*/
|
||||
public void setAppend(boolean append) {
|
||||
validate(DistCpOptionSwitch.APPEND, append);
|
||||
this.append = append;
|
||||
}
|
||||
|
||||
/**
|
||||
* Should CRC/checksum check be skipped while checking files are identical
|
||||
*
|
||||
|
@ -472,6 +489,7 @@ public class DistCpOptions {
|
|||
value : this.atomicCommit);
|
||||
boolean skipCRC = (option == DistCpOptionSwitch.SKIP_CRC ?
|
||||
value : this.skipCRC);
|
||||
boolean append = (option == DistCpOptionSwitch.APPEND ? value : this.append);
|
||||
|
||||
if (syncFolder && atomicCommit) {
|
||||
throw new IllegalArgumentException("Atomic commit can't be used with " +
|
||||
|
@ -492,6 +510,14 @@ public class DistCpOptions {
|
|||
throw new IllegalArgumentException("Skip CRC is valid only with update options");
|
||||
}
|
||||
|
||||
if (!syncFolder && append) {
|
||||
throw new IllegalArgumentException(
|
||||
"Append is valid only with update options");
|
||||
}
|
||||
if (skipCRC && append) {
|
||||
throw new IllegalArgumentException(
|
||||
"Append is disallowed when skipping CRC");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -510,6 +536,8 @@ public class DistCpOptions {
|
|||
String.valueOf(deleteMissing));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.OVERWRITE,
|
||||
String.valueOf(overwrite));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.APPEND,
|
||||
String.valueOf(append));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.SKIP_CRC,
|
||||
String.valueOf(skipCRC));
|
||||
DistCpOptionSwitch.addToConf(conf, DistCpOptionSwitch.BANDWIDTH,
|
||||
|
|
|
@ -140,6 +140,10 @@ public class OptionsParser {
|
|||
option.setOverwrite(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.APPEND.getSwitch())) {
|
||||
option.setAppend(true);
|
||||
}
|
||||
|
||||
if (command.hasOption(DistCpOptionSwitch.DELETE_MISSING.getSwitch())) {
|
||||
option.setDeleteMissing(true);
|
||||
}
|
||||
|
|
|
@ -18,13 +18,20 @@
|
|||
|
||||
package org.apache.hadoop.tools.mapred;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.util.Arrays;
|
||||
import java.util.EnumSet;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.AclEntry;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.mapreduce.JobContext;
|
||||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
|
@ -36,11 +43,6 @@ import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
|||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
|
||||
import java.io.*;
|
||||
import java.util.EnumSet;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Mapper class that executes the DistCp copy operation.
|
||||
* Implements the o.a.h.mapreduce.Mapper<> interface.
|
||||
|
@ -62,6 +64,15 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
|
|||
BYTESSKIPPED, // Number of bytes that were skipped from copy.
|
||||
}
|
||||
|
||||
/**
|
||||
* Indicate the action for each file
|
||||
*/
|
||||
static enum FileAction {
|
||||
SKIP, // Skip copying the file since it's already in the target FS
|
||||
APPEND, // Only need to append new data to the file in the target FS
|
||||
OVERWRITE, // Overwrite the whole file
|
||||
}
|
||||
|
||||
private static Log LOG = LogFactory.getLog(CopyMapper.class);
|
||||
|
||||
private Configuration conf;
|
||||
|
@ -70,6 +81,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
|
|||
private boolean ignoreFailures = false;
|
||||
private boolean skipCrc = false;
|
||||
private boolean overWrite = false;
|
||||
private boolean append = false;
|
||||
private EnumSet<FileAttribute> preserve = EnumSet.noneOf(FileAttribute.class);
|
||||
|
||||
private FileSystem targetFS = null;
|
||||
|
@ -90,6 +102,7 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
|
|||
ignoreFailures = conf.getBoolean(DistCpOptionSwitch.IGNORE_FAILURES.getConfigLabel(), false);
|
||||
skipCrc = conf.getBoolean(DistCpOptionSwitch.SKIP_CRC.getConfigLabel(), false);
|
||||
overWrite = conf.getBoolean(DistCpOptionSwitch.OVERWRITE.getConfigLabel(), false);
|
||||
append = conf.getBoolean(DistCpOptionSwitch.APPEND.getConfigLabel(), false);
|
||||
preserve = DistCpUtils.unpackAttributes(conf.get(DistCpOptionSwitch.
|
||||
PRESERVE_STATUS.getConfigLabel()));
|
||||
|
||||
|
@ -224,20 +237,19 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
|
|||
return;
|
||||
}
|
||||
|
||||
if (skipFile(sourceFS, sourceCurrStatus, target)) {
|
||||
FileAction action = checkUpdate(sourceFS, sourceCurrStatus, target);
|
||||
if (action == FileAction.SKIP) {
|
||||
LOG.info("Skipping copy of " + sourceCurrStatus.getPath()
|
||||
+ " to " + target);
|
||||
updateSkipCounters(context, sourceCurrStatus);
|
||||
context.write(null, new Text("SKIP: " + sourceCurrStatus.getPath()));
|
||||
}
|
||||
else {
|
||||
} else {
|
||||
copyFileWithRetry(description, sourceCurrStatus, target, context,
|
||||
fileAttributes);
|
||||
action, fileAttributes);
|
||||
}
|
||||
|
||||
DistCpUtils.preserve(target.getFileSystem(conf), target,
|
||||
sourceCurrStatus, fileAttributes);
|
||||
|
||||
} catch (IOException exception) {
|
||||
handleFailures(exception, sourceFileStatus, target, context);
|
||||
}
|
||||
|
@ -254,14 +266,14 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
|
|||
return DistCpUtils.unpackAttributes(attributeString);
|
||||
}
|
||||
|
||||
private void copyFileWithRetry(String description, FileStatus sourceFileStatus,
|
||||
Path target, Context context,
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes) throws IOException {
|
||||
|
||||
private void copyFileWithRetry(String description,
|
||||
FileStatus sourceFileStatus, Path target, Context context,
|
||||
FileAction action, EnumSet<DistCpOptions.FileAttribute> fileAttributes)
|
||||
throws IOException {
|
||||
long bytesCopied;
|
||||
try {
|
||||
bytesCopied = (Long)new RetriableFileCopyCommand(skipCrc, description)
|
||||
.execute(sourceFileStatus, target, context, fileAttributes);
|
||||
bytesCopied = (Long) new RetriableFileCopyCommand(skipCrc, description,
|
||||
action).execute(sourceFileStatus, target, context, fileAttributes);
|
||||
} catch (Exception e) {
|
||||
context.setStatus("Copy Failure: " + sourceFileStatus.getPath());
|
||||
throw new IOException("File copy failed: " + sourceFileStatus.getPath() +
|
||||
|
@ -311,25 +323,48 @@ public class CopyMapper extends Mapper<Text, CopyListingFileStatus, Text, Text>
|
|||
context.getCounter(counter).increment(value);
|
||||
}
|
||||
|
||||
private boolean skipFile(FileSystem sourceFS, FileStatus source, Path target)
|
||||
throws IOException {
|
||||
return targetFS.exists(target)
|
||||
&& !overWrite
|
||||
&& !mustUpdate(sourceFS, source, target);
|
||||
private FileAction checkUpdate(FileSystem sourceFS, FileStatus source,
|
||||
Path target) throws IOException {
|
||||
final FileStatus targetFileStatus;
|
||||
try {
|
||||
targetFileStatus = targetFS.getFileStatus(target);
|
||||
} catch (FileNotFoundException e) {
|
||||
return FileAction.OVERWRITE;
|
||||
}
|
||||
if (targetFileStatus != null && !overWrite) {
|
||||
if (canSkip(sourceFS, source, targetFileStatus)) {
|
||||
return FileAction.SKIP;
|
||||
} else if (append) {
|
||||
long targetLen = targetFileStatus.getLen();
|
||||
if (targetLen < source.getLen()) {
|
||||
FileChecksum sourceChecksum = sourceFS.getFileChecksum(
|
||||
source.getPath(), targetLen);
|
||||
if (sourceChecksum != null
|
||||
&& sourceChecksum.equals(targetFS.getFileChecksum(target))) {
|
||||
// We require that the checksum is not null. Thus currently only
|
||||
// DistributedFileSystem is supported
|
||||
return FileAction.APPEND;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return FileAction.OVERWRITE;
|
||||
}
|
||||
|
||||
private boolean mustUpdate(FileSystem sourceFS, FileStatus source, Path target)
|
||||
throws IOException {
|
||||
final FileStatus targetFileStatus = targetFS.getFileStatus(target);
|
||||
|
||||
return syncFolders
|
||||
&& (
|
||||
targetFileStatus.getLen() != source.getLen()
|
||||
|| (!skipCrc &&
|
||||
!DistCpUtils.checksumsAreEqual(sourceFS,
|
||||
source.getPath(), null, targetFS, target))
|
||||
|| (source.getBlockSize() != targetFileStatus.getBlockSize() &&
|
||||
preserve.contains(FileAttribute.BLOCKSIZE))
|
||||
);
|
||||
private boolean canSkip(FileSystem sourceFS, FileStatus source,
|
||||
FileStatus target) throws IOException {
|
||||
if (!syncFolders) {
|
||||
return true;
|
||||
}
|
||||
boolean sameLength = target.getLen() == source.getLen();
|
||||
boolean sameBlockSize = source.getBlockSize() == target.getBlockSize()
|
||||
|| !preserve.contains(FileAttribute.BLOCKSIZE);
|
||||
if (sameLength && sameBlockSize) {
|
||||
return skipCrc ||
|
||||
DistCpUtils.checksumsAreEqual(sourceFS, source.getPath(), null,
|
||||
targetFS, target.getPath());
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -18,10 +18,8 @@
|
|||
|
||||
package org.apache.hadoop.tools.mapred;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.EnumSet;
|
||||
|
||||
|
@ -29,6 +27,8 @@ import org.apache.commons.logging.Log;
|
|||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileChecksum;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
@ -39,6 +39,7 @@ import org.apache.hadoop.io.IOUtils;
|
|||
import org.apache.hadoop.mapreduce.Mapper;
|
||||
import org.apache.hadoop.tools.DistCpConstants;
|
||||
import org.apache.hadoop.tools.DistCpOptions.FileAttribute;
|
||||
import org.apache.hadoop.tools.mapred.CopyMapper.FileAction;
|
||||
import org.apache.hadoop.tools.util.DistCpUtils;
|
||||
import org.apache.hadoop.tools.util.RetriableCommand;
|
||||
import org.apache.hadoop.tools.util.ThrottledInputStream;
|
||||
|
@ -54,13 +55,15 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
private static Log LOG = LogFactory.getLog(RetriableFileCopyCommand.class);
|
||||
private static int BUFFER_SIZE = 8 * 1024;
|
||||
private boolean skipCrc = false;
|
||||
private FileAction action;
|
||||
|
||||
/**
|
||||
* Constructor, taking a description of the action.
|
||||
* @param description Verbose description of the copy operation.
|
||||
*/
|
||||
public RetriableFileCopyCommand(String description) {
|
||||
public RetriableFileCopyCommand(String description, FileAction action) {
|
||||
super(description);
|
||||
this.action = action;
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -68,9 +71,11 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
*
|
||||
* @param skipCrc Whether to skip the crc check.
|
||||
* @param description A verbose description of the copy operation.
|
||||
* @param action We should overwrite the target file or append new data to it.
|
||||
*/
|
||||
public RetriableFileCopyCommand(boolean skipCrc, String description) {
|
||||
this(description);
|
||||
public RetriableFileCopyCommand(boolean skipCrc, String description,
|
||||
FileAction action) {
|
||||
this(description, action);
|
||||
this.skipCrc = skipCrc;
|
||||
}
|
||||
|
||||
|
@ -96,18 +101,17 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
}
|
||||
|
||||
private long doCopy(FileStatus sourceFileStatus, Path target,
|
||||
Mapper.Context context,
|
||||
EnumSet<FileAttribute> fileAttributes)
|
||||
throws IOException {
|
||||
|
||||
Path tmpTargetPath = getTmpFile(target, context);
|
||||
Mapper.Context context, EnumSet<FileAttribute> fileAttributes)
|
||||
throws IOException {
|
||||
final boolean toAppend = action == FileAction.APPEND;
|
||||
Path targetPath = toAppend ? target : getTmpFile(target, context);
|
||||
final Configuration configuration = context.getConfiguration();
|
||||
FileSystem targetFS = target.getFileSystem(configuration);
|
||||
|
||||
try {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Copying " + sourceFileStatus.getPath() + " to " + target);
|
||||
LOG.debug("Tmp-file path: " + tmpTargetPath);
|
||||
LOG.debug("Target file path: " + targetPath);
|
||||
}
|
||||
final Path sourcePath = sourceFileStatus.getPath();
|
||||
final FileSystem sourceFS = sourcePath.getFileSystem(configuration);
|
||||
|
@ -115,22 +119,31 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
.contains(FileAttribute.CHECKSUMTYPE) ? sourceFS
|
||||
.getFileChecksum(sourcePath) : null;
|
||||
|
||||
long bytesRead = copyToTmpFile(tmpTargetPath, targetFS, sourceFileStatus,
|
||||
context, fileAttributes, sourceChecksum);
|
||||
final long offset = action == FileAction.APPEND ? targetFS.getFileStatus(
|
||||
target).getLen() : 0;
|
||||
long bytesRead = copyToFile(targetPath, targetFS, sourceFileStatus,
|
||||
offset, context, fileAttributes, sourceChecksum);
|
||||
|
||||
compareFileLengths(sourceFileStatus, tmpTargetPath, configuration,
|
||||
bytesRead);
|
||||
compareFileLengths(sourceFileStatus, targetPath, configuration, bytesRead
|
||||
+ offset);
|
||||
//At this point, src&dest lengths are same. if length==0, we skip checksum
|
||||
if ((bytesRead != 0) && (!skipCrc)) {
|
||||
compareCheckSums(sourceFS, sourceFileStatus.getPath(), sourceChecksum,
|
||||
targetFS, tmpTargetPath);
|
||||
targetFS, targetPath);
|
||||
}
|
||||
// it's not append case, thus we first write to a temporary file, rename
|
||||
// it to the target path.
|
||||
if (!toAppend) {
|
||||
promoteTmpToTarget(targetPath, target, targetFS);
|
||||
}
|
||||
promoteTmpToTarget(tmpTargetPath, target, targetFS);
|
||||
return bytesRead;
|
||||
|
||||
} finally {
|
||||
if (targetFS.exists(tmpTargetPath))
|
||||
targetFS.delete(tmpTargetPath, false);
|
||||
// note that for append case, it is possible that we append partial data
|
||||
// and then fail. In that case, for the next retry, we either reuse the
|
||||
// partial appended data if it is good or we overwrite the whole file
|
||||
if (!toAppend && targetFS.exists(targetPath)) {
|
||||
targetFS.delete(targetPath, false);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -147,29 +160,37 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
return null;
|
||||
}
|
||||
|
||||
private long copyToTmpFile(Path tmpTargetPath, FileSystem targetFS,
|
||||
FileStatus sourceFileStatus, Mapper.Context context,
|
||||
private long copyToFile(Path targetPath, FileSystem targetFS,
|
||||
FileStatus sourceFileStatus, long sourceOffset, Mapper.Context context,
|
||||
EnumSet<FileAttribute> fileAttributes, final FileChecksum sourceChecksum)
|
||||
throws IOException {
|
||||
FsPermission permission = FsPermission.getFileDefault().applyUMask(
|
||||
FsPermission.getUMask(targetFS.getConf()));
|
||||
OutputStream outStream = new BufferedOutputStream(
|
||||
targetFS.create(tmpTargetPath, permission,
|
||||
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE), BUFFER_SIZE,
|
||||
getReplicationFactor(fileAttributes, sourceFileStatus, targetFS,
|
||||
tmpTargetPath),
|
||||
getBlockSize(fileAttributes, sourceFileStatus, targetFS,
|
||||
tmpTargetPath),
|
||||
context, getChecksumOpt(fileAttributes, sourceChecksum)));
|
||||
return copyBytes(sourceFileStatus, outStream, BUFFER_SIZE, context);
|
||||
final OutputStream outStream;
|
||||
if (action == FileAction.OVERWRITE) {
|
||||
final short repl = getReplicationFactor(fileAttributes, sourceFileStatus,
|
||||
targetFS, targetPath);
|
||||
final long blockSize = getBlockSize(fileAttributes, sourceFileStatus,
|
||||
targetFS, targetPath);
|
||||
FSDataOutputStream out = targetFS.create(targetPath, permission,
|
||||
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
|
||||
BUFFER_SIZE, repl, blockSize, context,
|
||||
getChecksumOpt(fileAttributes, sourceChecksum));
|
||||
outStream = new BufferedOutputStream(out);
|
||||
} else {
|
||||
outStream = new BufferedOutputStream(targetFS.append(targetPath,
|
||||
BUFFER_SIZE));
|
||||
}
|
||||
return copyBytes(sourceFileStatus, sourceOffset, outStream, BUFFER_SIZE,
|
||||
context);
|
||||
}
|
||||
|
||||
private void compareFileLengths(FileStatus sourceFileStatus, Path target,
|
||||
Configuration configuration, long bytesRead)
|
||||
Configuration configuration, long targetLen)
|
||||
throws IOException {
|
||||
final Path sourcePath = sourceFileStatus.getPath();
|
||||
FileSystem fs = sourcePath.getFileSystem(configuration);
|
||||
if (fs.getFileStatus(sourcePath).getLen() != bytesRead)
|
||||
if (fs.getFileStatus(sourcePath).getLen() != targetLen)
|
||||
throw new IOException("Mismatch in length of source:" + sourcePath
|
||||
+ " and target:" + target);
|
||||
}
|
||||
|
@ -215,8 +236,8 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
}
|
||||
|
||||
@VisibleForTesting
|
||||
long copyBytes(FileStatus sourceFileStatus, OutputStream outStream,
|
||||
int bufferSize, Mapper.Context context)
|
||||
long copyBytes(FileStatus sourceFileStatus, long sourceOffset,
|
||||
OutputStream outStream, int bufferSize, Mapper.Context context)
|
||||
throws IOException {
|
||||
Path source = sourceFileStatus.getPath();
|
||||
byte buf[] = new byte[bufferSize];
|
||||
|
@ -225,19 +246,21 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
|
||||
try {
|
||||
inStream = getInputStream(source, context.getConfiguration());
|
||||
int bytesRead = readBytes(inStream, buf);
|
||||
int bytesRead = readBytes(inStream, buf, sourceOffset);
|
||||
while (bytesRead >= 0) {
|
||||
totalBytesRead += bytesRead;
|
||||
if (action == FileAction.APPEND) {
|
||||
sourceOffset += bytesRead;
|
||||
}
|
||||
outStream.write(buf, 0, bytesRead);
|
||||
updateContextStatus(totalBytesRead, context, sourceFileStatus);
|
||||
bytesRead = inStream.read(buf);
|
||||
bytesRead = readBytes(inStream, buf, sourceOffset);
|
||||
}
|
||||
outStream.close();
|
||||
outStream = null;
|
||||
} finally {
|
||||
IOUtils.cleanup(LOG, outStream, inStream);
|
||||
}
|
||||
|
||||
return totalBytesRead;
|
||||
}
|
||||
|
||||
|
@ -254,24 +277,27 @@ public class RetriableFileCopyCommand extends RetriableCommand {
|
|||
context.setStatus(message.toString());
|
||||
}
|
||||
|
||||
private static int readBytes(InputStream inStream, byte buf[])
|
||||
throws IOException {
|
||||
private static int readBytes(ThrottledInputStream inStream, byte buf[],
|
||||
long position) throws IOException {
|
||||
try {
|
||||
return inStream.read(buf);
|
||||
}
|
||||
catch (IOException e) {
|
||||
if (position == 0) {
|
||||
return inStream.read(buf);
|
||||
} else {
|
||||
return inStream.read(position, buf, 0, buf.length);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new CopyReadException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private static ThrottledInputStream getInputStream(Path path, Configuration conf)
|
||||
throws IOException {
|
||||
private static ThrottledInputStream getInputStream(Path path,
|
||||
Configuration conf) throws IOException {
|
||||
try {
|
||||
FileSystem fs = path.getFileSystem(conf);
|
||||
long bandwidthMB = conf.getInt(DistCpConstants.CONF_LABEL_BANDWIDTH_MB,
|
||||
DistCpConstants.DEFAULT_BANDWIDTH_MB);
|
||||
return new ThrottledInputStream(new BufferedInputStream(fs.open(path)),
|
||||
bandwidthMB * 1024 * 1024);
|
||||
FSDataInputStream in = fs.open(path);
|
||||
return new ThrottledInputStream(in, bandwidthMB * 1024 * 1024);
|
||||
}
|
||||
catch (IOException e) {
|
||||
throw new CopyReadException(e);
|
||||
|
|
|
@ -21,6 +21,11 @@ package org.apache.hadoop.tools.util;
|
|||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.PositionedReadable;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
|
||||
/**
|
||||
* The ThrottleInputStream provides bandwidth throttling on a specified
|
||||
* InputStream. It is implemented as a wrapper on top of another InputStream
|
||||
|
@ -90,6 +95,25 @@ public class ThrottledInputStream extends InputStream {
|
|||
return readLen;
|
||||
}
|
||||
|
||||
/**
|
||||
* Read bytes starting from the specified position. This requires rawStream is
|
||||
* an instance of {@link PositionedReadable}.
|
||||
*/
|
||||
public int read(long position, byte[] buffer, int offset, int length)
|
||||
throws IOException {
|
||||
if (!(rawStream instanceof PositionedReadable)) {
|
||||
throw new UnsupportedOperationException(
|
||||
"positioned read is not supported by the internal stream");
|
||||
}
|
||||
throttle();
|
||||
int readLen = ((PositionedReadable) rawStream).read(position, buffer,
|
||||
offset, length);
|
||||
if (readLen != -1) {
|
||||
bytesRead += readLen;
|
||||
}
|
||||
return readLen;
|
||||
}
|
||||
|
||||
private void throttle() throws IOException {
|
||||
if (getBytesPerSec() > maxBytesPerSec) {
|
||||
try {
|
||||
|
|
|
@ -18,9 +18,12 @@
|
|||
|
||||
package org.apache.hadoop.tools;
|
||||
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.test.GenericTestUtils;
|
||||
import org.apache.hadoop.tools.DistCpOptions.*;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
|
@ -554,4 +557,45 @@ public class TestOptionsParser {
|
|||
Assert.assertEquals(conf.get(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel()), "U");
|
||||
Assert.assertEquals(conf.getInt(DistCpOptionSwitch.BANDWIDTH.getConfigLabel(), -1), 11);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAppendOption() {
|
||||
Configuration conf = new Configuration();
|
||||
Assert.assertFalse(conf.getBoolean(
|
||||
DistCpOptionSwitch.APPEND.getConfigLabel(), false));
|
||||
Assert.assertFalse(conf.getBoolean(
|
||||
DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
|
||||
|
||||
DistCpOptions options = OptionsParser.parse(new String[] { "-update",
|
||||
"-append", "hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/" });
|
||||
options.appendToConf(conf);
|
||||
Assert.assertTrue(conf.getBoolean(
|
||||
DistCpOptionSwitch.APPEND.getConfigLabel(), false));
|
||||
Assert.assertTrue(conf.getBoolean(
|
||||
DistCpOptionSwitch.SYNC_FOLDERS.getConfigLabel(), false));
|
||||
|
||||
// make sure -append is only valid when -update is specified
|
||||
try {
|
||||
options = OptionsParser.parse(new String[] { "-append",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/" });
|
||||
fail("Append should fail if update option is not specified");
|
||||
} catch (IllegalArgumentException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"Append is valid only with update options", e);
|
||||
}
|
||||
|
||||
// make sure -append is invalid when skipCrc is specified
|
||||
try {
|
||||
options = OptionsParser.parse(new String[] {
|
||||
"-append", "-update", "-skipcrccheck",
|
||||
"hdfs://localhost:8020/source/first",
|
||||
"hdfs://localhost:8020/target/" });
|
||||
fail("Append should fail if skipCrc option is specified");
|
||||
} catch (IllegalArgumentException e) {
|
||||
GenericTestUtils.assertExceptionContains(
|
||||
"Append is disallowed when skipping CRC", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,11 +25,13 @@ import java.security.PrivilegedAction;
|
|||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Random;
|
||||
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Options.ChecksumOpt;
|
||||
|
@ -118,6 +120,16 @@ public class TestCopyMapper {
|
|||
touchFile(SOURCE_PATH + "/7/8/9");
|
||||
}
|
||||
|
||||
private static void appendSourceData() throws Exception {
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
for (Path source : pathList) {
|
||||
if (fs.getFileStatus(source).isFile()) {
|
||||
// append 2048 bytes per file
|
||||
appendFile(source, DEFAULT_FILE_SIZE * 2);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private static void createSourceDataWithDifferentBlockSize() throws Exception {
|
||||
mkdirs(SOURCE_PATH + "/1");
|
||||
mkdirs(SOURCE_PATH + "/2");
|
||||
|
@ -201,85 +213,132 @@ public class TestCopyMapper {
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Append specified length of bytes to a given file
|
||||
*/
|
||||
private static void appendFile(Path p, int length) throws IOException {
|
||||
byte[] toAppend = new byte[length];
|
||||
Random random = new Random();
|
||||
random.nextBytes(toAppend);
|
||||
FSDataOutputStream out = cluster.getFileSystem().append(p);
|
||||
try {
|
||||
out.write(toAppend);
|
||||
} finally {
|
||||
IOUtils.closeStream(out);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCopyWithDifferentChecksumType() throws Exception {
|
||||
testCopy(true);
|
||||
}
|
||||
|
||||
@Test(timeout=40000)
|
||||
public void testRun() {
|
||||
public void testRun() throws Exception {
|
||||
testCopy(false);
|
||||
}
|
||||
|
||||
private void testCopy(boolean preserveChecksum) {
|
||||
try {
|
||||
deleteState();
|
||||
if (preserveChecksum) {
|
||||
createSourceDataWithDifferentChecksumType();
|
||||
} else {
|
||||
createSourceData();
|
||||
}
|
||||
@Test
|
||||
public void testCopyWithAppend() throws Exception {
|
||||
final FileSystem fs = cluster.getFileSystem();
|
||||
// do the first distcp
|
||||
testCopy(false);
|
||||
// start appending data to source
|
||||
appendSourceData();
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
Configuration configuration = context.getConfiguration();
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes
|
||||
= EnumSet.of(DistCpOptions.FileAttribute.REPLICATION);
|
||||
if (preserveChecksum) {
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.CHECKSUMTYPE);
|
||||
}
|
||||
configuration.set(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel(),
|
||||
DistCpUtils.packAttributes(fileAttributes));
|
||||
|
||||
copyMapper.setup(context);
|
||||
|
||||
for (Path path: pathList) {
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
new CopyListingFileStatus(fs.getFileStatus(path)), context);
|
||||
}
|
||||
|
||||
// Check that the maps worked.
|
||||
for (Path path : pathList) {
|
||||
final Path targetPath = new Path(path.toString()
|
||||
.replaceAll(SOURCE_PATH, TARGET_PATH));
|
||||
Assert.assertTrue(fs.exists(targetPath));
|
||||
Assert.assertTrue(fs.isFile(targetPath) == fs.isFile(path));
|
||||
FileStatus sourceStatus = fs.getFileStatus(path);
|
||||
FileStatus targetStatus = fs.getFileStatus(targetPath);
|
||||
Assert.assertEquals(sourceStatus.getReplication(),
|
||||
targetStatus.getReplication());
|
||||
if (preserveChecksum) {
|
||||
Assert.assertEquals(sourceStatus.getBlockSize(),
|
||||
targetStatus.getBlockSize());
|
||||
}
|
||||
Assert.assertTrue(!fs.isFile(targetPath)
|
||||
|| fs.getFileChecksum(targetPath).equals(fs.getFileChecksum(path)));
|
||||
}
|
||||
|
||||
Assert.assertEquals(pathList.size(),
|
||||
stubContext.getReporter().getCounter(CopyMapper.Counter.COPY).getValue());
|
||||
if (!preserveChecksum) {
|
||||
Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
|
||||
.getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
|
||||
.getValue());
|
||||
} else {
|
||||
Assert.assertEquals(nFiles * NON_DEFAULT_BLOCK_SIZE * 2, stubContext
|
||||
.getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
|
||||
.getValue());
|
||||
}
|
||||
|
||||
testCopyingExistingFiles(fs, copyMapper, context);
|
||||
for (Text value : stubContext.getWriter().values()) {
|
||||
Assert.assertTrue(value.toString() + " is not skipped", value.toString().startsWith("SKIP:"));
|
||||
}
|
||||
// do the distcp again with -update and -append option
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, CopyListingFileStatus, Text, Text>.Context context =
|
||||
stubContext.getContext();
|
||||
// Enable append
|
||||
context.getConfiguration().setBoolean(
|
||||
DistCpOptionSwitch.APPEND.getConfigLabel(), true);
|
||||
copyMapper.setup(context);
|
||||
for (Path path: pathList) {
|
||||
copyMapper.map(new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
new CopyListingFileStatus(cluster.getFileSystem().getFileStatus(
|
||||
path)), context);
|
||||
}
|
||||
catch (Exception e) {
|
||||
LOG.error("Unexpected exception: ", e);
|
||||
Assert.assertTrue(false);
|
||||
|
||||
verifyCopy(fs, false);
|
||||
// verify that we only copied new appended data
|
||||
Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE * 2, stubContext
|
||||
.getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
|
||||
.getValue());
|
||||
Assert.assertEquals(pathList.size(), stubContext.getReporter().
|
||||
getCounter(CopyMapper.Counter.COPY).getValue());
|
||||
}
|
||||
|
||||
private void testCopy(boolean preserveChecksum) throws Exception {
|
||||
deleteState();
|
||||
if (preserveChecksum) {
|
||||
createSourceDataWithDifferentChecksumType();
|
||||
} else {
|
||||
createSourceData();
|
||||
}
|
||||
|
||||
FileSystem fs = cluster.getFileSystem();
|
||||
CopyMapper copyMapper = new CopyMapper();
|
||||
StubContext stubContext = new StubContext(getConfiguration(), null, 0);
|
||||
Mapper<Text, CopyListingFileStatus, Text, Text>.Context context
|
||||
= stubContext.getContext();
|
||||
|
||||
Configuration configuration = context.getConfiguration();
|
||||
EnumSet<DistCpOptions.FileAttribute> fileAttributes
|
||||
= EnumSet.of(DistCpOptions.FileAttribute.REPLICATION);
|
||||
if (preserveChecksum) {
|
||||
fileAttributes.add(DistCpOptions.FileAttribute.CHECKSUMTYPE);
|
||||
}
|
||||
configuration.set(DistCpOptionSwitch.PRESERVE_STATUS.getConfigLabel(),
|
||||
DistCpUtils.packAttributes(fileAttributes));
|
||||
|
||||
copyMapper.setup(context);
|
||||
|
||||
for (Path path: pathList) {
|
||||
copyMapper.map(
|
||||
new Text(DistCpUtils.getRelativePath(new Path(SOURCE_PATH), path)),
|
||||
new CopyListingFileStatus(fs.getFileStatus(path)), context);
|
||||
}
|
||||
|
||||
// Check that the maps worked.
|
||||
verifyCopy(fs, preserveChecksum);
|
||||
Assert.assertEquals(pathList.size(), stubContext.getReporter()
|
||||
.getCounter(CopyMapper.Counter.COPY).getValue());
|
||||
if (!preserveChecksum) {
|
||||
Assert.assertEquals(nFiles * DEFAULT_FILE_SIZE, stubContext
|
||||
.getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
|
||||
.getValue());
|
||||
} else {
|
||||
Assert.assertEquals(nFiles * NON_DEFAULT_BLOCK_SIZE * 2, stubContext
|
||||
.getReporter().getCounter(CopyMapper.Counter.BYTESCOPIED)
|
||||
.getValue());
|
||||
}
|
||||
|
||||
testCopyingExistingFiles(fs, copyMapper, context);
|
||||
for (Text value : stubContext.getWriter().values()) {
|
||||
Assert.assertTrue(value.toString() + " is not skipped", value
|
||||
.toString().startsWith("SKIP:"));
|
||||
}
|
||||
}
|
||||
|
||||
private void verifyCopy(FileSystem fs, boolean preserveChecksum)
|
||||
throws Exception {
|
||||
for (Path path : pathList) {
|
||||
final Path targetPath = new Path(path.toString().replaceAll(SOURCE_PATH,
|
||||
TARGET_PATH));
|
||||
Assert.assertTrue(fs.exists(targetPath));
|
||||
Assert.assertTrue(fs.isFile(targetPath) == fs.isFile(path));
|
||||
FileStatus sourceStatus = fs.getFileStatus(path);
|
||||
FileStatus targetStatus = fs.getFileStatus(targetPath);
|
||||
Assert.assertEquals(sourceStatus.getReplication(),
|
||||
targetStatus.getReplication());
|
||||
if (preserveChecksum) {
|
||||
Assert.assertEquals(sourceStatus.getBlockSize(),
|
||||
targetStatus.getBlockSize());
|
||||
}
|
||||
Assert.assertTrue(!fs.isFile(targetPath)
|
||||
|| fs.getFileChecksum(targetPath).equals(fs.getFileChecksum(path)));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -22,6 +22,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.mapreduce.*;
|
||||
import org.apache.hadoop.tools.mapred.CopyMapper.FileAction;
|
||||
import org.junit.Test;
|
||||
import static org.junit.Assert.*;
|
||||
import static org.mockito.Mockito.*;
|
||||
|
@ -48,8 +49,8 @@ public class TestRetriableFileCopyCommand {
|
|||
|
||||
Exception actualEx = null;
|
||||
try {
|
||||
new RetriableFileCopyCommand("testFailOnCloseError")
|
||||
.copyBytes(stat, out, 512, context);
|
||||
new RetriableFileCopyCommand("testFailOnCloseError", FileAction.OVERWRITE)
|
||||
.copyBytes(stat, 0, out, 512, context);
|
||||
} catch (Exception e) {
|
||||
actualEx = e;
|
||||
}
|
||||
|
|
|
@ -18,8 +18,6 @@
|
|||
|
||||
Yarn Scheduler Load Simulator (SLS)
|
||||
|
||||
\[ {{{./index.html}Go Back}} \]
|
||||
|
||||
%{toc|section=1|fromDepth=0}
|
||||
|
||||
* Overview
|
||||
|
|
|
@ -90,6 +90,33 @@ Release 2.5.0 - UNRELEASED
|
|||
|
||||
YARN-2017. Merged some of the common scheduler code. (Jian He via vinodkv)
|
||||
|
||||
YARN-2049. Added delegation-token support for the Timeline Server. (Zhijie
|
||||
Shen via vinodkv)
|
||||
|
||||
YARN-1936. Added security support for the Timeline Client. (Zhijie Shen via
|
||||
vinodkv)
|
||||
|
||||
YARN-1937. Added owner-only ACLs support for Timeline Client and server.
|
||||
(Zhijie Shen via vinodkv)
|
||||
|
||||
YARN-2012. Fair Scheduler: allow default queue placement rule to take an
|
||||
arbitrary queue (Ashwin Shankar via Sandy Ryza)
|
||||
|
||||
YARN-2059. Added admin ACLs support to Timeline Server. (Zhijie Shen via
|
||||
vinodkv)
|
||||
|
||||
YARN-2073. Fair Scheduler: Add a utilization threshold to prevent preempting
|
||||
resources when cluster is free (Karthik Kambatla via Sandy Ryza)
|
||||
|
||||
YARN-2071. Modified levelDB store permissions to be readable only by the
|
||||
server user. (Zhijie Shen via vinodkv)
|
||||
|
||||
YARN-2107. Refactored timeline classes into o.a.h.y.s.timeline package. (Vinod
|
||||
Kumar Vavilapalli via zjshen)
|
||||
|
||||
YARN-596. Use scheduling policies throughout the queue hierarchy to decide
|
||||
which containers to preempt (Wei Yan via Sandy Ryza)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
||||
BUG FIXES
|
||||
|
@ -125,6 +152,16 @@ Release 2.5.0 - UNRELEASED
|
|||
YARN-2089. FairScheduler: QueuePlacementPolicy and QueuePlacementRule
|
||||
are missing audience annotations. (Zhihai Xu via kasha)
|
||||
|
||||
YARN-2096. Race in TestRMRestart#testQueueMetricsOnRMRestart.
|
||||
(Anubhav Dhoot via kasha)
|
||||
|
||||
YARN-2105. Fix TestFairScheduler after YARN-2012. (Ashwin Shankar via
|
||||
Sandy Ryza)
|
||||
|
||||
YARN-2112. Fixed yarn-common's pom.xml to include jackson dependencies so
|
||||
that both Timeline Server and client can access them. (Zhijie Shen via
|
||||
vinodkv)
|
||||
|
||||
Release 2.4.1 - UNRELEASED
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -0,0 +1,63 @@
|
|||
/**
|
||||
* 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.yarn.api.records.timeline;
|
||||
|
||||
import javax.xml.bind.annotation.XmlAccessType;
|
||||
import javax.xml.bind.annotation.XmlAccessorType;
|
||||
import javax.xml.bind.annotation.XmlElement;
|
||||
import javax.xml.bind.annotation.XmlRootElement;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
|
||||
/**
|
||||
* The response of delegation token related request
|
||||
*/
|
||||
@XmlRootElement(name = "delegationtoken")
|
||||
@XmlAccessorType(XmlAccessType.NONE)
|
||||
@Public
|
||||
@Unstable
|
||||
public class TimelineDelegationTokenResponse {
|
||||
|
||||
private String type;
|
||||
private Object content;
|
||||
|
||||
public TimelineDelegationTokenResponse() {
|
||||
|
||||
}
|
||||
|
||||
@XmlElement(name = "type")
|
||||
public String getType() {
|
||||
return type;
|
||||
}
|
||||
|
||||
public void setType(String type) {
|
||||
this.type = type;
|
||||
}
|
||||
|
||||
@XmlElement(name = "content")
|
||||
public Object getContent() {
|
||||
return content;
|
||||
}
|
||||
|
||||
public void setContent(Object content) {
|
||||
this.content = content;
|
||||
}
|
||||
|
||||
}
|
|
@ -107,6 +107,17 @@ public class TimelinePutResponse {
|
|||
*/
|
||||
public static final int IO_EXCEPTION = 2;
|
||||
|
||||
/**
|
||||
* Error code returned if the user specifies the timeline system reserved
|
||||
* filter key
|
||||
*/
|
||||
public static final int SYSTEM_FILTER_CONFLICT = 3;
|
||||
|
||||
/**
|
||||
* Error code returned if the user is denied to access the timeline data
|
||||
*/
|
||||
public static final int ACCESS_DENIED = 4;
|
||||
|
||||
private String entityId;
|
||||
private String entityType;
|
||||
private int errorCode;
|
||||
|
|
|
@ -1136,14 +1136,6 @@ public class YarnConfiguration extends Configuration {
|
|||
public static final String DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_ADDRESS =
|
||||
"0.0.0.0:" + DEFAULT_TIMELINE_SERVICE_WEBAPP_HTTPS_PORT;
|
||||
|
||||
/**The kerberos principal to be used for spnego filter for timeline service.*/
|
||||
public static final String TIMELINE_SERVICE_WEBAPP_SPNEGO_USER_NAME_KEY =
|
||||
TIMELINE_SERVICE_PREFIX + "webapp.spnego-principal";
|
||||
|
||||
/**The kerberos keytab to be used for spnego filter for timeline service.*/
|
||||
public static final String TIMELINE_SERVICE_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
|
||||
TIMELINE_SERVICE_PREFIX + "webapp.spnego-keytab-file";
|
||||
|
||||
/** Timeline service store class */
|
||||
public static final String TIMELINE_SERVICE_STORE =
|
||||
TIMELINE_SERVICE_PREFIX + "store-class";
|
||||
|
|
|
@ -23,17 +23,17 @@ import java.io.IOException;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Public;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.service.AbstractService;
|
||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
||||
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
|
||||
import org.apache.hadoop.yarn.client.api.impl.TimelineClientImpl;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||
|
||||
/**
|
||||
* A client library that can be used to post some information in terms of a
|
||||
* number of conceptual entities.
|
||||
*
|
||||
* @See Entity
|
||||
*/
|
||||
@Public
|
||||
@Unstable
|
||||
|
@ -67,4 +67,22 @@ public abstract class TimelineClient extends AbstractService {
|
|||
public abstract TimelinePutResponse putEntities(
|
||||
TimelineEntity... entities) throws IOException, YarnException;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Get a delegation token so as to be able to talk to the timeline server in a
|
||||
* secure way.
|
||||
* </p>
|
||||
*
|
||||
* @param renewer
|
||||
* Address of the renewer who can renew these tokens when needed by
|
||||
* securely talking to the timeline server
|
||||
* @return a delegation token ({@link Token}) that can be used to talk to the
|
||||
* timeline server
|
||||
* @throws IOException
|
||||
* @throws YarnException
|
||||
*/
|
||||
@Public
|
||||
public abstract Token<TimelineDelegationTokenIdentifier> getDelegationToken(
|
||||
String renewer) throws IOException, YarnException;
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,252 @@
|
|||
/**
|
||||
* 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.yarn.client.api.impl;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URL;
|
||||
import java.net.URLEncoder;
|
||||
import java.text.MessageFormat;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||
import org.apache.hadoop.security.authentication.client.Authenticator;
|
||||
import org.apache.hadoop.security.authentication.client.KerberosAuthenticator;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineDelegationTokenResponse;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenOperation;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineAuthenticationConsts;
|
||||
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
|
||||
import org.codehaus.jackson.JsonNode;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
/**
|
||||
* A <code>KerberosAuthenticator</code> subclass that fallback to
|
||||
* {@link TimelineAuthenticationConsts}.
|
||||
*/
|
||||
@Private
|
||||
@Unstable
|
||||
public class TimelineAuthenticator extends KerberosAuthenticator {
|
||||
|
||||
private static ObjectMapper mapper;
|
||||
|
||||
static {
|
||||
mapper = new ObjectMapper();
|
||||
YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the fallback authenticator if the server does not use Kerberos
|
||||
* SPNEGO HTTP authentication.
|
||||
*
|
||||
* @return a {@link TimelineAuthenticationConsts} instance.
|
||||
*/
|
||||
@Override
|
||||
protected Authenticator getFallBackAuthenticator() {
|
||||
return new TimelineAuthenticator();
|
||||
}
|
||||
|
||||
public static void injectDelegationToken(Map<String, String> params,
|
||||
Token<?> dtToken)
|
||||
throws IOException {
|
||||
if (dtToken != null) {
|
||||
params.put(TimelineAuthenticationConsts.DELEGATION_PARAM,
|
||||
dtToken.encodeToUrlString());
|
||||
}
|
||||
}
|
||||
|
||||
private boolean hasDelegationToken(URL url) {
|
||||
return url.getQuery().contains(
|
||||
TimelineAuthenticationConsts.DELEGATION_PARAM + "=");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void authenticate(URL url, AuthenticatedURL.Token token)
|
||||
throws IOException, AuthenticationException {
|
||||
if (!hasDelegationToken(url)) {
|
||||
super.authenticate(url, token);
|
||||
}
|
||||
}
|
||||
|
||||
public static Token<TimelineDelegationTokenIdentifier> getDelegationToken(
|
||||
URL url, AuthenticatedURL.Token token, String renewer) throws IOException {
|
||||
TimelineDelegationTokenOperation op =
|
||||
TimelineDelegationTokenOperation.GETDELEGATIONTOKEN;
|
||||
Map<String, String> params = new HashMap<String, String>();
|
||||
params.put(TimelineAuthenticationConsts.OP_PARAM, op.toString());
|
||||
params.put(TimelineAuthenticationConsts.RENEWER_PARAM, renewer);
|
||||
url = appendParams(url, params);
|
||||
AuthenticatedURL aUrl =
|
||||
new AuthenticatedURL(new TimelineAuthenticator());
|
||||
try {
|
||||
HttpURLConnection conn = aUrl.openConnection(url, token);
|
||||
conn.setRequestMethod(op.getHttpMethod());
|
||||
TimelineDelegationTokenResponse dtRes = validateAndParseResponse(conn);
|
||||
if (!dtRes.getType().equals(
|
||||
TimelineAuthenticationConsts.DELEGATION_TOKEN_URL)) {
|
||||
throw new IOException("The response content is not expected: "
|
||||
+ dtRes.getContent());
|
||||
}
|
||||
String tokenStr = dtRes.getContent().toString();
|
||||
Token<TimelineDelegationTokenIdentifier> dToken =
|
||||
new Token<TimelineDelegationTokenIdentifier>();
|
||||
dToken.decodeFromUrlString(tokenStr);
|
||||
return dToken;
|
||||
} catch (AuthenticationException ex) {
|
||||
throw new IOException(ex.toString(), ex);
|
||||
}
|
||||
}
|
||||
|
||||
public static long renewDelegationToken(URL url,
|
||||
AuthenticatedURL.Token token,
|
||||
Token<TimelineDelegationTokenIdentifier> dToken) throws IOException {
|
||||
Map<String, String> params = new HashMap<String, String>();
|
||||
params.put(TimelineAuthenticationConsts.OP_PARAM,
|
||||
TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.toString());
|
||||
params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
|
||||
dToken.encodeToUrlString());
|
||||
url = appendParams(url, params);
|
||||
AuthenticatedURL aUrl =
|
||||
new AuthenticatedURL(new TimelineAuthenticator());
|
||||
try {
|
||||
HttpURLConnection conn = aUrl.openConnection(url, token);
|
||||
conn.setRequestMethod(
|
||||
TimelineDelegationTokenOperation.RENEWDELEGATIONTOKEN.getHttpMethod());
|
||||
TimelineDelegationTokenResponse dtRes = validateAndParseResponse(conn);
|
||||
if (!dtRes.getType().equals(
|
||||
TimelineAuthenticationConsts.DELEGATION_TOKEN_EXPIRATION_TIME)) {
|
||||
throw new IOException("The response content is not expected: "
|
||||
+ dtRes.getContent());
|
||||
}
|
||||
return Long.valueOf(dtRes.getContent().toString());
|
||||
} catch (AuthenticationException ex) {
|
||||
throw new IOException(ex.toString(), ex);
|
||||
}
|
||||
}
|
||||
|
||||
public static void cancelDelegationToken(URL url,
|
||||
AuthenticatedURL.Token token,
|
||||
Token<TimelineDelegationTokenIdentifier> dToken) throws IOException {
|
||||
Map<String, String> params = new HashMap<String, String>();
|
||||
params.put(TimelineAuthenticationConsts.OP_PARAM,
|
||||
TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN.toString());
|
||||
params.put(TimelineAuthenticationConsts.TOKEN_PARAM,
|
||||
dToken.encodeToUrlString());
|
||||
url = appendParams(url, params);
|
||||
AuthenticatedURL aUrl =
|
||||
new AuthenticatedURL(new TimelineAuthenticator());
|
||||
try {
|
||||
HttpURLConnection conn = aUrl.openConnection(url, token);
|
||||
conn.setRequestMethod(TimelineDelegationTokenOperation.CANCELDELEGATIONTOKEN
|
||||
.getHttpMethod());
|
||||
validateAndParseResponse(conn);
|
||||
} catch (AuthenticationException ex) {
|
||||
throw new IOException(ex.toString(), ex);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Convenience method that appends parameters an HTTP <code>URL</code>.
|
||||
*
|
||||
* @param url
|
||||
* the url.
|
||||
* @param params
|
||||
* the query string parameters.
|
||||
*
|
||||
* @return a <code>URL</code>
|
||||
*
|
||||
* @throws IOException
|
||||
* thrown if an IO error occurs.
|
||||
*/
|
||||
public static URL appendParams(URL url, Map<String, String> params)
|
||||
throws IOException {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append(url);
|
||||
String separator = url.toString().contains("?") ? "&" : "?";
|
||||
for (Map.Entry<String, String> entry : params.entrySet()) {
|
||||
sb.append(separator).append(entry.getKey()).append("=").
|
||||
append(URLEncoder.encode(entry.getValue(), "UTF8"));
|
||||
separator = "&";
|
||||
}
|
||||
return new URL(sb.toString());
|
||||
}
|
||||
|
||||
/**
|
||||
* Validates the response of an <code>HttpURLConnection</code>. If the current
|
||||
* status code is not 200, it will throw an exception with a detail message
|
||||
* using Server side error messages if available. Otherwise,
|
||||
* {@link TimelineDelegationTokenResponse} will be parsed and returned.
|
||||
*
|
||||
* @param conn
|
||||
* the <code>HttpURLConnection</code>.
|
||||
* @return
|
||||
* @throws IOException
|
||||
* thrown if the current status code is not 200 or the JSON response
|
||||
* cannot be parsed correctly
|
||||
*/
|
||||
private static TimelineDelegationTokenResponse validateAndParseResponse(
|
||||
HttpURLConnection conn) throws IOException {
|
||||
int status = conn.getResponseCode();
|
||||
JsonNode json = mapper.readTree(conn.getInputStream());
|
||||
if (status == HttpURLConnection.HTTP_OK) {
|
||||
return mapper.readValue(json, TimelineDelegationTokenResponse.class);
|
||||
} else {
|
||||
// If the status code is not 200, some thing wrong should happen at the
|
||||
// server side, the JSON content is going to contain exception details.
|
||||
// We can use the JSON content to reconstruct the exception object.
|
||||
try {
|
||||
String message =
|
||||
json.get(TimelineAuthenticationConsts.ERROR_MESSAGE_JSON)
|
||||
.getTextValue();
|
||||
String exception =
|
||||
json.get(TimelineAuthenticationConsts.ERROR_EXCEPTION_JSON)
|
||||
.getTextValue();
|
||||
String className =
|
||||
json.get(TimelineAuthenticationConsts.ERROR_CLASSNAME_JSON)
|
||||
.getTextValue();
|
||||
|
||||
try {
|
||||
ClassLoader cl = TimelineAuthenticator.class.getClassLoader();
|
||||
Class<?> klass = cl.loadClass(className);
|
||||
Constructor<?> constr = klass.getConstructor(String.class);
|
||||
throw (IOException) constr.newInstance(message);
|
||||
} catch (IOException ex) {
|
||||
throw ex;
|
||||
} catch (Exception ex) {
|
||||
throw new IOException(MessageFormat.format("{0} - {1}", exception,
|
||||
message));
|
||||
}
|
||||
} catch (IOException ex) {
|
||||
if (ex.getCause() instanceof IOException) {
|
||||
throw (IOException) ex.getCause();
|
||||
}
|
||||
throw new IOException(
|
||||
MessageFormat.format("HTTP status [{0}], {1}",
|
||||
status, conn.getResponseMessage()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -18,24 +18,43 @@
|
|||
|
||||
package org.apache.hadoop.yarn.client.api.impl;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import javax.ws.rs.core.MediaType;
|
||||
|
||||
import org.apache.commons.cli.CommandLine;
|
||||
import org.apache.commons.cli.GnuParser;
|
||||
import org.apache.commons.cli.HelpFormatter;
|
||||
import org.apache.commons.cli.Options;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticatedURL;
|
||||
import org.apache.hadoop.security.authentication.client.AuthenticationException;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntities;
|
||||
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
||||
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
|
||||
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenSelector;
|
||||
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
||||
import org.apache.hadoop.yarn.webapp.YarnJacksonJaxbJsonProvider;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Joiner;
|
||||
|
@ -44,6 +63,8 @@ import com.sun.jersey.api.client.ClientResponse;
|
|||
import com.sun.jersey.api.client.WebResource;
|
||||
import com.sun.jersey.api.client.config.ClientConfig;
|
||||
import com.sun.jersey.api.client.config.DefaultClientConfig;
|
||||
import com.sun.jersey.client.urlconnection.HttpURLConnectionFactory;
|
||||
import com.sun.jersey.client.urlconnection.URLConnectionClientHandler;
|
||||
|
||||
@Private
|
||||
@Unstable
|
||||
|
@ -52,16 +73,29 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
private static final Log LOG = LogFactory.getLog(TimelineClientImpl.class);
|
||||
private static final String RESOURCE_URI_STR = "/ws/v1/timeline/";
|
||||
private static final Joiner JOINER = Joiner.on("");
|
||||
private static Options opts;
|
||||
static {
|
||||
opts = new Options();
|
||||
opts.addOption("put", true, "Put the TimelineEntities in a JSON file");
|
||||
opts.getOption("put").setArgName("Path to the JSON file");
|
||||
opts.addOption("help", false, "Print usage");
|
||||
}
|
||||
|
||||
private Client client;
|
||||
private URI resURI;
|
||||
private boolean isEnabled;
|
||||
private TimelineAuthenticatedURLConnectionFactory urlFactory;
|
||||
|
||||
public TimelineClientImpl() {
|
||||
super(TimelineClientImpl.class.getName());
|
||||
ClientConfig cc = new DefaultClientConfig();
|
||||
cc.getClasses().add(YarnJacksonJaxbJsonProvider.class);
|
||||
client = Client.create(cc);
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
urlFactory = new TimelineAuthenticatedURLConnectionFactory();
|
||||
client = new Client(new URLConnectionClientHandler(urlFactory), cc);
|
||||
} else {
|
||||
client = Client.create(cc);
|
||||
}
|
||||
}
|
||||
|
||||
protected void serviceInit(Configuration conf) throws Exception {
|
||||
|
@ -83,6 +117,9 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_WEBAPP_ADDRESS),
|
||||
RESOURCE_URI_STR));
|
||||
}
|
||||
if (UserGroupInformation.isSecurityEnabled()) {
|
||||
urlFactory.setService(TimelineUtils.buildTimelineTokenService(conf));
|
||||
}
|
||||
LOG.info("Timeline service address: " + resURI);
|
||||
}
|
||||
super.serviceInit(conf);
|
||||
|
@ -124,6 +161,13 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
return resp.getEntity(TimelinePutResponse.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Token<TimelineDelegationTokenIdentifier> getDelegationToken(
|
||||
String renewer) throws IOException, YarnException {
|
||||
return TimelineAuthenticator.getDelegationToken(resURI.toURL(),
|
||||
urlFactory.token, renewer);
|
||||
}
|
||||
|
||||
@Private
|
||||
@VisibleForTesting
|
||||
public ClientResponse doPostingEntities(TimelineEntities entities) {
|
||||
|
@ -133,4 +177,138 @@ public class TimelineClientImpl extends TimelineClient {
|
|||
.post(ClientResponse.class, entities);
|
||||
}
|
||||
|
||||
private static class TimelineAuthenticatedURLConnectionFactory
|
||||
implements HttpURLConnectionFactory {
|
||||
|
||||
private AuthenticatedURL.Token token;
|
||||
private TimelineAuthenticator authenticator;
|
||||
private Token<TimelineDelegationTokenIdentifier> dToken;
|
||||
private Text service;
|
||||
|
||||
public TimelineAuthenticatedURLConnectionFactory() {
|
||||
token = new AuthenticatedURL.Token();
|
||||
authenticator = new TimelineAuthenticator();
|
||||
}
|
||||
|
||||
@Override
|
||||
public HttpURLConnection getHttpURLConnection(URL url) throws IOException {
|
||||
try {
|
||||
if (dToken == null) {
|
||||
//TODO: need to take care of the renew case
|
||||
dToken = selectToken();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Timeline delegation token: " + dToken.toString());
|
||||
}
|
||||
}
|
||||
if (dToken != null) {
|
||||
Map<String, String> params = new HashMap<String, String>();
|
||||
TimelineAuthenticator.injectDelegationToken(params, dToken);
|
||||
url = TimelineAuthenticator.appendParams(url, params);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("URL with delegation token: " + url);
|
||||
}
|
||||
}
|
||||
return new AuthenticatedURL(authenticator).openConnection(url, token);
|
||||
} catch (AuthenticationException e) {
|
||||
LOG.error("Authentication failed when openning connection [" + url
|
||||
+ "] with token [" + token + "].", e);
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
private Token<TimelineDelegationTokenIdentifier> selectToken() {
|
||||
UserGroupInformation ugi;
|
||||
try {
|
||||
ugi = UserGroupInformation.getCurrentUser();
|
||||
} catch (IOException e) {
|
||||
String msg = "Error when getting the current user";
|
||||
LOG.error(msg, e);
|
||||
throw new YarnRuntimeException(msg, e);
|
||||
}
|
||||
TimelineDelegationTokenSelector tokenSelector =
|
||||
new TimelineDelegationTokenSelector();
|
||||
return tokenSelector.selectToken(
|
||||
service, ugi.getCredentials().getAllTokens());
|
||||
}
|
||||
|
||||
public void setService(Text service) {
|
||||
this.service = service;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
public static void main(String[] argv) throws Exception {
|
||||
CommandLine cliParser = new GnuParser().parse(opts, argv);
|
||||
if (cliParser.hasOption("put")) {
|
||||
String path = cliParser.getOptionValue("put");
|
||||
if (path != null && path.length() > 0) {
|
||||
putTimelineEntitiesInJSONFile(path);
|
||||
return;
|
||||
}
|
||||
}
|
||||
printUsage();
|
||||
}
|
||||
|
||||
/**
|
||||
* Put timeline data in a JSON file via command line.
|
||||
*
|
||||
* @param path
|
||||
* path to the {@link TimelineEntities} JSON file
|
||||
*/
|
||||
private static void putTimelineEntitiesInJSONFile(String path) {
|
||||
File jsonFile = new File(path);
|
||||
if (!jsonFile.exists()) {
|
||||
System.out.println("Error: File [" + jsonFile.getAbsolutePath()
|
||||
+ "] doesn't exist");
|
||||
return;
|
||||
}
|
||||
ObjectMapper mapper = new ObjectMapper();
|
||||
YarnJacksonJaxbJsonProvider.configObjectMapper(mapper);
|
||||
TimelineEntities entities = null;
|
||||
try {
|
||||
entities = mapper.readValue(jsonFile, TimelineEntities.class);
|
||||
} catch (Exception e) {
|
||||
System.err.println("Error: " + e.getMessage());
|
||||
e.printStackTrace(System.err);
|
||||
return;
|
||||
}
|
||||
Configuration conf = new YarnConfiguration();
|
||||
TimelineClient client = TimelineClient.createTimelineClient();
|
||||
client.init(conf);
|
||||
client.start();
|
||||
try {
|
||||
if (UserGroupInformation.isSecurityEnabled()
|
||||
&& conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, false)) {
|
||||
Token<TimelineDelegationTokenIdentifier> token =
|
||||
client.getDelegationToken(
|
||||
UserGroupInformation.getCurrentUser().getUserName());
|
||||
UserGroupInformation.getCurrentUser().addToken(token);
|
||||
}
|
||||
TimelinePutResponse response = client.putEntities(
|
||||
entities.getEntities().toArray(
|
||||
new TimelineEntity[entities.getEntities().size()]));
|
||||
if (response.getErrors().size() == 0) {
|
||||
System.out.println("Timeline data is successfully put");
|
||||
} else {
|
||||
for (TimelinePutResponse.TimelinePutError error : response.getErrors()) {
|
||||
System.out.println("TimelineEntity [" + error.getEntityType() + ":" +
|
||||
error.getEntityId() + "] is not successfully put. Error code: " +
|
||||
error.getErrorCode());
|
||||
}
|
||||
}
|
||||
} catch (Exception e) {
|
||||
System.err.println("Error: " + e.getMessage());
|
||||
e.printStackTrace(System.err);
|
||||
} finally {
|
||||
client.stop();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Helper function to print out usage
|
||||
*/
|
||||
private static void printUsage() {
|
||||
new HelpFormatter().printHelp("TimelineClient", opts);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
package org.apache.hadoop.yarn.client.api.impl;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
|
@ -29,8 +30,13 @@ import org.apache.commons.logging.LogFactory;
|
|||
import org.apache.hadoop.classification.InterfaceAudience.Private;
|
||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.DataInputByteBuffer;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
|
||||
|
@ -64,6 +70,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|||
import org.apache.hadoop.yarn.api.records.ApplicationReport;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerReport;
|
||||
import org.apache.hadoop.yarn.api.records.NodeReport;
|
||||
import org.apache.hadoop.yarn.api.records.NodeState;
|
||||
|
@ -74,6 +81,7 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
|||
import org.apache.hadoop.yarn.api.records.YarnClusterMetrics;
|
||||
import org.apache.hadoop.yarn.client.ClientRMProxy;
|
||||
import org.apache.hadoop.yarn.client.api.AHSClient;
|
||||
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
||||
import org.apache.hadoop.yarn.client.api.YarnClient;
|
||||
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
|
@ -82,8 +90,10 @@ import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException;
|
|||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
|
@ -97,8 +107,11 @@ public class YarnClientImpl extends YarnClient {
|
|||
protected long submitPollIntervalMillis;
|
||||
private long asyncApiPollIntervalMillis;
|
||||
private long asyncApiPollTimeoutMillis;
|
||||
protected AHSClient historyClient;
|
||||
private AHSClient historyClient;
|
||||
private boolean historyServiceEnabled;
|
||||
protected TimelineClient timelineClient;
|
||||
protected Text timelineService;
|
||||
protected boolean timelineServiceEnabled;
|
||||
|
||||
private static final String ROOT = "root";
|
||||
|
||||
|
@ -126,10 +139,17 @@ public class YarnClientImpl extends YarnClient {
|
|||
if (conf.getBoolean(YarnConfiguration.APPLICATION_HISTORY_ENABLED,
|
||||
YarnConfiguration.DEFAULT_APPLICATION_HISTORY_ENABLED)) {
|
||||
historyServiceEnabled = true;
|
||||
historyClient = AHSClientImpl.createAHSClient();
|
||||
historyClient.init(getConfig());
|
||||
historyClient = AHSClient.createAHSClient();
|
||||
historyClient.init(conf);
|
||||
}
|
||||
|
||||
if (conf.getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
|
||||
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
|
||||
timelineServiceEnabled = true;
|
||||
timelineClient = TimelineClient.createTimelineClient();
|
||||
timelineClient.init(conf);
|
||||
timelineService = TimelineUtils.buildTimelineTokenService(conf);
|
||||
}
|
||||
super.serviceInit(conf);
|
||||
}
|
||||
|
||||
|
@ -141,6 +161,9 @@ public class YarnClientImpl extends YarnClient {
|
|||
if (historyServiceEnabled) {
|
||||
historyClient.start();
|
||||
}
|
||||
if (timelineServiceEnabled) {
|
||||
timelineClient.start();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new YarnRuntimeException(e);
|
||||
}
|
||||
|
@ -155,6 +178,9 @@ public class YarnClientImpl extends YarnClient {
|
|||
if (historyServiceEnabled) {
|
||||
historyClient.stop();
|
||||
}
|
||||
if (timelineServiceEnabled) {
|
||||
timelineClient.stop();
|
||||
}
|
||||
super.serviceStop();
|
||||
}
|
||||
|
||||
|
@ -189,6 +215,12 @@ public class YarnClientImpl extends YarnClient {
|
|||
Records.newRecord(SubmitApplicationRequest.class);
|
||||
request.setApplicationSubmissionContext(appContext);
|
||||
|
||||
// Automatically add the timeline DT into the CLC
|
||||
// Only when the security and the timeline service are both enabled
|
||||
if (isSecurityEnabled() && timelineServiceEnabled) {
|
||||
addTimelineDelegationToken(appContext.getAMContainerSpec());
|
||||
}
|
||||
|
||||
//TODO: YARN-1763:Handle RM failovers during the submitApplication call.
|
||||
rmClient.submitApplication(request);
|
||||
|
||||
|
@ -238,6 +270,48 @@ public class YarnClientImpl extends YarnClient {
|
|||
return applicationId;
|
||||
}
|
||||
|
||||
private void addTimelineDelegationToken(
|
||||
ContainerLaunchContext clc) throws YarnException, IOException {
|
||||
org.apache.hadoop.security.token.Token<TimelineDelegationTokenIdentifier> timelineDelegationToken =
|
||||
timelineClient.getDelegationToken(
|
||||
UserGroupInformation.getCurrentUser().getUserName());
|
||||
if (timelineDelegationToken == null) {
|
||||
return;
|
||||
}
|
||||
Credentials credentials = new Credentials();
|
||||
DataInputByteBuffer dibb = new DataInputByteBuffer();
|
||||
ByteBuffer tokens = clc.getTokens();
|
||||
if (tokens != null) {
|
||||
dibb.reset(tokens);
|
||||
credentials.readTokenStorageStream(dibb);
|
||||
tokens.rewind();
|
||||
}
|
||||
// If the timeline delegation token is already in the CLC, no need to add
|
||||
// one more
|
||||
for (org.apache.hadoop.security.token.Token<? extends TokenIdentifier> token : credentials
|
||||
.getAllTokens()) {
|
||||
TokenIdentifier tokenIdentifier = token.decodeIdentifier();
|
||||
if (tokenIdentifier instanceof TimelineDelegationTokenIdentifier) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
credentials.addToken(timelineService, timelineDelegationToken);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Add timline delegation token into credentials: "
|
||||
+ timelineDelegationToken);
|
||||
}
|
||||
DataOutputBuffer dob = new DataOutputBuffer();
|
||||
credentials.writeTokenStorageToStream(dob);
|
||||
tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
|
||||
clc.setTokens(tokens);
|
||||
}
|
||||
|
||||
@Private
|
||||
@VisibleForTesting
|
||||
protected boolean isSecurityEnabled() {
|
||||
return UserGroupInformation.isSecurityEnabled();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void killApplication(ApplicationId applicationId)
|
||||
throws YarnException, IOException {
|
||||
|
|
|
@ -25,19 +25,26 @@ import static org.mockito.Mockito.verify;
|
|||
import static org.mockito.Mockito.when;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.security.PrivilegedExceptionAction;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.EnumSet;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.junit.Assert;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.DataInputByteBuffer;
|
||||
import org.apache.hadoop.io.DataOutputBuffer;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
|
||||
|
@ -69,19 +76,23 @@ import org.apache.hadoop.yarn.api.records.Priority;
|
|||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.api.records.YarnApplicationAttemptState;
|
||||
import org.apache.hadoop.yarn.api.records.YarnApplicationState;
|
||||
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
||||
import org.apache.hadoop.yarn.client.api.YarnClient;
|
||||
import org.apache.hadoop.yarn.client.api.YarnClientApplication;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.ApplicationIdNotProvidedException;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.server.MiniYARNCluster;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
|
||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
|
||||
import org.apache.hadoop.yarn.util.Records;
|
||||
import org.apache.hadoop.yarn.util.timeline.TimelineUtils;
|
||||
import org.apache.log4j.Level;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Test;
|
||||
|
||||
public class TestYarnClient {
|
||||
|
@ -725,4 +736,80 @@ public class TestYarnClient {
|
|||
IOUtils.closeQuietly(client);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testAutomaticTimelineDelegationTokenLoading()
|
||||
throws Exception {
|
||||
Configuration conf = new YarnConfiguration();
|
||||
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||
SecurityUtil.setAuthenticationMethod(AuthenticationMethod.KERBEROS, conf);
|
||||
final Token<TimelineDelegationTokenIdentifier> dToken =
|
||||
new Token<TimelineDelegationTokenIdentifier>();
|
||||
// crate a mock client
|
||||
YarnClientImpl client = new YarnClientImpl() {
|
||||
@Override
|
||||
protected void serviceInit(Configuration conf) throws Exception {
|
||||
if (getConfig().getBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED,
|
||||
YarnConfiguration.DEFAULT_TIMELINE_SERVICE_ENABLED)) {
|
||||
timelineServiceEnabled = true;
|
||||
timelineClient = mock(TimelineClient.class);
|
||||
when(timelineClient.getDelegationToken(any(String.class)))
|
||||
.thenReturn(dToken);
|
||||
timelineClient.init(getConfig());
|
||||
timelineService = TimelineUtils.buildTimelineTokenService(getConfig());
|
||||
}
|
||||
this.setConfig(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStart() throws Exception {
|
||||
rmClient = mock(ApplicationClientProtocol.class);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStop() throws Exception {
|
||||
}
|
||||
|
||||
@Override
|
||||
public ApplicationReport getApplicationReport(ApplicationId appId) {
|
||||
ApplicationReport report = mock(ApplicationReport.class);
|
||||
when(report.getYarnApplicationState())
|
||||
.thenReturn(YarnApplicationState.SUBMITTED);
|
||||
return report;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isSecurityEnabled() {
|
||||
return true;
|
||||
}
|
||||
};
|
||||
client.init(conf);
|
||||
client.start();
|
||||
ApplicationSubmissionContext context =
|
||||
mock(ApplicationSubmissionContext.class);
|
||||
ApplicationId applicationId = ApplicationId.newInstance(0, 1);
|
||||
when(context.getApplicationId()).thenReturn(applicationId);
|
||||
DataOutputBuffer dob = new DataOutputBuffer();
|
||||
Credentials credentials = new Credentials();
|
||||
credentials.writeTokenStorageToStream(dob);
|
||||
ByteBuffer tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
|
||||
ContainerLaunchContext clc = ContainerLaunchContext.newInstance(
|
||||
null, null, null, null, tokens, null);
|
||||
when(context.getAMContainerSpec()).thenReturn(clc);
|
||||
client.submitApplication(context);
|
||||
// Check whether token is added or not
|
||||
credentials = new Credentials();
|
||||
DataInputByteBuffer dibb = new DataInputByteBuffer();
|
||||
tokens = clc.getTokens();
|
||||
if (tokens != null) {
|
||||
dibb.reset(tokens);
|
||||
credentials.readTokenStorageStream(dibb);
|
||||
tokens.rewind();
|
||||
}
|
||||
Collection<Token<? extends TokenIdentifier>> dTokens =
|
||||
credentials.getAllTokens();
|
||||
Assert.assertEquals(1, dTokens.size());
|
||||
Assert.assertEquals(dToken, dTokens.iterator().next());
|
||||
client.stop();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -72,7 +72,19 @@
|
|||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-mapper-asl</artifactId>
|
||||
<artifactId>jackson-core-asl</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-mapper-asl</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-jaxrs</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.codehaus.jackson</groupId>
|
||||
<artifactId>jackson-xc</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue