HADOOP-13817. Add a finite shell command timeout to ShellBasedUnixGroupsMapping. (harsh)

(cherry picked from commit e8694deb6a)
This commit is contained in:
Harsh J 2016-11-14 15:59:58 +05:30
parent b5a8c49cf3
commit c097d0343c
6 changed files with 277 additions and 38 deletions

View File

@ -526,6 +526,21 @@ public class CommonConfigurationKeysPublic {
* <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
* core-default.xml</a>
*/
public static final String HADOOP_SECURITY_GROUP_SHELL_COMMAND_TIMEOUT_SECS =
"hadoop.security.groups.shell.command.timeout";
/**
* @see
* <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
* core-default.xml</a>
*/
public static final long
HADOOP_SECURITY_GROUP_SHELL_COMMAND_TIMEOUT_SECS_DEFAULT =
0L;
/**
* @see
* <a href="{@docRoot}/../hadoop-project-dist/hadoop-common/core-default.xml">
* core-default.xml</a>
*/
public static final String HADOOP_SECURITY_AUTHENTICATION =
"hadoop.security.authentication";
/**

View File

@ -18,17 +18,25 @@
package org.apache.hadoop.security;
import java.io.IOException;
import java.util.Arrays;
import java.util.LinkedList;
import java.util.List;
import java.util.StringTokenizer;
import java.util.concurrent.TimeUnit;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Joiner;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.conf.Configured;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
* A simple shell-based implementation of {@link GroupMappingServiceProvider}
@ -37,11 +45,28 @@ import org.apache.hadoop.util.Shell.ShellCommandExecutor;
*/
@InterfaceAudience.LimitedPrivate({"HDFS", "MapReduce"})
@InterfaceStability.Evolving
public class ShellBasedUnixGroupsMapping
public class ShellBasedUnixGroupsMapping extends Configured
implements GroupMappingServiceProvider {
private static final Log LOG =
LogFactory.getLog(ShellBasedUnixGroupsMapping.class);
@VisibleForTesting
protected static final Logger LOG =
LoggerFactory.getLogger(ShellBasedUnixGroupsMapping.class);
private long timeout = 0L;
private static final List<String> EMPTY_GROUPS = new LinkedList<>();
@Override
public void setConf(Configuration conf) {
super.setConf(conf);
if (conf != null) {
timeout = conf.getTimeDuration(
CommonConfigurationKeys.
HADOOP_SECURITY_GROUP_SHELL_COMMAND_TIMEOUT_SECS,
CommonConfigurationKeys.
HADOOP_SECURITY_GROUP_SHELL_COMMAND_TIMEOUT_SECS_DEFAULT,
TimeUnit.SECONDS);
}
}
@SuppressWarnings("serial")
private static class PartialGroupNameException extends IOException {
@ -98,7 +123,17 @@ public class ShellBasedUnixGroupsMapping
*/
protected ShellCommandExecutor createGroupExecutor(String userName) {
return new ShellCommandExecutor(
Shell.getGroupsForUserCommand(userName), null, null, 0L);
getGroupsForUserCommand(userName), null, null, timeout);
}
/**
* Returns just the shell command to be used to fetch a user's groups list.
* This is mainly separate to make some tests easier.
* @param userName The username that needs to be passed into the command built
* @return An appropriate shell command with arguments
*/
protected String[] getGroupsForUserCommand(String userName) {
return Shell.getGroupsForUserCommand(userName);
}
/**
@ -109,7 +144,17 @@ public class ShellBasedUnixGroupsMapping
*/
protected ShellCommandExecutor createGroupIDExecutor(String userName) {
return new ShellCommandExecutor(
Shell.getGroupsIDForUserCommand(userName), null, null, 0L);
getGroupsIDForUserCommand(userName), null, null, timeout);
}
/**
* Returns just the shell command to be used to fetch a user's group IDs list.
* This is mainly separate to make some tests easier.
* @param userName The username that needs to be passed into the command built
* @return An appropriate shell command with arguments
*/
protected String[] getGroupsIDForUserCommand(String userName) {
return Shell.getGroupsIDForUserCommand(userName);
}
/**
@ -133,8 +178,26 @@ public class ShellBasedUnixGroupsMapping
groups = resolvePartialGroupNames(user, e.getMessage(),
executor.getOutput());
} catch (PartialGroupNameException pge) {
LOG.warn("unable to return groups for user " + user, pge);
return new LinkedList<>();
LOG.warn("unable to return groups for user {}", user, pge);
return EMPTY_GROUPS;
}
} catch (IOException ioe) {
// If its a shell executor timeout, indicate so in the message
// but treat the result as empty instead of throwing it up,
// similar to how partial resolution failures are handled above
if (executor.isTimedOut()) {
LOG.warn(
"Unable to return groups for user '{}' as shell group lookup " +
"command '{}' ran longer than the configured timeout limit of " +
"{} seconds.",
user,
Joiner.on(' ').join(executor.getExecString()),
timeout
);
return EMPTY_GROUPS;
} else {
// If its not an executor timeout, we should let the caller handle it
throw ioe;
}
}
@ -196,7 +259,7 @@ public class ShellBasedUnixGroupsMapping
* @param errMessage error message from the shell command
* @param groupNames the incomplete list of group names
* @return a list of resolved group names
* @throws PartialGroupNameException
* @throws PartialGroupNameException if the resolution fails or times out
*/
private List<String> resolvePartialGroupNames(String userName,
String errMessage, String groupNames) throws PartialGroupNameException {
@ -212,21 +275,29 @@ public class ShellBasedUnixGroupsMapping
throw new PartialGroupNameException("The user name '" + userName
+ "' is not found. " + errMessage);
} else {
LOG.warn("Some group names for '" + userName + "' are not resolvable. "
+ errMessage);
LOG.warn("Some group names for '{}' are not resolvable. {}",
userName, errMessage);
// attempt to partially resolve group names
ShellCommandExecutor partialResolver = createGroupIDExecutor(userName);
try {
ShellCommandExecutor exec2 = createGroupIDExecutor(userName);
exec2.execute();
return parsePartialGroupNames(groupNames, exec2.getOutput());
partialResolver.execute();
return parsePartialGroupNames(
groupNames, partialResolver.getOutput());
} catch (ExitCodeException ece) {
// If exception is thrown trying to get group id list,
// something is terribly wrong, so give up.
throw new PartialGroupNameException("failed to get group id list for " +
"user '" + userName + "'", ece);
throw new PartialGroupNameException(
"failed to get group id list for user '" + userName + "'", ece);
} catch (IOException ioe) {
throw new PartialGroupNameException("can't execute the shell command to"
+ " get the list of group id for user '" + userName + "'", ioe);
String message =
"Can't execute the shell command to " +
"get the list of group id for user '" + userName + "'";
if (partialResolver.isTimedOut()) {
message +=
" because of the command taking longer than " +
"the configured timeout: " + timeout + " seconds";
}
throw new PartialGroupNameException(message, ioe);
}
}
}
@ -237,7 +308,8 @@ public class ShellBasedUnixGroupsMapping
* @param groupNames a string representing the user's group names
* @return a linked list of group names
*/
private List<String> resolveFullGroupNames(String groupNames) {
@VisibleForTesting
protected List<String> resolveFullGroupNames(String groupNames) {
StringTokenizer tokenizer =
new StringTokenizer(groupNames, Shell.TOKEN_SEPARATOR_REGEX);
List<String> groups = new LinkedList<String>();

View File

@ -959,7 +959,15 @@ public abstract class Shell {
line = errReader.readLine();
}
} catch(IOException ioe) {
LOG.warn("Error reading the error stream", ioe);
// Its normal to observe a "Stream closed" I/O error on
// command timeouts destroying the underlying process
// so only log a WARN if the command didn't time out
if (!isTimedOut()) {
LOG.warn("Error reading the error stream", ioe);
} else {
LOG.debug("Error reading the error stream due to shell "
+ "command timeout", ioe);
}
}
}
};
@ -1184,6 +1192,15 @@ public abstract class Shell {
this.inheritParentEnv = inheritParentEnv;
}
/**
* Returns the timeout value set for the executor's sub-commands.
* @return The timeout value in seconds
*/
@VisibleForTesting
public long getTimeoutInterval() {
return timeOutInterval;
}
/**
* Execute the shell command.
* @throws IOException if the command fails, or if the command is

View File

@ -196,6 +196,19 @@
</description>
</property>
<property>
<name>hadoop.security.groups.shell.command.timeout</name>
<value>0s</value>
<description>
Used by the ShellBasedUnixGroupsMapping class, this property controls how
long to wait for the underlying shell command that is run to fetch groups.
Expressed in seconds (e.g. 10s, 1m, etc.), if the running command takes
longer than the value configured, the command is aborted and the groups
resolver would return a result of no groups found. A value of 0s (default)
would mean an infinite wait (i.e. wait until the command exits on its own).
</description>
</property>
<property>
<name>hadoop.security.group.mapping.ldap.connection.timeout.ms</name>
<value>60000</value>

View File

@ -50,7 +50,7 @@ import org.apache.hadoop.security.ShellBasedUnixGroupsMapping;
public class TestGroupsCaching {
public static final Log LOG = LogFactory.getLog(TestGroupsCaching.class);
public static final Log TESTLOG = LogFactory.getLog(TestGroupsCaching.class);
private static String[] myGroups = {"grp1", "grp2"};
private Configuration conf;
@ -76,7 +76,7 @@ public class TestGroupsCaching {
@Override
public List<String> getGroups(String user) throws IOException {
LOG.info("Getting groups for " + user);
TESTLOG.info("Getting groups for " + user);
delayIfNecessary();
requestCount++;
@ -115,18 +115,18 @@ public class TestGroupsCaching {
@Override
public void cacheGroupsRefresh() throws IOException {
LOG.info("Cache is being refreshed.");
TESTLOG.info("Cache is being refreshed.");
clearBlackList();
return;
}
public static void clearBlackList() throws IOException {
LOG.info("Clearing the blacklist");
TESTLOG.info("Clearing the blacklist");
blackList.clear();
}
public static void clearAll() throws IOException {
LOG.info("Resetting FakeGroupMapping");
TESTLOG.info("Resetting FakeGroupMapping");
blackList.clear();
allGroups.clear();
requestCount = 0;
@ -137,12 +137,12 @@ public class TestGroupsCaching {
@Override
public void cacheGroupsAdd(List<String> groups) throws IOException {
LOG.info("Adding " + groups + " to groups.");
TESTLOG.info("Adding " + groups + " to groups.");
allGroups.addAll(groups);
}
public static void addToBlackList(String user) throws IOException {
LOG.info("Adding " + user + " to the blacklist");
TESTLOG.info("Adding " + user + " to the blacklist");
blackList.add(user);
}
@ -226,11 +226,12 @@ public class TestGroupsCaching {
// ask for a negative entry
try {
LOG.error("We are not supposed to get here." + groups.getGroups("user1").toString());
TESTLOG.error("We are not supposed to get here."
+ groups.getGroups("user1").toString());
fail();
} catch (IOException ioe) {
if(!ioe.getMessage().startsWith("No groups found")) {
LOG.error("Got unexpected exception: " + ioe.getMessage());
TESTLOG.error("Got unexpected exception: " + ioe.getMessage());
fail();
}
}

View File

@ -22,9 +22,15 @@ import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.test.GenericTestUtils;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.Shell.ExitCodeException;
import org.apache.hadoop.util.Shell.ShellCommandExecutor;
import org.junit.Test;
import static org.junit.Assert.*;
import static org.mockito.Mockito.doNothing;
import static org.mockito.Mockito.doThrow;
@ -32,9 +38,13 @@ import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
public class TestShellBasedUnixGroupsMapping {
private static final Log LOG =
private static final Log TESTLOG =
LogFactory.getLog(TestShellBasedUnixGroupsMapping.class);
private final GenericTestUtils.LogCapturer shellMappingLog =
GenericTestUtils.LogCapturer.captureLogs(
ShellBasedUnixGroupsMapping.LOG);
private class TestGroupUserNotExist
extends ShellBasedUnixGroupsMapping {
/**
@ -55,7 +65,7 @@ public class TestShellBasedUnixGroupsMapping {
when(executor.getOutput()).thenReturn("");
} catch (IOException e) {
LOG.warn(e.getMessage());
TESTLOG.warn(e.getMessage());
}
return executor;
}
@ -90,7 +100,7 @@ public class TestShellBasedUnixGroupsMapping {
when(executor.getOutput()).thenReturn("9999\n9999 abc def");
} catch (IOException e) {
LOG.warn(e.getMessage());
TESTLOG.warn(e.getMessage());
}
return executor;
}
@ -133,7 +143,7 @@ public class TestShellBasedUnixGroupsMapping {
doNothing().when(executor).execute();
when(executor.getOutput()).thenReturn("23\n23 groupname zzz");
} catch (IOException e) {
LOG.warn(e.getMessage());
TESTLOG.warn(e.getMessage());
}
return executor;
}
@ -146,7 +156,7 @@ public class TestShellBasedUnixGroupsMapping {
doNothing().when(executor).execute();
when(executor.getOutput()).thenReturn("111\n111 112 113");
} catch (IOException e) {
LOG.warn(e.getMessage());
TESTLOG.warn(e.getMessage());
}
return executor;
}
@ -179,7 +189,7 @@ public class TestShellBasedUnixGroupsMapping {
doNothing().when(executor).execute();
when(executor.getOutput()).thenReturn("abc\ndef abc hij");
} catch (IOException e) {
LOG.warn(e.getMessage());
TESTLOG.warn(e.getMessage());
}
return executor;
}
@ -192,7 +202,7 @@ public class TestShellBasedUnixGroupsMapping {
doNothing().when(executor).execute();
when(executor.getOutput()).thenReturn("1\n1 2 3");
} catch (IOException e) {
LOG.warn(e.getMessage());
TESTLOG.warn(e.getMessage());
}
return executor;
}
@ -208,6 +218,117 @@ public class TestShellBasedUnixGroupsMapping {
assertTrue(groups.contains("def"));
assertTrue(groups.contains("hij"));
}
private static class TestDelayedGroupCommand
extends ShellBasedUnixGroupsMapping {
private Long timeoutSecs = 2L;
TestDelayedGroupCommand() {
super();
}
@Override
protected String[] getGroupsForUserCommand(String userName) {
// Sleeps 2 seconds when executed and writes no output
if (Shell.WINDOWS) {
return new String[]{"timeout", timeoutSecs.toString()};
}
return new String[]{"sleep", timeoutSecs.toString()};
}
@Override
protected String[] getGroupsIDForUserCommand(String userName) {
return getGroupsForUserCommand(userName);
}
}
@Test(timeout=4000)
public void testFiniteGroupResolutionTime() throws Exception {
Configuration conf = new Configuration();
String userName = "foobarnonexistinguser";
String commandTimeoutMessage =
"ran longer than the configured timeout limit";
long testTimeout = 1L;
// Test a 1 second max-runtime timeout
conf.setLong(
CommonConfigurationKeys.
HADOOP_SECURITY_GROUP_SHELL_COMMAND_TIMEOUT_SECS,
testTimeout);
TestDelayedGroupCommand mapping =
ReflectionUtils.newInstance(TestDelayedGroupCommand.class, conf);
ShellCommandExecutor executor = mapping.createGroupExecutor(userName);
assertEquals(
"Expected the group names executor to carry the configured timeout",
testTimeout,
executor.getTimeoutInterval());
executor = mapping.createGroupIDExecutor(userName);
assertEquals(
"Expected the group ID executor to carry the configured timeout",
testTimeout,
executor.getTimeoutInterval());
assertEquals(
"Expected no groups to be returned given a shell command timeout",
0,
mapping.getGroups(userName).size());
assertTrue(
"Expected the logs to carry " +
"a message about command timeout but was: " +
shellMappingLog.getOutput(),
shellMappingLog.getOutput().contains(commandTimeoutMessage));
shellMappingLog.clearOutput();
// Test also the parent Groups framework for expected behaviour
conf.setClass(CommonConfigurationKeys.HADOOP_SECURITY_GROUP_MAPPING,
TestDelayedGroupCommand.class,
GroupMappingServiceProvider.class);
Groups groups = new Groups(conf);
try {
groups.getGroups(userName);
fail(
"The groups framework call should " +
"have failed with a command timeout");
} catch (IOException e) {
assertTrue(
"Expected the logs to carry " +
"a message about command timeout but was: " +
shellMappingLog.getOutput(),
shellMappingLog.getOutput().contains(commandTimeoutMessage));
}
shellMappingLog.clearOutput();
// Test the no-timeout (default) configuration
conf = new Configuration();
long defaultTimeout =
CommonConfigurationKeys.
HADOOP_SECURITY_GROUP_SHELL_COMMAND_TIMEOUT_SECS_DEFAULT;
mapping =
ReflectionUtils.newInstance(TestDelayedGroupCommand.class, conf);
executor = mapping.createGroupExecutor(userName);
assertEquals(
"Expected the group names executor to carry the default timeout",
defaultTimeout,
executor.getTimeoutInterval());
executor = mapping.createGroupIDExecutor(userName);
assertEquals(
"Expected the group ID executor to carry the default timeout",
defaultTimeout,
executor.getTimeoutInterval());
mapping.getGroups(userName);
assertFalse(
"Didn't expect a timeout of command in execution but logs carry it: " +
shellMappingLog.getOutput(),
shellMappingLog.getOutput().contains(commandTimeoutMessage));
}
}