To wrap this up, it's the shuffle manager sending the FIN so setting
spark.shuffle.io.connectionTimeout to 3600s is the only workaround right
now. SPARK-12583 raised.
Adrian
--
*Adrian Bridgett*
I've worked around this by setting
spark.shuffle.io.connectionTimeout=3600s, uploading the spark tarball to
HDFS again and restarting the shuffle service (not 100% sure that last
step is needed).
I attempted (with my newbie Scala skills) to allow
ExternalShuffleClient() to accept an optional
Hi Ted,
sorry I should have been a bit more consistent in my cut and paste
(there are nine nodes +driver) - I was concentrating on S9/6 (these logs
are from that box - 10.1.201.165). S1/4 lines are:
15/12/29 18:49:45 INFO CoarseMesosSchedulerBackend: Registered executor
NettyRpcEndpointRef(n
Have you searched log for 'f02cb67a-3519-4655-b23a-edc0dd082bf1-S1/4' ?
In the snippet you posted, I don't see registration of this Executor.
Cheers
On Tue, Dec 29, 2015 at 12:43 PM, Adrian Bridgett
wrote:
> We're seeing an "Executor is not registered" error on a Spark (1.6.0rc4,
> mesos-0.26)
We're seeing an "Executor is not registered" error on a Spark (1.6.0rc4,
mesos-0.26) cluster. It seems as if the logic in
MesosExternalShuffleService.scala isn't working for some reason (new in
1.6 I believe).
spark application sees this:
...
15/12/29 18:49:41 INFO MesosExternalShuffleClient: