[ 
https://issues.apache.org/jira/browse/YARN-6013?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15840244#comment-15840244
 ] 

Steven Rand commented on YARN-6013:
-----------------------------------

The problem is that {{Client$IpcStreams#readResponse}} is trying to get the 
length of the input stream _after_ it's been unwrapped. Before being unwrapped, 
the input stream does in fact contain the length of the RPC message in its 
first four bytes, as we see in SaslRpcClient#readNextRpcPacket(): 
https://github.com/apache/hadoop/blob/branch-2.8.0/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/security/SaslRpcClient.java#L589.

However, the stream that we pass to {{saslClient.unwrap}} can't contain those 
four bytes. From the 
[javadoc|https://docs.oracle.com/javase/8/docs/api/javax/security/sasl/SaslClient.html#unwrap-byte:A-int-int-]:
 "incoming is the contents of the SASL buffer as defined in RFC 2222 without 
the leading four octet field that represents the length." And indeed, the 
length of the incoming RPC is 3586 in the case I'm looking at, but then 
{{token}} is only 3570 bytes when we pass it to {{saslClient.unwrap}}, so the 
first four bytes are definitely gone. (I'm not sure what the other 12 missing 
bytes are though.)

Also, from what I can tell of the [SASL 
code|http://grepcode.com/file/repository.grepcode.com/java/root/jdk/openjdk/8-b132/com/sun/security/sasl/digest/DigestMD5Base.java],
 the {{unwrap}} implementation won't put the 4-byte header back on.

So it doesn't make sense to call {{int length = in.readInt();}} on an unwrapped 
stream, as the first four bytes of that do not contain its length. I can submit 
a patch for this.


> ApplicationMasterProtocolPBClientImpl.allocate fails with EOFException when 
> RPC privacy is enabled
> --------------------------------------------------------------------------------------------------
>
>                 Key: YARN-6013
>                 URL: https://issues.apache.org/jira/browse/YARN-6013
>             Project: Hadoop YARN
>          Issue Type: Bug
>          Components: client, yarn
>    Affects Versions: 2.8.0
>            Reporter: Steven Rand
>            Priority: Critical
>         Attachments: yarn-rm-log.txt
>
>
> When privacy is enabled for RPC (hadoop.rpc.protection = privacy), 
> {{ApplicationMasterProtocolPBClientImpl.allocate}} sometimes (but not always) 
> fails with an EOFException. I've reproduced this with Spark 2.0.2 built 
> against latest branch-2.8 and with a simple distcp job on latest branch-2.8.
> Steps to reproduce using distcp:
> 1. Set hadoop.rpc.protection equal to privacy
> 2. Write data to HDFS. I did this with Spark as follows: 
> {code}
> sc.parallelize(1 to (5*1024*1024)).map(k => Seq(k, 
> org.apache.commons.lang.RandomStringUtils.random(1024, 
> "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWxyZ0123456789")).mkString("|")).toDF().repartition(100).write.parquet("hdfs:///tmp/testData")
> {code}
> 3. Attempt to distcp that data to another location in HDFS. For example:
> {code}
> hadoop distcp -Dmapreduce.framework.name=yarn hdfs:///tmp/testData 
> hdfs:///tmp/testDataCopy
> {code}
> I observed this error in the ApplicationMaster's syslog:
> {code}
> 2016-12-19 19:13:50,097 INFO [eventHandlingThread] 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler: Event Writer 
> setup for JobId: job_1482189777425_0004, File: 
> hdfs://<namenode_host>:8020/tmp/hadoop-yarn/staging/<hdfs_user>/.staging/job_1482189777425_0004/job_1482189777425_0004_1.jhist
> 2016-12-19 19:13:51,004 INFO [RMCommunicator Allocator] 
> org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator: Before 
> Scheduling: PendingReds:0 ScheduledMaps:4 ScheduledReds:0 AssignedMaps:0 
> AssignedReds:0 CompletedMaps:0 CompletedReds:0 ContAlloc:0 ContRel:0 
> HostLocal:0 RackLocal:0
> 2016-12-19 19:13:51,031 INFO [RMCommunicator Allocator] 
> org.apache.hadoop.mapreduce.v2.app.rm.RMContainerRequestor: getResources() 
> for application_1482189777425_0004: ask=1 release= 0 newContainers=0 
> finishedContainers=0 resourcelimit=<memory:22528, vCores:23> knownNMs=3
> 2016-12-19 19:13:52,043 INFO [RMCommunicator Allocator] 
> org.apache.hadoop.io.retry.RetryInvocationHandler: Exception while invoking 
> ApplicationMasterProtocolPBClientImpl.allocate over null. Retrying after 
> sleeping for 30000ms.
> java.io.EOFException: End of File Exception between local host is: 
> "<application_master_host>/<ip_addr>"; destination host is: "<rm_host>":8030; 
> : java.io.EOFException; For more details see:  
> http://wiki.apache.org/hadoop/EOFException
>       at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
>       at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
>       at java.lang.reflect.Constructor.newInstance(Constructor.java:422)
>       at org.apache.hadoop.net.NetUtils.wrapWithMessage(NetUtils.java:801)
>       at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:765)
>       at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1486)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1428)
>       at org.apache.hadoop.ipc.Client.call(Client.java:1338)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:227)
>       at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:116)
>       at com.sun.proxy.$Proxy80.allocate(Unknown Source)
>       at 
> org.apache.hadoop.yarn.api.impl.pb.client.ApplicationMasterProtocolPBClientImpl.allocate(ApplicationMasterProtocolPBClientImpl.java:77)
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.lang.reflect.Method.invoke(Method.java:497)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:398)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:163)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:155)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
>       at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:335)
>       at com.sun.proxy.$Proxy81.allocate(Unknown Source)
>       at 
> org.apache.hadoop.mapreduce.v2.app.rm.RMContainerRequestor.makeRemoteRequest(RMContainerRequestor.java:204)
>       at 
> org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator.getResources(RMContainerAllocator.java:735)
>       at 
> org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator.heartbeat(RMContainerAllocator.java:269)
>       at 
> org.apache.hadoop.mapreduce.v2.app.rm.RMCommunicator$AllocatorRunnable.run(RMCommunicator.java:281)
>       at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.EOFException
>       at java.io.DataInputStream.readInt(DataInputStream.java:392)
>       at 
> org.apache.hadoop.ipc.Client$IpcStreams.readResponse(Client.java:1785)
>       at 
> org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1156)
>       at org.apache.hadoop.ipc.Client$Connection.run(Client.java:1053)
> {code}
> Marking as "critical" since this blocks YARN users from encrypting RPC in 
> their Hadoop clusters.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org

Reply via email to