You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@spark.apache.org by "zuotingbing (JIRA)" <ji...@apache.org> on 2019/05/08 02:22:00 UTC

[jira] [Comment Edited] (SPARK-23191) Workers registration failes in case of network drop

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

zuotingbing edited comment on SPARK-23191 at 5/8/19 2:21 AM:
-------------------------------------------------------------

See these detail logs, master changed from vmax18 to vmax17.

In master vmax18,  worker be removed because got no heartbeat but soon got heartbeat and asking to re-register with master vmax18.

In the same time, worker has bean registered with master vmax17 when master vmax17 got leadership.

So Worker registration failed: Duplicate worker ID.

 

spark-mr-master-vmax18.log:
{code:java}
2019-03-15 20:22:09,441 INFO ZooKeeperLeaderElectionAgent: We have lost leadership
2019-03-15 20:22:14,544 WARN Master: Removing worker-20190218183101-vmax18-33129 because we got no heartbeat in 60 seconds
2019-03-15 20:22:14,544 INFO Master: Removing worker worker-20190218183101-vmax18-33129 on vmax18:33129
2019-03-15 20:22:14,864 WARN Master: Got heartbeat from unregistered worker worker-20190218183101-vmax18-33129. Asking it to re-register.
2019-03-15 20:22:14,975 ERROR Master: Leadership has been revoked -- master shutting down.
{code}
 

spark-mr-master-vmax17.log:
{code:java}
2019-03-15 20:22:14,870 INFO Master: Registering worker vmax18:33129 with 21 cores, 125.0 GB RAM
2019-03-15 20:22:15,261 INFO Master: vmax18:33129 got disassociated, removing it.
2019-03-15 20:22:15,263 INFO Master: Removing worker worker-20190218183101-vmax18-33129 on vmax18:33129
2019-03-15 20:22:15,311 ERROR Inbox: Ignoring error
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /spark/master_status/worker_worker-20190218183101-vmax18-33129
{code}
 

spark-mr-worker-vmax18.log:
{code:java}
2019-03-15 20:22:10,474 INFO Worker: Master has changed, new master is at spark://vmax17:7077
2019-03-15 20:22:14,862 INFO Worker: Master with url spark://vmax18:7077 requested this worker to reconnect.
2019-03-15 20:22:14,865 INFO Worker: Not spawning another attempt to register with the master, since there is an attempt scheduled already.
2019-03-15 20:22:14,879 ERROR Worker: Worker registration failed: Duplicate worker ID
2019-03-15 20:22:14,895 INFO ExecutorRunner: Killing process!
2019-03-15 20:22:14,896 INFO ShutdownHookManager: Shutdown hook called{code}
 

PS, this will result another issue: The leader will always in COMPLETING_RECOVERY state.

worker-vmax18 shut down cause duplicate worker ID,and clear the worker's node on persist Engine(we use zookeeper). Then the new leader(master-vmax17) find the worker died and trying to remove it ,and try to clear the node on zookeeper,but the node has been removed yet during worker-vmax18 shut down ,so {color:#ff0000}*an exception was thrown in function completeRecovery()* *. Then the leader will always in COMPLETING_RECOVERY state.*{color}

 


was (Author: zuo.tingbing9):
See these detail logs, master changed from vmax18 to vmax17.

In master vmax18,  worker be removed because got no heartbeat but soon got heartbeat and asking to re-register with master vmax18.

In the same time, worker has bean registered with master vmax17 when master vmax17 got leadership.

So Worker registration failed: Duplicate worker ID.

 

spark-mr-master-vmax18.log:
**
{code:java}
2019-03-15 20:22:09,441 INFO ZooKeeperLeaderElectionAgent: We have lost leadership
2019-03-15 20:22:14,544 WARN Master: Removing worker-20190218183101-vmax18-33129 because we got no heartbeat in 60 seconds
2019-03-15 20:22:14,544 INFO Master: Removing worker worker-20190218183101-vmax18-33129 on vmax18:33129
2019-03-15 20:22:14,864 WARN Master: Got heartbeat from unregistered worker worker-20190218183101-vmax18-33129. Asking it to re-register.
2019-03-15 20:22:14,975 ERROR Master: Leadership has been revoked -- master shutting down.
{code}
 

spark-mr-master-vmax17.log:

 
{code:java}
2019-03-15 20:22:14,870 INFO Master: Registering worker vmax18:33129 with 21 cores, 125.0 GB RAM
2019-03-15 20:22:15,261 INFO Master: vmax18:33129 got disassociated, removing it.
2019-03-15 20:22:15,263 INFO Master: Removing worker worker-20190218183101-vmax18-33129 on vmax18:33129
2019-03-15 20:22:15,311 ERROR Inbox: Ignoring error
org.apache.zookeeper.KeeperException$NoNodeException: KeeperErrorCode = NoNode for /spark/master_status/worker_worker-20190218183101-vmax18-33129
{code}
 

 

spark-mr-worker-vmax18.log:

 
{code:java}
2019-03-15 20:22:10,474 INFO Worker: Master has changed, new master is at spark://vmax17:7077
2019-03-15 20:22:14,862 INFO Worker: Master with url spark://vmax18:7077 requested this worker to reconnect.
2019-03-15 20:22:14,865 INFO Worker: Not spawning another attempt to register with the master, since there is an attempt scheduled already.
2019-03-15 20:22:14,879 ERROR Worker: Worker registration failed: Duplicate worker ID
2019-03-15 20:22:14,895 INFO ExecutorRunner: Killing process!
2019-03-15 20:22:14,896 INFO ShutdownHookManager: Shutdown hook called{code}
 

 

