You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "Sébastien BARNOUD (JIRA)" <ji...@apache.org> on 2019/04/29 10:03:00 UTC

[jira] [Commented] (SPARK-21827) Task fail due to executor exception when enable Sasl Encryption

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

Sébastien BARNOUD commented on SPARK-21827:
-------------------------------------------

Hi,

 

I was investigating timeout on HBase Client (version 1.1.2) on my Hadoop cluster with security enabled using hotspot jdk 1.8.0_92-b14.

I have found the following message in logs each time a get a timeout:

*sasl:1481  - DIGEST41:Unmatched MACs*

 

After a look at the code, if understand that the message is simply ignored if an invalid MAC is received. In my opinion, this is not a normal behavior. It allows at least an attacker to flood the connection.

 

But, in my case, there is no men in the middle, but a get this message. It looks like there is bug (probably a not thread safe method somewhere) in the MAC validation, leading to the message to be ignored, and to my HBase timeout.

In the same time, we have found some TEZ job stuck on our cluster since we have enabled security on shuffle (mapreduce, TEZ and Spark). In each hanged job, we could identify that the SSL handshake never finished:

 

"fetcher \{Map_4} #34" #78 daemon prio=5 os_prio=0 tid=0x00007fd86905d000 nid=0x13dad runnable [0x00007fd83beb6000]

   java.lang.Thread.State: RUNNABLE

       at java.net.SocketInputStream.socketRead0(Native Method)

       at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)

       at java.net.SocketInputStream.read(SocketInputStream.java:170)

       at java.net.SocketInputStream.read(SocketInputStream.java:141)

       at sun.security.ssl.InputRecord.readFully(InputRecord.java:465)

       at sun.security.ssl.InputRecord.read(InputRecord.java:503)

       at sun.security.ssl.SSLSocketImpl.readRecord(SSLSocketImpl.java:973)

       - locked <0x00000007b997a470> (a java.lang.Object)

       at sun.security.ssl.SSLSocketImpl.performInitialHandshake(SSLSocketImpl.java:1375)

       - locked <0x00000007b997a430> (a java.lang.Object)

       at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1403)

       at sun.security.ssl.SSLSocketImpl.startHandshake(SSLSocketImpl.java:1387)

       at sun.net.www.protocol.https.HttpsClient.afterConnect(HttpsClient.java:559)

       at sun.net.www.protocol.https.AbstractDelegateHttpsURLConnection.setNewClient(AbstractDelegateHttpsURLConnection.java:100)

       at sun.net.www.protocol.https.AbstractDelegateHttpsURLConnection.setNewClient(AbstractDelegateHttpsURLConnection.java:80)

       at sun.net.www.protocol.http.HttpURLConnection.writeRequests(HttpURLConnection.java:672)

       at sun.net.www.protocol.http.HttpURLConnection.getInputStream0(HttpURLConnection.java:1534)

       - locked <0x00000007b9979f10> (a sun.net.www.protocol.https.DelegateHttpsURLConnection)

       at sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1441)

       - locked <0x00000007b9979f10> (a sun.net.www.protocol.https.DelegateHttpsURLConnection)

       at sun.net.www.protocol.https.HttpsURLConnectionImpl.getInputStream(HttpsURLConnectionImpl.java:254)

       - locked <0x00000007b9979ea8> (a sun.net.www.protocol.https.HttpsURLConnectionImpl)

       at org.apache.tez.runtime.library.common.shuffle.HttpConnection.getInputStream(HttpConnection.java:253)

       at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.setupConnection(FetcherOrderedGrouped.java:356)

       at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.copyFromHost(FetcherOrderedGrouped.java:264)

       at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.fetchNext(FetcherOrderedGrouped.java:176)

       at org.apache.tez.runtime.library.common.shuffle.orderedgrouped.FetcherOrderedGrouped.run(FetcherOrderedGrouped.java:191)

 

Looking a TEZ source, shows that there are no timeout in the code leading to this infinite wait.

 

After some more investigation, I found:

-) https://issues.apache.org/jira/browse/SPARK-21827

