[ https://issues.apache.org/jira/browse/YARN-2893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14337301#comment-14337301 ]
zhihai xu commented on YARN-2893: --------------------------------- I feel I know what cause this issue. This issue most likely in the following code: {code} Credentials credentials = new Credentials(); DataInputByteBuffer dibb = new DataInputByteBuffer(); if (container.getTokens() != null) { // TODO: Don't do this kind of checks everywhere. dibb.reset(container.getTokens()); credentials.readTokenStorageStream(dibb); } {code} we didn't rewind the token after credentials.readTokenStorageStream(dibb), I checked the code in DataInputByteBuffer. It will move the position of ByteBuffer (container.getTokens) in DataInputByteBuffer.read. (HeapByteBuffer is used for container.getTokens). {code} public int read(byte[] b, int off, int len) { if (bidx >= buffers.length) { return -1; } int cur = 0; do { int rem = Math.min(len, buffers[bidx].remaining()); buffers[bidx].get(b, off, rem); cur += rem; off += rem; len -= rem; } while (len > 0 && ++bidx < buffers.length); pos += cur; return cur; } {code} So If exception happen in AMLauncher.setupTokens before the ByteBuffer changed in container.setTokens. Then the position of ByteBuffer of Tokens will be at the end and we will see this issue next time when we retry. So it think it will be good to add container.getTokens().rewind() after credentials.readTokenStorageStream(dibb);. I will create a patch for this. > AMLaucher: sporadic job failures due to EOFException in readTokenStorageStream > ------------------------------------------------------------------------------ > > Key: YARN-2893 > URL: https://issues.apache.org/jira/browse/YARN-2893 > Project: Hadoop YARN > Issue Type: Bug > Components: resourcemanager > Affects Versions: 2.4.0 > Reporter: Gera Shegalov > Assignee: zhihai xu > > MapReduce jobs on our clusters experience sporadic failures due to corrupt > tokens in the AM launch context. -- This message was sent by Atlassian JIRA (v6.3.4#6332)