Skip to content

Use forkserver on Unix and Python 3 #687

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

Merged
merged 4 commits into from
Nov 21, 2016
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
9 changes: 5 additions & 4 deletions .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ install:
# Install dependencies
- conda create -q -n test-environment python=$TRAVIS_PYTHON_VERSION
- source activate test-environment
- conda install -q pytest pytest-timeout coverage tornado toolz dill futures dask ipywidgets psutil bokeh requests joblib mock ipykernel jupyter_client h5py netcdf4
- conda install -q pytest pytest-timeout coverage tornado toolz dill futures dask ipywidgets psutil bokeh requests joblib mock ipykernel jupyter_client h5py netcdf4 lz4 paramiko
- |
if [[ $HDFS == true ]]; then
conda install -q libxml2 krb5 boost
Expand All @@ -63,9 +63,10 @@ install:
- python setup.py install

script:
- export PYTEST_OPTIONS="--verbose -r s --timeout-method=thread --timeout=300"
- |
if [[ $HDFS == true ]]; then
py.test distributed/tests/test_hdfs.py --verbose -r s --timeout-method=thread --timeout=30
py.test distributed/tests/test_hdfs.py $PYTEST_OPTIONS
if [ $? -ne 0 ]; then
# Diagnose test error
echo "--"
Expand All @@ -75,9 +76,9 @@ script:
(exit 1)
fi
elif [[ $COVERAGE == true ]]; then
coverage run $(which py.test) distributed -m "not avoid_travis" --verbose;
coverage run $(which py.test) distributed -m "not avoid_travis" $PYTEST_OPTIONS;
else
py.test -m "not avoid_travis" distributed --verbose;
py.test -m "not avoid_travis" distributed $PYTEST_OPTIONS;
fi;

after_success:
Expand Down
1 change: 0 additions & 1 deletion distributed/bokeh/application.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@
import atexit
import json
import logging
import multiprocessing
import os
import socket
import sys
Expand Down
1 change: 0 additions & 1 deletion distributed/cli/dask_scheduler.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,6 @@
import atexit
import json
import logging
import multiprocessing
import os
import socket
import subprocess
Expand Down
21 changes: 11 additions & 10 deletions distributed/nanny.py
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@
from datetime import datetime, timedelta
import json
import logging
from multiprocessing import Process, Queue, queues
from multiprocessing.queues import Empty
import os
import shutil
import subprocess
Expand All @@ -18,7 +18,7 @@
from .compatibility import JSONDecodeError
from .core import Server, rpc, write, RPCClosed
from .protocol import to_serialize
from .utils import get_ip, ignoring, log_errors, tmpfile
from .utils import get_ip, ignoring, log_errors, mp_context, tmpfile
from .worker import _ncores, Worker, run, TOTAL_MEMORY

nanny_environment = os.path.dirname(sys.executable)
Expand Down Expand Up @@ -190,13 +190,14 @@ def instantiate(self, stream=None, environment=None):
except JSONDecodeError:
yield gen.sleep(0.01)
else:
q = Queue()
self.process = Process(target=run_worker_fork,
args=(q, self.ip, self.scheduler.ip,
self.scheduler.port, self.ncores,
self.port, self._given_worker_port,
self.local_dir, self.services, self.name,
self.memory_limit, self.reconnect))
q = mp_context.Queue()
self.process = mp_context.Process(
target=run_worker_fork,
args=(q, self.ip, self.scheduler.ip,
self.scheduler.port, self.ncores,
self.port, self._given_worker_port,
self.local_dir, self.services, self.name,
self.memory_limit, self.reconnect))
self.process.daemon = True
self.process.start()
while True:
Expand All @@ -208,7 +209,7 @@ def instantiate(self, stream=None, environment=None):
self.worker_dir = msg['dir']
assert self.worker_port
break
except queues.Empty:
except Empty:
yield gen.sleep(0.1)

logger.info("Nanny %s:%d starts worker process %s:%d",
Expand Down
5 changes: 2 additions & 3 deletions distributed/tests/test_client.py
Original file line number Diff line number Diff line change
Expand Up @@ -6,7 +6,6 @@
from concurrent.futures import CancelledError
from datetime import timedelta
import itertools
from multiprocessing import Process
import os
import pickle
from random import random, choice
Expand Down Expand Up @@ -34,7 +33,7 @@
temp_default_client, get_restrictions)
from distributed.scheduler import Scheduler, KilledWorker
from distributed.sizeof import sizeof
from distributed.utils import sync, tmp_text, ignoring, tokey, All
from distributed.utils import sync, tmp_text, ignoring, tokey, All, mp_context
from distributed.utils_test import (cluster, slow, slowinc, slowadd, randominc,
loop, inc, dec, div, throws, gen_cluster, gen_test, double, deep)

Expand Down Expand Up @@ -1614,7 +1613,7 @@ def long_running_client_connection(ip, port):

@gen_cluster()
def test_cleanup_after_broken_client_connection(s, a, b):
proc = Process(target=long_running_client_connection, args=(s.ip, s.port))
proc = mp_context.Process(target=long_running_client_connection, args=(s.ip, s.port))
proc.daemon = True
proc.start()

Expand Down
2 changes: 1 addition & 1 deletion distributed/tests/test_core.py
Original file line number Diff line number Diff line change
@@ -1,7 +1,6 @@
from __future__ import print_function, division, absolute_import

from functools import partial
from multiprocessing import Process
import socket
from time import time

Expand All @@ -13,6 +12,7 @@
coerce_to_rpc, send_recv, coerce_to_address, ConnectionPool)
from distributed.utils_test import slow, loop, gen_test


def test_server(loop):
@gen.coroutine
def f():
Expand Down
Loading