-) [https://issues.cask.co/browse/CDAP-12737]

-) [https://bugster.forgerock.org/jira/browse/OPENDJ-4956]

 

It seems that this issue affects a lot of software, and ForgeRock seems to have identified the thread safety issue.

 

To summarize, there are 2 issues:
 # the message shouldn’t be ignored when the MAC is invalid, an exception should be throwed.
 # The thread safety issue should be investigated and corrected in the JDK, because relying on a synchronized method at the application layer is not viable. Typically, an application like Spark uses multiple SASL implementation and can’t synchronize all of them.

 

I sent this to [secalert_us@oracle.com|mailto:secalert_us@oracle.com] because IMO it's a JDK bug. 

Regards,

 

Sébastien BARNOUD

> Task fail due to executor exception when enable Sasl Encryption
> ---------------------------------------------------------------
>
>                 Key: SPARK-21827
>                 URL: https://issues.apache.org/jira/browse/SPARK-21827
>             Project: Spark
>          Issue Type: Bug
>          Components: Shuffle, Spark Core
>    Affects Versions: 1.6.1, 2.1.1, 2.2.0
>         Environment: OS: RedHat 7.1 64bit
>            Reporter: Yishan Jiang
>            Priority: Major
>
> We met authentication and Sasl encryption on many versions, just append 161 version like this:
> spark.local.dir /tmp/test-161
> spark.shuffle.service.enabled true
> *spark.authenticate true*
> *spark.authenticate.enableSaslEncryption true*
> *spark.network.sasl.serverAlwaysEncrypt true*
> spark.authenticate.secret e25d4369-bec3-4266-8fc5-fb6d4fcee66f
> spark.history.ui.port 18089
> spark.shuffle.service.port 7347
> spark.master.rest.port 6076
> spark.deploy.recoveryMode NONE
> spark.ssl.enabled true
> spark.executor.extraJavaOptions -Djava.security.egd=file:/dev/./urandom
> We run an Spark example and task fail with Exception messages:
> 17/08/22 03:56:52 INFO BlockManager: external shuffle service port = 7347
> 17/08/22 03:56:52 INFO BlockManagerMaster: Trying to register BlockManager
> 17/08/22 03:56:52 INFO sasl: DIGEST41:Unmatched MACs
> 17/08/22 03:56:52 WARN TransportChannelHandler: Exception in connection from cws57n6.ma.platformlab.ibm.com/172.29.8.66:49394
> java.lang.IllegalArgumentException: Frame length should be positive: -5594407078713290673       
>         at org.spark-project.guava.base.Preconditions.checkArgument(Preconditions.java:119)
>         at org.apache.spark.network.util.TransportFrameDecoder.decodeNext(TransportFrameDecoder.java:135)
>         at org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:82)
>         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>         at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>         at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>         at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:785)
> 17/08/22 03:56:52 ERROR TransportResponseHandler: Still have 1 requests outstanding when connection from cws57n6.ma.platformlab.ibm.com/172.29.8.66:49394 is closed
> 17/08/22 03:56:52 WARN NettyRpcEndpointRef: Error sending message [message = RegisterBlockManager(BlockManagerId(fe9d31da-f70c-40a2-9032-05a5af4ba4c5, cws58n1.ma.platformlab.ibm.com, 45852),2985295872,NettyRpcEn
> dpointRef(null))] in 1 attempts
> java.lang.IllegalArgumentException: Frame length should be positive: -5594407078713290673
>         at org.spark-project.guava.base.Preconditions.checkArgument(Preconditions.java:119)
>         at org.apache.spark.network.util.TransportFrameDecoder.decodeNext(TransportFrameDecoder.java:135)
>         at org.apache.spark.network.util.TransportFrameDecoder.channelRead(TransportFrameDecoder.java:82)
>         at io.netty.channel.AbstractChannelHandlerContext.invokeChannelRead(AbstractChannelHandlerContext.java:308)
>         at io.netty.channel.AbstractChannelHandlerContext.fireChannelRead(AbstractChannelHandlerContext.java:294)
>         at io.netty.channel.DefaultChannelPipeline.fireChannelRead(DefaultChannelPipeline.java:846)
>         at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:131)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:511)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:468)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:382)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:354)
>         at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:785)
> 17/08/22 03:56:55 ERROR TransportClient: Failed to send RPC 9091046580632843491 to cws57n6.ma.platformlab.ibm.com/172.29.8.66:49394: java.nio.channels.ClosedChannelException
> java.nio.channels.ClosedChannelException
> 17/08/22 03:56:55 WARN NettyRpcEndpointRef: Error sending message [message = RegisterBlockManager(BlockManagerId(fe9d31da-f70c-40a2-9032-05a5af4ba4c5, cws58n1.ma.platformlab.ibm.com, 45852),2985295872,NettyRpcEndpointRef(null))] in 2 attempts
> java.io.IOException: Failed to send RPC 9091046580632843491 to cws57n6.ma.platformlab.ibm.com/172.29.8.66:49394: java.nio.channels.ClosedChannelException
>         at org.apache.spark.network.client.TransportClient$3.operationComplete(TransportClient.java:239)
>         at org.apache.spark.network.client.TransportClient$3.operationComplete(TransportClient.java:226)
>         at io.netty.util.concurrent.DefaultPromise.notifyListener0(DefaultPromise.java:680)
>         at io.netty.util.concurrent.DefaultPromise.notifyListeners(DefaultPromise.java:567)
>         at io.netty.util.concurrent.DefaultPromise.tryFailure(DefaultPromise.java:424)
>         at io.netty.channel.AbstractChannel$AbstractUnsafe.safeSetFailure(AbstractChannel.java:801)
>         at io.netty.channel.AbstractChannel$AbstractUnsafe.write(AbstractChannel.java:699)
>         at io.netty.channel.DefaultChannelPipeline$HeadContext.write(DefaultChannelPipeline.java:1122)
>         at io.netty.channel.AbstractChannelHandlerContext.invokeWrite(AbstractChannelHandlerContext.java:633)
>         at io.netty.channel.AbstractChannelHandlerContext.access$1900(AbstractChannelHandlerContext.java:32)
>         at io.netty.channel.AbstractChannelHandlerContext$AbstractWriteTask.write(AbstractChannelHandlerContext.java:908)
>         at io.netty.channel.AbstractChannelHandlerContext$WriteAndFlushTask.write(AbstractChannelHandlerContext.java:960)
>         at io.netty.channel.AbstractChannelHandlerContext$AbstractWriteTask.run(AbstractChannelHandlerContext.java:893)
>         at io.netty.util.concurrent.SingleThreadEventExecutor.runAllTasks(SingleThreadEventExecutor.java:357)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:357)
>         at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:111)
>         at java.lang.Thread.run(Thread.java:785)
> Caused by: java.nio.channels.ClosedChannelException



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@spark.apache.org
For additional commands, e-mail: issues-help@spark.apache.org