PS, this will result another issue: The leader will always in COMPLETING_RECOVERY state.

worker-vmax18 shut down cause duplicate worker ID,and clear the worker's node on persist Engine(we use zookeeper). Then the new leader(master-vmax17) find the worker died and trying to remove it ,and try to clear the node on zookeeper,but the node has been removed yet during worker-vmax18 shut down ,so {color:#FF0000}*an exception was thrown in function completeRecovery()* *. Then the leader will always in COMPLETING_RECOVERY state.*{color}

 

> Workers registration failes in case of network drop
> ---------------------------------------------------
>
>                 Key: SPARK-23191
>                 URL: https://issues.apache.org/jira/browse/SPARK-23191
>             Project: Spark
>          Issue Type: Bug
>          Components: Spark Core
>    Affects Versions: 1.6.3, 2.2.1, 2.3.0
>         Environment: OS:- Centos 6.9(64 bit)
>  
>            Reporter: Neeraj Gupta
>            Priority: Critical
>
> We have a 3 node cluster. We were facing issues of multiple driver running in some scenario in production.
> On further investigation we were able to reproduce iin both 1.6.3 and 2.2.1 versions the scenario with following steps:-
>  # Setup a 3 node cluster. Start master and slaves.
>  # On any node where the worker process is running block the connections on port 7077 using iptables.
> {code:java}
> iptables -A OUTPUT -p tcp --dport 7077 -j DROP
> {code}
>  # After about 10-15 secs we get the error on node that it is unable to connect to master.
> {code:java}
> 2018-01-23 12:08:51,639 [rpc-client-1-1] WARN  org.apache.spark.network.server.TransportChannelHandler - Exception in connection from <servername>
> java.io.IOException: Connection timed out
>         at sun.nio.ch.FileDispatcherImpl.read0(Native Method)
>         at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:39)
>         at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
>         at sun.nio.ch.IOUtil.read(IOUtil.java:192)
>         at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
>         at io.netty.buffer.PooledUnsafeDirectByteBuf.setBytes(PooledUnsafeDirectByteBuf.java:221)
>         at io.netty.buffer.AbstractByteBuf.writeBytes(AbstractByteBuf.java:899)
>         at io.netty.channel.socket.nio.NioSocketChannel.doReadBytes(NioSocketChannel.java:275)
>         at io.netty.channel.nio.AbstractNioByteChannel$NioByteUnsafe.read(AbstractNioByteChannel.java:119)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:643)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:566)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:480)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:442)
>         at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:131)
>         at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:144)
>         at java.lang.Thread.run(Thread.java:745)
> 2018-01-23 12:08:51,647 [dispatcher-event-loop-0] ERROR org.apache.spark.deploy.worker.Worker - Connection to master failed! Waiting for master to reconnect...
> 2018-01-23 12:08:51,647 [dispatcher-event-loop-0] ERROR org.apache.spark.deploy.worker.Worker - Connection to master failed! Waiting for master to reconnect...
> {code}
>  # Once we get this exception we renable the connections to port 7077 using
> {code:java}
> iptables -D OUTPUT -p tcp --dport 7077 -j DROP
> {code}
>  # Worker tries to register again with master but is unable to do so. It gives following error
> {code:java}
> 2018-01-23 12:08:58,657 [worker-register-master-threadpool-2] WARN  org.apache.spark.deploy.worker.Worker - Failed to connect to master <servername>:7077
> org.apache.spark.SparkException: Exception thrown in awaitResult:
>         at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:205)
>         at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75)
>         at org.apache.spark.rpc.RpcEnv.setupEndpointRefByURI(RpcEnv.scala:100)
>         at org.apache.spark.rpc.RpcEnv.setupEndpointRef(RpcEnv.scala:108)
>         at org.apache.spark.deploy.worker.Worker$$anonfun$org$apache$spark$deploy$worker$Worker$$tryRegisterAllMasters$1$$anon$1.run(Worker.scala:241)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: Failed to connect to <servername>:7077
>         at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:232)
>         at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:182)
>         at org.apache.spark.rpc.netty.NettyRpcEnv.createClient(NettyRpcEnv.scala:197)
>         at org.apache.spark.rpc.netty.Outbox$$anon$1.call(Outbox.scala:194)
>         at org.apache.spark.rpc.netty.Outbox$$anon$1.call(Outbox.scala:190)
>         ... 4 more
> Caused by: io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection timed out: <servername>:7077
>         at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
>         at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
>         at io.netty.channel.socket.nio.NioSocketChannel.doFinishConnect(NioSocketChannel.java:257)
>         at io.netty.channel.nio.AbstractNioChannel$AbstractNioUnsafe.finishConnect(AbstractNioChannel.java:291)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKey(NioEventLoop.java:631)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeysOptimized(NioEventLoop.java:566)
>         at io.netty.channel.nio.NioEventLoop.processSelectedKeys(NioEventLoop.java:480)
>         at io.netty.channel.nio.NioEventLoop.run(NioEventLoop.java:442)
>         at io.netty.util.concurrent.SingleThreadEventExecutor$2.run(SingleThreadEventExecutor.java:131)
>         at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:144)
>         ... 1 more
> 2018-01-23 12:09:03,705 [dispatcher-event-loop-5] ERROR org.apache.spark.deploy.worker.Worker - Worker registration failed: Duplicate worker ID
> 2018-01-23 12:09:03,705 [dispatcher-event-loop-5] ERROR org.apache.spark.deploy.worker.Worker - Worker registration failed: Duplicate worker ID{code}
>  # The worker state is changed to DEAD in spark UI. As a result of which duplicate driver is launched.



--
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