Description
What happened:
- read in two large dataframes via (both dfs are around 40Gb of parquet file data and we select a subset of the data so in memory is less)
COLUMNS = []
df1_path = 's3://some_path/df1'
df2_path = 's3://some_path/df2'
df1 = dd.read_parquet(df1_path,columns=COLUMNS)
df2 = dd.read_parquet(df2_path,columns=COLUMNS)
- set the index, global index column is sorted across all the parquet files before hand
df1 = df1.set_index("global_index", sorted=True)
df2 = df2.set_index("global_index", sorted=True)
- merge and persist df3 in memory
df3 = df2.merge(df1,suffixes=["_df1","_df2"] , left_index=True,right_index=True)
df3 = df3.persist()
- Dask dashboard just shows tasks stalled after a few of them are completed
no progress for over 30 mins (after which I just cancel the job)
workers seem idle -- no cpu usage (memory usage seems to be utilized, workers have a limit of 10Gb memory)
the graph page shows some tasks stuck "in-memory" at the set_index
stage ?
In the worker logs just see this about communication being lost while the workers are talking to each other (I'm assuming they are shuffling data for joins)
worker logs
distributed.worker - INFO - -------------------------------------------------
distributed.worker - ERROR - Worker stream died during communication: tls://123.12.123.123:37995 Traceback (most recent call last): File "internal_pip_dependency_tornado/pypi__tornado/tornado/iostream.py", line 971, in _handle_write num_bytes = self.write_to_fd(self._write_buffer.peek(size)) File "internal_pip_dependency_tornado/pypi__tornado/tornado/iostream.py", line 1568, in write_to_fd return self.socket.send(data) # type: ignore File "python3.6/usr/lib/python3.6/ssl.py", line 944, in send return self._sslobj.write(data) File "python3.6/usr/lib/python3.6/ssl.py", line 642, in write return self._sslobj.write(data) BrokenPipeError: [Errno 32] Broken pipe The above exception was the direct cause of the following exception: Traceback (most recent call last): File "distributed/worker.py", line 2064, in gather_dep self.rpc, deps, worker, who=self.address File "distributed/worker.py", line 3333, in get_data_from_worker return await retry_operation(_get_data, operation="get_data_from_worker") File "distributed/utils_comm.py", line 389, in retry_operation operation=operation, File "distributed/utils_comm.py", line 369, in retry return await coro() File "distributed/worker.py", line 3320, in _get_data max_connections=max_connections, File "distributed/core.py", line 644, in send_recv response = await comm.read(deserializers=deserializers) File "distributed/comm/tcp.py", line 205, in read convert_stream_closed_error(self, e) File "distributed/comm/tcp.py", line 126, in convert_stream_closed_error ) from exc distributed.comm.core.CommClosedError: in : BrokenPipeError: [Errno 32] Broken pipe
distributed.worker - INFO - Can't find dependencies for key ('fix-overlap-0ca23fb934d93f7d6d24d7aba82d6b8e', 128)
distributed.worker - INFO - Dependent not found: ('set_index-893549a2ed1d56f483fd0c8b82d2e14c', 128) 0 . Asking scheduler
distributed.worker - ERROR - Handle missing dep failed, retrying Traceback (most recent call last): File "distributed/comm/core.py", line 288, in connect timeout=min(intermediate_cap, time_left()), File "python3.6/usr/lib/python3.6/asyncio/tasks.py", line 362, in wait_for raise futures.TimeoutError() concurrent.futures._base.TimeoutError The above exception was the direct cause of the following exception: Traceback (most recent call last): File "distributed/worker.py", line 2212, in handle_missing_dep self.scheduler.who_has, keys=list(dep.key for dep in deps) File "distributed/utils_comm.py", line 389, in retry_operation operation=operation, File "distributed/utils_comm.py", line 369, in retry return await coro() File "distributed/core.py", line 858, in send_recv_from_rpc comm = await self.pool.connect(self.addr) File "distributed/core.py", line 1013, in connect **self.connection_args, File "distributed/comm/core.py", line 310, in connect ) from active_exception OSError: Timed out trying to connect to tls://dask-8f0f8f8ffd3942fd8143478f1ffa95e2.daskgateway:8786 after 10 s
distributed.worker - INFO - Dependent not found: ('set_index-893549a2ed1d56f483fd0c8b82d2e14c', 128) 0 . Asking scheduler
distributed.worker - ERROR - Worker stream died during communication: tls://123.12.123.123:37995 Traceback (most recent call last): File "distributed/comm/core.py", line 319, in connect handshake = await asyncio.wait_for(comm.read(), time_left()) File "python3.6/usr/lib/python3.6/asyncio/tasks.py", line 362, in wait_for raise futures.TimeoutError() concurrent.futures._base.TimeoutError The above exception was the direct cause of the following exception: Traceback (most recent call last): File "distributed/worker.py", line 2064, in gather_dep self.rpc, deps, worker, who=self.address File "distributed/worker.py", line 3333, in get_data_from_worker return await retry_operation(_get_data, operation="get_data_from_worker") File "distributed/utils_comm.py", line 389, in retry_operation operation=operation, File "distributed/utils_comm.py", line 369, in retry return await coro() File "distributed/worker.py", line 3310, in _get_data comm = await rpc.connect(worker) File "distributed/core.py", line 1013, in connect **self.connection_args, File "distributed/comm/core.py", line 326, in connect ) from exc OSError: Timed out during handshake while connecting to tls://123.12.123.123:37995 after 10 s
distributed.worker - INFO - Can't find dependencies for key ('fix-overlap-0ca23fb934d93f7d6d24d7aba82d6b8e', 164)
distributed.worker - INFO - Dependent not found: ('set_index-893549a2ed1d56f483fd0c8b82d2e14c', 163) 0 . Asking scheduler
distributed.worker - ERROR - Worker stream died during communication: tls://123.12.123.123:42005 Traceback (most recent call last): File "distributed/comm/core.py", line 288, in connect timeout=min(intermediate_cap, time_left()), File "python3.6/usr/lib/python3.6/asyncio/tasks.py", line 362, in wait_for raise futures.TimeoutError() concurrent.futures._base.TimeoutError The above exception was the direct cause of the following exception: Traceback (most recent call last): File "distributed/worker.py", line 2064, in gather_dep self.rpc, deps, worker, who=self.address File "distributed/worker.py", line 3333, in get_data_from_worker return await retry_operation(_get_data, operation="get_data_from_worker") File "distributed/utils_comm.py", line 389, in retry_operation operation=operation, File "distributed/utils_comm.py", line 369, in retry return await coro() File "distributed/worker.py", line 3310, in _get_data comm = await rpc.connect(worker) File "distributed/core.py", line 1013, in connect **self.connection_args, File "distributed/comm/core.py", line 310, in connect ) from active_exception OSError: Timed out trying to connect to tls://123.12.123.123:42005 after 10 s
distributed.worker - INFO - Can't find dependencies for key ('join-indexed-rename-1b8b6d5e81d4af20b409b7152fd1f5e4', 112)
distributed.worker - ERROR - Worker stream died during communication: tls://123.12.123.123:39863 Traceback (most recent call last): File "distributed/comm/core.py", line 288, in connect timeout=min(intermediate_cap, time_left()), File "python3.6/usr/lib/python3.6/asyncio/tasks.py", line 362, in wait_for raise futures.TimeoutError() concurrent.futures._base.TimeoutError The above exception was the direct cause of the following exception: Traceback (most recent call last): File "distributed/worker.py", line 2064, in gather_dep self.rpc, deps, worker, who=self.address File "distributed/worker.py", line 3333, in get_data_from_worker return await retry_operation(_get_data, operation="get_data_from_worker") File "distributed/utils_comm.py", line 389, in retry_operation operation=operation, File "distributed/utils_comm.py", line 369, in retry return await coro() File "distributed/worker.py", line 3310, in _get_data comm = await rpc.connect(worker) File "distributed/core.py", line 1013, in connect **self.connection_args, File "distributed/comm/core.py", line 310, in connect ) from active_exception OSError: Timed out trying to connect to tls://123.12.123.123:39863 after 10 s
distributed.worker - INFO - Can't find dependencies for key ('join-indexed-rename-1b8b6d5e81d4af20b409b7152fd1f5e4', 262)
distributed.worker - INFO - Dependent not found: ('repartition-merge-771c2501dd21c164d0c6e29ee1493490', 112) 0 . Asking scheduler
distributed.worker - INFO - Dependent not found: ('repartition-merge-10c4905ff08d78501e2eaf92c56e1bf9', 262) 0 . Asking scheduler
distributed.worker - ERROR - Worker stream died during communication: tls://123.12.123.123:41695 Traceback (most recent call last): File "internal_pip_dependency_tornado/pypi__tornado/tornado/iostream.py", line 971, in _handle_write num_bytes = self.write_to_fd(self._write_buffer.peek(size)) File "internal_pip_dependency_tornado/pypi__tornado/tornado/iostream.py", line 1568, in write_to_fd return self.socket.send(data) # type: ignore File "python3.6/usr/lib/python3.6/ssl.py", line 944, in send return self._sslobj.write(data) File "python3.6/usr/lib/python3.6/ssl.py", line 642, in write return self._sslobj.write(data) BrokenPipeError: [Errno 32] Broken pipe The above exception was the direct cause of the following exception: Traceback (most recent call last): File "distributed/worker.py", line 2064, in gather_dep self.rpc, deps, worker, who=self.address File "distributed/worker.py", line 3333, in get_data_from_worker return await retry_operation(_get_data, operation="get_data_from_worker") File "distributed/utils_comm.py", line 389, in retry_operation operation=operation, File "distributed/utils_comm.py", line 369, in retry return await coro() File "distributed/worker.py", line 3320, in _get_data max_connections=max_connections, File "distributed/core.py", line 644, in send_recv response = await comm.read(deserializers=deserializers) File "distributed/comm/tcp.py", line 205, in read convert_stream_closed_error(self, e) File "distributed/comm/tcp.py", line 126, in convert_stream_closed_error ) from exc distributed.comm.core.CommClosedError: in : BrokenPipeError: [Errno 32] Broken pipe
distributed.worker - INFO - Can't find dependencies for key ('join-indexed-rename-1b8b6d5e81d4af20b409b7152fd1f5e4', 358)
distributed.worker - INFO - Can't find dependencies for key ('join-indexed-rename-1b8b6d5e81d4af20b409b7152fd1f5e4', 361)
distributed.worker - INFO - Dependent not found: ('repartition-merge-10c4905ff08d78501e2eaf92c56e1bf9', 358) 0 . Asking scheduler
distributed.worker - INFO - Dependent not found: ('repartition-merge-10c4905ff08d78501e2eaf92c56e1bf9', 361) 0 . Asking scheduler
What you expected to happen:
This seems to be a fairly simple join operation of data not sure why the tasks are just stalling, I also tried not persisting the data and just doing len(df3)
and that too has the same result
Minimal Complete Verifiable Example:
Embedded in the description above
Anything else we need to know?:
I was using daskgateway to manage the cluster, I tried to use the approach in this video to launch subprocesses from the dask worker but I don't think that option is available in my dask version (I did try from a newer version but faced some issues as well but will leave that out)
Environment:
- Dask version: 2021.03.0
- Python version: 3.6
- Install method (conda, pip, source): pip
Cluster dump state seems to not be supported in this version