[
https://issues.apache.org/jira/browse/YARN-2090?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanelfocusedCommentId=14007603#comment-14007603
]
Victor Kim commented on YARN-2090:
--
I figured out when\where it happens.
If you use kerberized user impersonation when submitting MR job, we actually
submit with user (lets say john) via yarn (or whatever kerberos principal,
whose user also presented on the box). During the shuffle phase in public void
messageReceived(ChannelHandlerContext ctx, MessageEvent evt) ShuffleHandler
tries to read\write the map output file as john who ran the MR job. This is not
going to work in case of user impersonation, when john is not presented on
local box (e.g. he comes from ActiveDirectory), because the file from local FS
cannot be read.
The fix is to use the JVM process owner instead of the user:
System.getProperty(user.name) in two methods: populateHeaders, sendMapOutput
{code:title=ShuffleHandler.java|borderStyle=solid}
protected void populateHeaders(ListString mapIds, String outputBaseStr,
String user, int reduce, HttpRequest
request, HttpResponse response,
boolean keepAliveParam, MapString,
MapOutputInfo mapOutputInfoMap)
throws IOException {
{
// Some code here..
String processOwner = System.getProperty(user.name);
MapOutputInfo outputInfo = getMapOutputInfo(base, mapId, reduce, processOwner);
// Some code here..
}
{code}
{code:title=ShuffleHandler.java|borderStyle=solid}
protected ChannelFuture sendMapOutput(ChannelHandlerContext ctx, Channel ch,
String user, String mapId, int
reduce, MapOutputInfo mapOutputInfo)
throws IOException {
// Some code here..
String processOwner = System.getProperty(user.name);
spill = SecureIOUtils.openForRandomRead(spillfile, r, processOwner, null);
// Some code here..
}
{code}
If Kerberos Authentication is enabled, MapReduce job is failing on reducer
phase
Key: YARN-2090
URL: https://issues.apache.org/jira/browse/YARN-2090
Project: Hadoop YARN
Issue Type: Bug
Components: applications, nodemanager
Affects Versions: 2.4.0
Environment: hadoop: 2.4.0.2.1.2.0
Reporter: Victor Kim
Priority: Critical
I have 3-node cluster configuration: 1 ResourceManager and 3 NodeManagers,
Kerberos is enabled, have hdfs, yarn, mapred principals\keytabs.
ResourceManager and NodeManager are ran under yarn user, using yarn Kerberos
principal.
Use case 1: WordCount, submit job using yarn UGI (i.e. superuser, the one
having Kerberos principal on all boxes). Result: job successfully completed.
Use case 2: WordCount, submit job using LDAP user impersonation via yarn UGI.
Result: Map tasks are completed SUCCESSfully, Reduce task fails with
ShuffleError Caused by: java.io.IOException: Exceeded
MAX_FAILED_UNIQUE_FETCHES (see the stack trace below).
The use case with user impersonation used to work on earlier versions,
without YARN (with JTTT).
I found similar issue with Kerberos AUTH involved here:
https://groups.google.com/forum/#!topic/nosql-databases/tGDqs75ACqQ
And here https://issues.apache.org/jira/browse/MAPREDUCE-4030 it's marked as
resolved, which is not the case when Kerberos Authentication is enabled.
The exception trace from YarnChild JVM:
2014-05-21 12:49:35,687 FATAL [fetcher#3]
org.apache.hadoop.mapreduce.task.reduce.ShuffleSchedulerImpl: Shuffle failed
with too many fetch failures and insufficient progress!
2014-05-21 12:49:35,688 WARN [main] org.apache.hadoop.mapred.YarnChild:
Exception running child :
org.apache.hadoop.mapreduce.task.reduce.Shuffle$ShuffleError: error in
shuffle in fetcher#3
at
org.apache.hadoop.mapreduce.task.reduce.Shuffle.run(Shuffle.java:134)
at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:376)
at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:167)
at java.security.AccessController.doPrivileged(Native Method)
at javax.security.auth.Subject.doAs(Subject.java:416)
at
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1557)
at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:162)
Caused by: java.io.IOException: Exceeded MAX_FAILED_UNIQUE_FETCHES;
bailing-out.
at
org.apache.hadoop.mapreduce.task.reduce.ShuffleSchedulerImpl.checkReducerHealth(ShuffleSchedulerImpl.java:323)
at
org.apache.hadoop.mapreduce.task.reduce.ShuffleSchedulerImpl.copyFailed(ShuffleSchedulerImpl.java:245)
at