Skip to content

Slow running process dying at the last hurdle #1836

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
birdsarah opened this issue Mar 13, 2018 · 6 comments
Open

Slow running process dying at the last hurdle #1836

birdsarah opened this issue Mar 13, 2018 · 6 comments

Comments

@birdsarah
Copy link
Contributor

Twice now I've seen a slow running job, that's taking up a good amount of memory, but still is okay, slow down exponentially towards the end and then die with just a few tasks left with the following error:

distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:42369, Got: tcp://127.0.0.1:37509, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1647)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:40963, Got: tcp://127.0.0.1:37509, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1646)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:41049, Got: tcp://127.0.0.1:37509, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1640)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:40963, Got: tcp://127.0.0.1:37509, Key: ('getitem-51ca87464cb65fa6caa7647aca61003c', 1613)
distributed.scheduler - ERROR - 'tcp://127.0.0.1:38763'
Traceback (most recent call last):
  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/distributed/scheduler.py", line 2147, in handle_worker
    handler(worker=worker, **msg)
  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/distributed/scheduler.py", line 2041, in handle_missing_data
    ws = self.workers[errant_worker]
KeyError: 'tcp://127.0.0.1:38763'
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:40089, Got: tcp://127.0.0.1:32821, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1705)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:42369, Got: tcp://127.0.0.1:32821, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1748)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:41049, Got: tcp://127.0.0.1:32821, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1675)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:41049, Got: tcp://127.0.0.1:32821, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1625)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:40089, Got: tcp://127.0.0.1:32821, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1638)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:46085, Got: tcp://127.0.0.1:32821, Key: ('getitem-607bed68c5b7edc5d31ceee3800d4a54', 1349)
distributed.scheduler - WARNING - Unexpected worker completed task, likely due to work stealing.  Expected: tcp://127.0.0.1:40089, Got: tcp://127.0.0.1:32821, Key: ('read-parquet-2f32a06e1434bc59edb228d24c886dd2', 1526)
tornado.application - ERROR - Exception in Future <tornado.concurrent.Future object at 0x7f95485fb470> after timeout
Traceback (most recent call last):
  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/tornado/gen.py", line 910, in error_callback
    future.result()
  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/tornado/concurrent.py", line 238, in result
    raise_exc_info(self._exc_info)
  File "<string>", line 4, in raise_exc_info
  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/tornado/gen.py", line 1069, in run
    yielded = self.gen.send(value)
  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/distributed/nanny.py", line 471, in _wait_until_started
    assert msg == 'started', msg
AssertionError: {'address': 'tcp://127.0.0.1:35799', 'dir': '/home/bird/Dev/mozilla/sb2018/sandpit/dask-worker-space/worker-aydke_wy'}
distributed.nanny - WARNING - Worker process 13244 was killed by signal 15
distributed.nanny - WARNING - Restarting worker

In both cases I was able to successfully get the job to run without distributed using the vanilla dask scheduler in less time with far less memory. I believe the memory requirements were being affected by the pandas issue pandas-dev/pandas#19941

System info:

  • Fedora 27
  • using spawn not forkserver
  • dask & dask-core 0.17.1
  • distributed 1.21.3
  • python 3.6
@bnaul
Copy link
Contributor

bnaul commented Mar 22, 2018

I'm seeing similar behavior on 1.21.3 for a job that was succeeding before (I think on 1.21.1 but I'm not sure). Memory on the workers was around 1 out of 2 GB, scheduler was 12 out of 24 GB. Lots of the above Unexpected worker throughout (but I added some workers as I went along so that seemed fine), then suddenly lots of

distributed.scheduler - INFO - Register tcp://10.63.236.4:40392
distributed.scheduler - INFO - Starting worker compute stream, tcp://10.63.236.4:40392

and

distributed.scheduler - INFO - Worker 'tcp://10.63.233.4:41486' failed from closed comm: in <closed TCP>: Stream is closed
distributed.scheduler - INFO - Remove worker tcp://10.63.233.4:41486 

and

"Traceback (most recent call last):
  File "/opt/conda/envs/dask/lib/python3.6/site-packages/tornado/ioloop.py", line 1026, in _run
    return self.callback()
  File "/opt/conda/envs/dask/lib/python3.6/site-packages/distributed/stealing.py", line 309, in balance
    stealable = self.stealable[sat.address][level]
KeyError: 'tcp://10.63.74.4:33090'

@mrocklin
Copy link
Member

mrocklin commented Mar 22, 2018 via email

@mrocklin
Copy link
Member

We should downgrade that warning to INFO level. I am a bit surprised that it occurs though. I would expect the newer stealing handshakes to make things pretty transactional.

@bnaul
Copy link
Contributor

bnaul commented Mar 22, 2018

In my case there aren't any interesting logs on the worker; it's just getting kicked out, not killed/restarted...

distributed.worker - INFO - Start worker at: tcp://10.61.112.17:33379
distributed.worker - INFO - Listening to: tcp://10.61.112.17:33379
distributed.worker - INFO - nanny at: 10.61.112.17:43885
distributed.worker - INFO - bokeh at: 10.61.112.17:43532
distributed.worker - INFO - Waiting to connect to: tcp://model-services-daskscheduler:8786
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO - Threads: 2
distributed.worker - INFO - Memory: 2.00 GB
distributed.worker - INFO - Local Directory: /src/dask-worker-space/worker-k1nyb7yp
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO - Registered to: tcp://model-services-daskscheduler:8786
distributed.worker - INFO - -------------------------------------------------
distributed.worker - INFO - Connection to scheduler broken. Reregistering
...

EDIT: also interestingly, the workers seem to be carrying on happily even though the scheduler isn't aware of any tasks...
image
image

mrocklin added a commit to mrocklin/distributed that referenced this issue Mar 22, 2018
Previously we would fail in the following situation:

1.  A and B both data
2.  C tries to get data from A
3.  A fails during this transfer
4.  The scheduler goes to clean things up and gets confused because
    someone (B) still has the data

The fix for this wasn't hard, but the test is a bit odd.  It seems
that the subsequent communication from C to B fails, which I don't
yet understand.

Related to dask#1836
@mrocklin
Copy link
Member

Partially resolved by #1853

mrocklin added a commit that referenced this issue Mar 23, 2018
Previously we would fail in the following situation:

1.  A and B both data
2.  C tries to get data from A
3.  A fails during this transfer
4.  The scheduler goes to clean things up and gets confused because
    someone (B) still has the data

The fix for this wasn't hard, but the test is a bit odd.  It seems
that the subsequent communication from C to B fails, which I don't
yet understand.

Related to #1836
@mrocklin
Copy link
Member

The KeyError at this line is likely resolved now. Thank you for reporting. Leaving this issue open as there appear to be a couple other things here.

  File "/home/bird/miniconda3/envs/sb2018/lib/python3.6/site-packages/distributed/scheduler.py", line 2041, in handle_missing_data
    ws = self.workers[errant_worker]

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
Development

No branches or pull requests

3 participants