Issue #12328: Fix multiprocessing's use of overlapped I/O on Windows.
Also, add a multiprocessing.connection.wait(rlist, timeout=None) function for polling multiple objects at once. Patch by sbt. Complete changelist from sbt's patch: * Adds a wait(rlist, timeout=None) function for polling multiple objects at once. On Unix this is just a wrapper for select(rlist, [], [], timeout=None). * Removes use of the SentinelReady exception and the sentinels argument to certain methods. concurrent.futures.process has been changed to use wait() instead of SentinelReady. * Fixes bugs concerning PipeConnection.poll() and messages of zero length. * Fixes PipeListener.accept() to call ConnectNamedPipe() with overlapped=True. * Fixes Queue.empty() and SimpleQueue.empty() so that they are threadsafe on Windows. * Now PipeConnection.poll() and wait() will not modify the pipe except possibly by consuming a zero length message. (Previously poll() could consume a partial message.) * All of multiprocesing's pipe related blocking functions/methods are now interruptible by SIGINT on Windows.
This commit is contained in:
parent
1e88f3faa6
commit
bdb1cf1ca5
|
@ -415,13 +415,14 @@ The :mod:`multiprocessing` package mostly replicates the API of the
|
||||||
A numeric handle of a system object which will become "ready" when
|
A numeric handle of a system object which will become "ready" when
|
||||||
the process ends.
|
the process ends.
|
||||||
|
|
||||||
|
You can use this value if you want to wait on several events at
|
||||||
|
once using :func:`multiprocessing.connection.wait`. Otherwise
|
||||||
|
calling :meth:`join()` is simpler.
|
||||||
|
|
||||||
On Windows, this is an OS handle usable with the ``WaitForSingleObject``
|
On Windows, this is an OS handle usable with the ``WaitForSingleObject``
|
||||||
and ``WaitForMultipleObjects`` family of API calls. On Unix, this is
|
and ``WaitForMultipleObjects`` family of API calls. On Unix, this is
|
||||||
a file descriptor usable with primitives from the :mod:`select` module.
|
a file descriptor usable with primitives from the :mod:`select` module.
|
||||||
|
|
||||||
You can use this value if you want to wait on several events at once.
|
|
||||||
Otherwise calling :meth:`join()` is simpler.
|
|
||||||
|
|
||||||
.. versionadded:: 3.3
|
.. versionadded:: 3.3
|
||||||
|
|
||||||
.. method:: terminate()
|
.. method:: terminate()
|
||||||
|
@ -785,6 +786,9 @@ Connection objects are usually created using :func:`Pipe` -- see also
|
||||||
*timeout* is a number then this specifies the maximum time in seconds to
|
*timeout* is a number then this specifies the maximum time in seconds to
|
||||||
block. If *timeout* is ``None`` then an infinite timeout is used.
|
block. If *timeout* is ``None`` then an infinite timeout is used.
|
||||||
|
|
||||||
|
Note that multiple connection objects may be polled at once by
|
||||||
|
using :func:`multiprocessing.connection.wait`.
|
||||||
|
|
||||||
.. method:: send_bytes(buffer[, offset[, size]])
|
.. method:: send_bytes(buffer[, offset[, size]])
|
||||||
|
|
||||||
Send byte data from an object supporting the buffer interface as a
|
Send byte data from an object supporting the buffer interface as a
|
||||||
|
@ -1779,8 +1783,9 @@ Usually message passing between processes is done using queues or by using
|
||||||
|
|
||||||
However, the :mod:`multiprocessing.connection` module allows some extra
|
However, the :mod:`multiprocessing.connection` module allows some extra
|
||||||
flexibility. It basically gives a high level message oriented API for dealing
|
flexibility. It basically gives a high level message oriented API for dealing
|
||||||
with sockets or Windows named pipes, and also has support for *digest
|
with sockets or Windows named pipes. It also has support for *digest
|
||||||
authentication* using the :mod:`hmac` module.
|
authentication* using the :mod:`hmac` module, and for polling
|
||||||
|
multiple connections at the same time.
|
||||||
|
|
||||||
|
|
||||||
.. function:: deliver_challenge(connection, authkey)
|
.. function:: deliver_challenge(connection, authkey)
|
||||||
|
@ -1878,6 +1883,38 @@ authentication* using the :mod:`hmac` module.
|
||||||
The address from which the last accepted connection came. If this is
|
The address from which the last accepted connection came. If this is
|
||||||
unavailable then it is ``None``.
|
unavailable then it is ``None``.
|
||||||
|
|
||||||
|
.. function:: wait(object_list, timeout=None)
|
||||||
|
|
||||||
|
Wait till an object in *object_list* is ready. Returns the list of
|
||||||
|
those objects in *object_list* which are ready. If *timeout* is a
|
||||||
|
float then the call blocks for at most that many seconds. If
|
||||||
|
*timeout* is ``None`` then it will block for an unlimited period.
|
||||||
|
|
||||||
|
For both Unix and Windows, an object can appear in *object_list* if
|
||||||
|
it is
|
||||||
|
|
||||||
|
* a readable :class:`~multiprocessing.Connection` object;
|
||||||
|
* a connected and readable :class:`socket.socket` object; or
|
||||||
|
* the :attr:`~multiprocessing.Process.sentinel` attribute of a
|
||||||
|
:class:`~multiprocessing.Process` object.
|
||||||
|
|
||||||
|
A connection or socket object is ready when there is data available
|
||||||
|
to be read from it, or the other end has been closed.
|
||||||
|
|
||||||
|
**Unix**: ``wait(object_list, timeout)`` almost equivalent
|
||||||
|
``select.select(object_list, [], [], timeout)``. The difference is
|
||||||
|
that, if :func:`select.select` is interrupted by a signal, it can
|
||||||
|
raise :exc:`OSError` with an error number of ``EINTR``, whereas
|
||||||
|
:func:`wait` will not.
|
||||||
|
|
||||||
|
**Windows**: An item in *object_list* must either be an integer
|
||||||
|
handle which is waitable (according to the definition used by the
|
||||||
|
documentation of the Win32 function ``WaitForMultipleObjects()``)
|
||||||
|
or it can be an object with a :meth:`fileno` method which returns a
|
||||||
|
socket handle or pipe handle. (Note that pipe handles and socket
|
||||||
|
handles are **not** waitable handles.)
|
||||||
|
|
||||||
|
.. versionadded:: 3.3
|
||||||
|
|
||||||
The module defines two exceptions:
|
The module defines two exceptions:
|
||||||
|
|
||||||
|
@ -1929,6 +1966,41 @@ server::
|
||||||
|
|
||||||
conn.close()
|
conn.close()
|
||||||
|
|
||||||
|
The following code uses :func:`~multiprocessing.connection.wait` to
|
||||||
|
wait for messages from multiple processes at once::
|
||||||
|
|
||||||
|
import time, random
|
||||||
|
from multiprocessing import Process, Pipe, current_process
|
||||||
|
from multiprocessing.connection import wait
|
||||||
|
|
||||||
|
def foo(w):
|
||||||
|
for i in range(10):
|
||||||
|
w.send((i, current_process().name))
|
||||||
|
w.close()
|
||||||
|
|
||||||
|
if __name__ == '__main__':
|
||||||
|
readers = []
|
||||||
|
|
||||||
|
for i in range(4):
|
||||||
|
r, w = Pipe(duplex=False)
|
||||||
|
readers.append(r)
|
||||||
|
p = Process(target=foo, args=(w,))
|
||||||
|
p.start()
|
||||||
|
# We close the writable end of the pipe now to be sure that
|
||||||
|
# p is the only process which owns a handle for it. This
|
||||||
|
# ensures that when p closes its handle for the writable end,
|
||||||
|
# wait() will promptly report the readable end as being ready.
|
||||||
|
w.close()
|
||||||
|
|
||||||
|
while readers:
|
||||||
|
for r in wait(readers):
|
||||||
|
try:
|
||||||
|
msg = r.recv()
|
||||||
|
except EOFError:
|
||||||
|
readers.remove(r)
|
||||||
|
else:
|
||||||
|
print(msg)
|
||||||
|
|
||||||
|
|
||||||
.. _multiprocessing-address-formats:
|
.. _multiprocessing-address-formats:
|
||||||
|
|
||||||
|
|
|
@ -50,7 +50,8 @@ import os
|
||||||
from concurrent.futures import _base
|
from concurrent.futures import _base
|
||||||
import queue
|
import queue
|
||||||
import multiprocessing
|
import multiprocessing
|
||||||
from multiprocessing.queues import SimpleQueue, SentinelReady, Full
|
from multiprocessing.queues import SimpleQueue, Full
|
||||||
|
from multiprocessing.connection import wait
|
||||||
import threading
|
import threading
|
||||||
import weakref
|
import weakref
|
||||||
|
|
||||||
|
@ -212,6 +213,8 @@ def _queue_management_worker(executor_reference,
|
||||||
for p in processes.values():
|
for p in processes.values():
|
||||||
p.join()
|
p.join()
|
||||||
|
|
||||||
|
reader = result_queue._reader
|
||||||
|
|
||||||
while True:
|
while True:
|
||||||
_add_call_item_to_queue(pending_work_items,
|
_add_call_item_to_queue(pending_work_items,
|
||||||
work_ids_queue,
|
work_ids_queue,
|
||||||
|
@ -219,9 +222,10 @@ def _queue_management_worker(executor_reference,
|
||||||
|
|
||||||
sentinels = [p.sentinel for p in processes.values()]
|
sentinels = [p.sentinel for p in processes.values()]
|
||||||
assert sentinels
|
assert sentinels
|
||||||
try:
|
ready = wait([reader] + sentinels)
|
||||||
result_item = result_queue.get(sentinels=sentinels)
|
if reader in ready:
|
||||||
except SentinelReady:
|
result_item = reader.recv()
|
||||||
|
else:
|
||||||
# Mark the process pool broken so that submits fail right now.
|
# Mark the process pool broken so that submits fail right now.
|
||||||
executor = executor_reference()
|
executor = executor_reference()
|
||||||
if executor is not None:
|
if executor is not None:
|
||||||
|
|
|
@ -32,7 +32,7 @@
|
||||||
# SUCH DAMAGE.
|
# SUCH DAMAGE.
|
||||||
#
|
#
|
||||||
|
|
||||||
__all__ = [ 'Client', 'Listener', 'Pipe' ]
|
__all__ = [ 'Client', 'Listener', 'Pipe', 'wait' ]
|
||||||
|
|
||||||
import io
|
import io
|
||||||
import os
|
import os
|
||||||
|
@ -58,8 +58,6 @@ except ImportError:
|
||||||
raise
|
raise
|
||||||
win32 = None
|
win32 = None
|
||||||
|
|
||||||
_select = _eintr_retry(select.select)
|
|
||||||
|
|
||||||
#
|
#
|
||||||
#
|
#
|
||||||
#
|
#
|
||||||
|
@ -122,15 +120,6 @@ def address_type(address):
|
||||||
else:
|
else:
|
||||||
raise ValueError('address type of %r unrecognized' % address)
|
raise ValueError('address type of %r unrecognized' % address)
|
||||||
|
|
||||||
|
|
||||||
class SentinelReady(Exception):
|
|
||||||
"""
|
|
||||||
Raised when a sentinel is ready when polling.
|
|
||||||
"""
|
|
||||||
def __init__(self, *args):
|
|
||||||
Exception.__init__(self, *args)
|
|
||||||
self.sentinels = args[0]
|
|
||||||
|
|
||||||
#
|
#
|
||||||
# Connection classes
|
# Connection classes
|
||||||
#
|
#
|
||||||
|
@ -268,11 +257,11 @@ class _ConnectionBase:
|
||||||
(offset + size) // itemsize])
|
(offset + size) // itemsize])
|
||||||
return size
|
return size
|
||||||
|
|
||||||
def recv(self, sentinels=None):
|
def recv(self):
|
||||||
"""Receive a (picklable) object"""
|
"""Receive a (picklable) object"""
|
||||||
self._check_closed()
|
self._check_closed()
|
||||||
self._check_readable()
|
self._check_readable()
|
||||||
buf = self._recv_bytes(sentinels=sentinels)
|
buf = self._recv_bytes()
|
||||||
return pickle.loads(buf.getbuffer())
|
return pickle.loads(buf.getbuffer())
|
||||||
|
|
||||||
def poll(self, timeout=0.0):
|
def poll(self, timeout=0.0):
|
||||||
|
@ -290,85 +279,80 @@ if win32:
|
||||||
Overlapped I/O is used, so the handles must have been created
|
Overlapped I/O is used, so the handles must have been created
|
||||||
with FILE_FLAG_OVERLAPPED.
|
with FILE_FLAG_OVERLAPPED.
|
||||||
"""
|
"""
|
||||||
_buffered = b''
|
_got_empty_message = False
|
||||||
|
|
||||||
def _close(self, _CloseHandle=win32.CloseHandle):
|
def _close(self, _CloseHandle=win32.CloseHandle):
|
||||||
_CloseHandle(self._handle)
|
_CloseHandle(self._handle)
|
||||||
|
|
||||||
def _send_bytes(self, buf):
|
def _send_bytes(self, buf):
|
||||||
overlapped = win32.WriteFile(self._handle, buf, overlapped=True)
|
ov, err = win32.WriteFile(self._handle, buf, overlapped=True)
|
||||||
nwritten, complete = overlapped.GetOverlappedResult(True)
|
try:
|
||||||
assert complete
|
if err == win32.ERROR_IO_PENDING:
|
||||||
|
waitres = win32.WaitForMultipleObjects(
|
||||||
|
[ov.event], False, INFINITE)
|
||||||
|
assert waitres == WAIT_OBJECT_0
|
||||||
|
except:
|
||||||
|
ov.cancel()
|
||||||
|
raise
|
||||||
|
finally:
|
||||||
|
nwritten, err = ov.GetOverlappedResult(True)
|
||||||
|
assert err == 0
|
||||||
assert nwritten == len(buf)
|
assert nwritten == len(buf)
|
||||||
|
|
||||||
def _recv_bytes(self, maxsize=None, sentinels=()):
|
def _recv_bytes(self, maxsize=None):
|
||||||
if sentinels:
|
if self._got_empty_message:
|
||||||
self._poll(-1.0, sentinels)
|
self._got_empty_message = False
|
||||||
buf = io.BytesIO()
|
return io.BytesIO()
|
||||||
firstchunk = self._buffered
|
|
||||||
if firstchunk:
|
|
||||||
lenfirstchunk = len(firstchunk)
|
|
||||||
buf.write(firstchunk)
|
|
||||||
self._buffered = b''
|
|
||||||
else:
|
else:
|
||||||
# A reasonable size for the first chunk transfer
|
bsize = 128 if maxsize is None else min(maxsize, 128)
|
||||||
bufsize = 128
|
|
||||||
if maxsize is not None and maxsize < bufsize:
|
|
||||||
bufsize = maxsize
|
|
||||||
try:
|
try:
|
||||||
overlapped = win32.ReadFile(self._handle, bufsize, overlapped=True)
|
ov, err = win32.ReadFile(self._handle, bsize,
|
||||||
lenfirstchunk, complete = overlapped.GetOverlappedResult(True)
|
overlapped=True)
|
||||||
firstchunk = overlapped.getbuffer()
|
try:
|
||||||
assert lenfirstchunk == len(firstchunk)
|
if err == win32.ERROR_IO_PENDING:
|
||||||
|
waitres = win32.WaitForMultipleObjects(
|
||||||
|
[ov.event], False, INFINITE)
|
||||||
|
assert waitres == WAIT_OBJECT_0
|
||||||
|
except:
|
||||||
|
ov.cancel()
|
||||||
|
raise
|
||||||
|
finally:
|
||||||
|
nread, err = ov.GetOverlappedResult(True)
|
||||||
|
if err == 0:
|
||||||
|
f = io.BytesIO()
|
||||||
|
f.write(ov.getbuffer())
|
||||||
|
return f
|
||||||
|
elif err == win32.ERROR_MORE_DATA:
|
||||||
|
return self._get_more_data(ov, maxsize)
|
||||||
except IOError as e:
|
except IOError as e:
|
||||||
if e.winerror == win32.ERROR_BROKEN_PIPE:
|
if e.winerror == win32.ERROR_BROKEN_PIPE:
|
||||||
raise EOFError
|
raise EOFError
|
||||||
raise
|
else:
|
||||||
buf.write(firstchunk)
|
raise
|
||||||
if complete:
|
raise RuntimeError("shouldn't get here; expected KeyboardInterrupt")
|
||||||
return buf
|
|
||||||
navail, nleft = win32.PeekNamedPipe(self._handle)
|
|
||||||
if maxsize is not None and lenfirstchunk + nleft > maxsize:
|
|
||||||
return None
|
|
||||||
if nleft > 0:
|
|
||||||
overlapped = win32.ReadFile(self._handle, nleft, overlapped=True)
|
|
||||||
res, complete = overlapped.GetOverlappedResult(True)
|
|
||||||
assert res == nleft
|
|
||||||
assert complete
|
|
||||||
buf.write(overlapped.getbuffer())
|
|
||||||
return buf
|
|
||||||
|
|
||||||
def _poll(self, timeout, sentinels=()):
|
def _poll(self, timeout):
|
||||||
# Fast non-blocking path
|
if (self._got_empty_message or
|
||||||
navail, nleft = win32.PeekNamedPipe(self._handle)
|
win32.PeekNamedPipe(self._handle)[0] != 0):
|
||||||
if navail > 0:
|
|
||||||
return True
|
return True
|
||||||
elif timeout == 0.0:
|
if timeout < 0:
|
||||||
return False
|
timeout = None
|
||||||
# Blocking: use overlapped I/O
|
return bool(wait([self], timeout))
|
||||||
if timeout < 0.0:
|
|
||||||
timeout = INFINITE
|
def _get_more_data(self, ov, maxsize):
|
||||||
else:
|
buf = ov.getbuffer()
|
||||||
timeout = int(timeout * 1000 + 0.5)
|
f = io.BytesIO()
|
||||||
overlapped = win32.ReadFile(self._handle, 1, overlapped=True)
|
f.write(buf)
|
||||||
try:
|
left = win32.PeekNamedPipe(self._handle)[1]
|
||||||
handles = [overlapped.event]
|
assert left > 0
|
||||||
handles += sentinels
|
if maxsize is not None and len(buf) + left > maxsize:
|
||||||
res = win32.WaitForMultipleObjects(handles, False, timeout)
|
self._bad_message_length()
|
||||||
finally:
|
ov, err = win32.ReadFile(self._handle, left, overlapped=True)
|
||||||
# Always cancel overlapped I/O in the same thread
|
rbytes, err = ov.GetOverlappedResult(True)
|
||||||
# (because CancelIoEx() appears only in Vista)
|
assert err == 0
|
||||||
overlapped.cancel()
|
assert rbytes == left
|
||||||
if res == WAIT_TIMEOUT:
|
f.write(ov.getbuffer())
|
||||||
return False
|
return f
|
||||||
idx = res - WAIT_OBJECT_0
|
|
||||||
if idx == 0:
|
|
||||||
# I/O was successful, store received data
|
|
||||||
overlapped.GetOverlappedResult(True)
|
|
||||||
self._buffered += overlapped.getbuffer()
|
|
||||||
return True
|
|
||||||
assert 0 < idx < len(handles)
|
|
||||||
raise SentinelReady([handles[idx]])
|
|
||||||
|
|
||||||
|
|
||||||
class Connection(_ConnectionBase):
|
class Connection(_ConnectionBase):
|
||||||
|
@ -397,17 +381,11 @@ class Connection(_ConnectionBase):
|
||||||
break
|
break
|
||||||
buf = buf[n:]
|
buf = buf[n:]
|
||||||
|
|
||||||
def _recv(self, size, sentinels=(), read=_read):
|
def _recv(self, size, read=_read):
|
||||||
buf = io.BytesIO()
|
buf = io.BytesIO()
|
||||||
handle = self._handle
|
handle = self._handle
|
||||||
if sentinels:
|
|
||||||
handles = [handle] + sentinels
|
|
||||||
remaining = size
|
remaining = size
|
||||||
while remaining > 0:
|
while remaining > 0:
|
||||||
if sentinels:
|
|
||||||
r = _select(handles, [], [])[0]
|
|
||||||
if handle not in r:
|
|
||||||
raise SentinelReady(r)
|
|
||||||
chunk = read(handle, remaining)
|
chunk = read(handle, remaining)
|
||||||
n = len(chunk)
|
n = len(chunk)
|
||||||
if n == 0:
|
if n == 0:
|
||||||
|
@ -428,17 +406,17 @@ class Connection(_ConnectionBase):
|
||||||
if n > 0:
|
if n > 0:
|
||||||
self._send(buf)
|
self._send(buf)
|
||||||
|
|
||||||
def _recv_bytes(self, maxsize=None, sentinels=()):
|
def _recv_bytes(self, maxsize=None):
|
||||||
buf = self._recv(4, sentinels)
|
buf = self._recv(4)
|
||||||
size, = struct.unpack("!i", buf.getvalue())
|
size, = struct.unpack("!i", buf.getvalue())
|
||||||
if maxsize is not None and size > maxsize:
|
if maxsize is not None and size > maxsize:
|
||||||
return None
|
return None
|
||||||
return self._recv(size, sentinels)
|
return self._recv(size)
|
||||||
|
|
||||||
def _poll(self, timeout):
|
def _poll(self, timeout):
|
||||||
if timeout < 0.0:
|
if timeout < 0.0:
|
||||||
timeout = None
|
timeout = None
|
||||||
r = _select([self._handle], [], [], timeout)[0]
|
r = wait([self._handle], timeout)
|
||||||
return bool(r)
|
return bool(r)
|
||||||
|
|
||||||
|
|
||||||
|
@ -559,7 +537,8 @@ else:
|
||||||
)
|
)
|
||||||
|
|
||||||
overlapped = win32.ConnectNamedPipe(h1, overlapped=True)
|
overlapped = win32.ConnectNamedPipe(h1, overlapped=True)
|
||||||
overlapped.GetOverlappedResult(True)
|
_, err = overlapped.GetOverlappedResult(True)
|
||||||
|
assert err == 0
|
||||||
|
|
||||||
c1 = PipeConnection(h1, writable=duplex)
|
c1 = PipeConnection(h1, writable=duplex)
|
||||||
c2 = PipeConnection(h2, readable=duplex)
|
c2 = PipeConnection(h2, readable=duplex)
|
||||||
|
@ -633,39 +612,40 @@ if sys.platform == 'win32':
|
||||||
'''
|
'''
|
||||||
def __init__(self, address, backlog=None):
|
def __init__(self, address, backlog=None):
|
||||||
self._address = address
|
self._address = address
|
||||||
handle = win32.CreateNamedPipe(
|
self._handle_queue = [self._new_handle(first=True)]
|
||||||
address, win32.PIPE_ACCESS_DUPLEX |
|
|
||||||
win32.FILE_FLAG_FIRST_PIPE_INSTANCE,
|
|
||||||
win32.PIPE_TYPE_MESSAGE | win32.PIPE_READMODE_MESSAGE |
|
|
||||||
win32.PIPE_WAIT,
|
|
||||||
win32.PIPE_UNLIMITED_INSTANCES, BUFSIZE, BUFSIZE,
|
|
||||||
win32.NMPWAIT_WAIT_FOREVER, win32.NULL
|
|
||||||
)
|
|
||||||
self._handle_queue = [handle]
|
|
||||||
self._last_accepted = None
|
self._last_accepted = None
|
||||||
|
|
||||||
sub_debug('listener created with address=%r', self._address)
|
sub_debug('listener created with address=%r', self._address)
|
||||||
|
|
||||||
self.close = Finalize(
|
self.close = Finalize(
|
||||||
self, PipeListener._finalize_pipe_listener,
|
self, PipeListener._finalize_pipe_listener,
|
||||||
args=(self._handle_queue, self._address), exitpriority=0
|
args=(self._handle_queue, self._address), exitpriority=0
|
||||||
)
|
)
|
||||||
|
|
||||||
def accept(self):
|
def _new_handle(self, first=False):
|
||||||
newhandle = win32.CreateNamedPipe(
|
flags = win32.PIPE_ACCESS_DUPLEX | win32.FILE_FLAG_OVERLAPPED
|
||||||
self._address, win32.PIPE_ACCESS_DUPLEX,
|
if first:
|
||||||
|
flags |= win32.FILE_FLAG_FIRST_PIPE_INSTANCE
|
||||||
|
return win32.CreateNamedPipe(
|
||||||
|
self._address, flags,
|
||||||
win32.PIPE_TYPE_MESSAGE | win32.PIPE_READMODE_MESSAGE |
|
win32.PIPE_TYPE_MESSAGE | win32.PIPE_READMODE_MESSAGE |
|
||||||
win32.PIPE_WAIT,
|
win32.PIPE_WAIT,
|
||||||
win32.PIPE_UNLIMITED_INSTANCES, BUFSIZE, BUFSIZE,
|
win32.PIPE_UNLIMITED_INSTANCES, BUFSIZE, BUFSIZE,
|
||||||
win32.NMPWAIT_WAIT_FOREVER, win32.NULL
|
win32.NMPWAIT_WAIT_FOREVER, win32.NULL
|
||||||
)
|
)
|
||||||
self._handle_queue.append(newhandle)
|
|
||||||
|
def accept(self):
|
||||||
|
self._handle_queue.append(self._new_handle())
|
||||||
handle = self._handle_queue.pop(0)
|
handle = self._handle_queue.pop(0)
|
||||||
|
ov = win32.ConnectNamedPipe(handle, overlapped=True)
|
||||||
try:
|
try:
|
||||||
win32.ConnectNamedPipe(handle, win32.NULL)
|
res = win32.WaitForMultipleObjects([ov.event], False, INFINITE)
|
||||||
except WindowsError as e:
|
except:
|
||||||
if e.winerror != win32.ERROR_PIPE_CONNECTED:
|
ov.cancel()
|
||||||
raise
|
win32.CloseHandle(handle)
|
||||||
|
raise
|
||||||
|
finally:
|
||||||
|
_, err = ov.GetOverlappedResult(True)
|
||||||
|
assert err == 0
|
||||||
return PipeConnection(handle)
|
return PipeConnection(handle)
|
||||||
|
|
||||||
@staticmethod
|
@staticmethod
|
||||||
|
@ -684,7 +664,8 @@ if sys.platform == 'win32':
|
||||||
win32.WaitNamedPipe(address, 1000)
|
win32.WaitNamedPipe(address, 1000)
|
||||||
h = win32.CreateFile(
|
h = win32.CreateFile(
|
||||||
address, win32.GENERIC_READ | win32.GENERIC_WRITE,
|
address, win32.GENERIC_READ | win32.GENERIC_WRITE,
|
||||||
0, win32.NULL, win32.OPEN_EXISTING, 0, win32.NULL
|
0, win32.NULL, win32.OPEN_EXISTING,
|
||||||
|
win32.FILE_FLAG_OVERLAPPED, win32.NULL
|
||||||
)
|
)
|
||||||
except WindowsError as e:
|
except WindowsError as e:
|
||||||
if e.winerror not in (win32.ERROR_SEM_TIMEOUT,
|
if e.winerror not in (win32.ERROR_SEM_TIMEOUT,
|
||||||
|
@ -773,6 +754,125 @@ def XmlClient(*args, **kwds):
|
||||||
import xmlrpc.client as xmlrpclib
|
import xmlrpc.client as xmlrpclib
|
||||||
return ConnectionWrapper(Client(*args, **kwds), _xml_dumps, _xml_loads)
|
return ConnectionWrapper(Client(*args, **kwds), _xml_dumps, _xml_loads)
|
||||||
|
|
||||||
|
#
|
||||||
|
# Wait
|
||||||
|
#
|
||||||
|
|
||||||
|
if sys.platform == 'win32':
|
||||||
|
|
||||||
|
def _exhaustive_wait(handles, timeout):
|
||||||
|
# Return ALL handles which are currently signalled. (Only
|
||||||
|
# returning the first signalled might create starvation issues.)
|
||||||
|
L = list(handles)
|
||||||
|
ready = []
|
||||||
|
while L:
|
||||||
|
res = win32.WaitForMultipleObjects(L, False, timeout)
|
||||||
|
if res == WAIT_TIMEOUT:
|
||||||
|
break
|
||||||
|
elif WAIT_OBJECT_0 <= res < WAIT_OBJECT_0 + len(L):
|
||||||
|
res -= WAIT_OBJECT_0
|
||||||
|
elif WAIT_ABANDONED_0 <= res < WAIT_ABANDONED_0 + len(L):
|
||||||
|
res -= WAIT_ABANDONED_0
|
||||||
|
else:
|
||||||
|
raise RuntimeError('Should not get here')
|
||||||
|
ready.append(L[res])
|
||||||
|
L = L[res+1:]
|
||||||
|
timeout = 0
|
||||||
|
return ready
|
||||||
|
|
||||||
|
_ready_errors = {win32.ERROR_BROKEN_PIPE, win32.ERROR_NETNAME_DELETED}
|
||||||
|
|
||||||
|
def wait(object_list, timeout=None):
|
||||||
|
'''
|
||||||
|
Wait till an object in object_list is ready/readable.
|
||||||
|
|
||||||
|
Returns list of those objects in object_list which are ready/readable.
|
||||||
|
'''
|
||||||
|
if timeout is None:
|
||||||
|
timeout = INFINITE
|
||||||
|
elif timeout < 0:
|
||||||
|
timeout = 0
|
||||||
|
else:
|
||||||
|
timeout = int(timeout * 1000 + 0.5)
|
||||||
|
|
||||||
|
object_list = list(object_list)
|
||||||
|
waithandle_to_obj = {}
|
||||||
|
ov_list = []
|
||||||
|
ready_objects = set()
|
||||||
|
ready_handles = set()
|
||||||
|
|
||||||
|
try:
|
||||||
|
for o in object_list:
|
||||||
|
try:
|
||||||
|
fileno = getattr(o, 'fileno')
|
||||||
|
except AttributeError:
|
||||||
|
waithandle_to_obj[o.__index__()] = o
|
||||||
|
else:
|
||||||
|
# start an overlapped read of length zero
|
||||||
|
try:
|
||||||
|
ov, err = win32.ReadFile(fileno(), 0, True)
|
||||||
|
except OSError as e:
|
||||||
|
err = e.winerror
|
||||||
|
if err not in _ready_errors:
|
||||||
|
raise
|
||||||
|
if err == win32.ERROR_IO_PENDING:
|
||||||
|
ov_list.append(ov)
|
||||||
|
waithandle_to_obj[ov.event] = o
|
||||||
|
else:
|
||||||
|
# If o.fileno() is an overlapped pipe handle and
|
||||||
|
# err == 0 then there is a zero length message
|
||||||
|
# in the pipe, but it HAS NOT been consumed.
|
||||||
|
ready_objects.add(o)
|
||||||
|
timeout = 0
|
||||||
|
|
||||||
|
ready_handles = _exhaustive_wait(waithandle_to_obj.keys(), timeout)
|
||||||
|
finally:
|
||||||
|
# request that overlapped reads stop
|
||||||
|
for ov in ov_list:
|
||||||
|
ov.cancel()
|
||||||
|
|
||||||
|
# wait for all overlapped reads to stop
|
||||||
|
for ov in ov_list:
|
||||||
|
try:
|
||||||
|
_, err = ov.GetOverlappedResult(True)
|
||||||
|
except OSError as e:
|
||||||
|
err = e.winerror
|
||||||
|
if err not in _ready_errors:
|
||||||
|
raise
|
||||||
|
if err != win32.ERROR_OPERATION_ABORTED:
|
||||||
|
o = waithandle_to_obj[ov.event]
|
||||||
|
ready_objects.add(o)
|
||||||
|
if err == 0:
|
||||||
|
# If o.fileno() is an overlapped pipe handle then
|
||||||
|
# a zero length message HAS been consumed.
|
||||||
|
if hasattr(o, '_got_empty_message'):
|
||||||
|
o._got_empty_message = True
|
||||||
|
|
||||||
|
ready_objects.update(waithandle_to_obj[h] for h in ready_handles)
|
||||||
|
return [o for o in object_list if o in ready_objects]
|
||||||
|
|
||||||
|
else:
|
||||||
|
|
||||||
|
def wait(object_list, timeout=None):
|
||||||
|
'''
|
||||||
|
Wait till an object in object_list is ready/readable.
|
||||||
|
|
||||||
|
Returns list of those objects in object_list which are ready/readable.
|
||||||
|
'''
|
||||||
|
if timeout is not None:
|
||||||
|
if timeout <= 0:
|
||||||
|
return select.select(object_list, [], [], 0)[0]
|
||||||
|
else:
|
||||||
|
deadline = time.time() + timeout
|
||||||
|
while True:
|
||||||
|
try:
|
||||||
|
return select.select(object_list, [], [], timeout)[0]
|
||||||
|
except OSError as e:
|
||||||
|
if e.errno != errno.EINTR:
|
||||||
|
raise
|
||||||
|
if timeout is not None:
|
||||||
|
timeout = deadline - time.time()
|
||||||
|
|
||||||
|
|
||||||
# Late import because of circular import
|
# Late import because of circular import
|
||||||
from multiprocessing.forking import duplicate, close
|
from multiprocessing.forking import duplicate, close
|
||||||
|
|
|
@ -44,7 +44,7 @@ import errno
|
||||||
|
|
||||||
from queue import Empty, Full
|
from queue import Empty, Full
|
||||||
import _multiprocessing
|
import _multiprocessing
|
||||||
from multiprocessing.connection import Pipe, SentinelReady
|
from multiprocessing.connection import Pipe
|
||||||
from multiprocessing.synchronize import Lock, BoundedSemaphore, Semaphore, Condition
|
from multiprocessing.synchronize import Lock, BoundedSemaphore, Semaphore, Condition
|
||||||
from multiprocessing.util import debug, info, Finalize, register_after_fork
|
from multiprocessing.util import debug, info, Finalize, register_after_fork
|
||||||
from multiprocessing.forking import assert_spawning
|
from multiprocessing.forking import assert_spawning
|
||||||
|
@ -360,6 +360,7 @@ class SimpleQueue(object):
|
||||||
def __init__(self):
|
def __init__(self):
|
||||||
self._reader, self._writer = Pipe(duplex=False)
|
self._reader, self._writer = Pipe(duplex=False)
|
||||||
self._rlock = Lock()
|
self._rlock = Lock()
|
||||||
|
self._poll = self._reader.poll
|
||||||
if sys.platform == 'win32':
|
if sys.platform == 'win32':
|
||||||
self._wlock = None
|
self._wlock = None
|
||||||
else:
|
else:
|
||||||
|
@ -367,7 +368,7 @@ class SimpleQueue(object):
|
||||||
self._make_methods()
|
self._make_methods()
|
||||||
|
|
||||||
def empty(self):
|
def empty(self):
|
||||||
return not self._reader.poll()
|
return not self._poll()
|
||||||
|
|
||||||
def __getstate__(self):
|
def __getstate__(self):
|
||||||
assert_spawning(self)
|
assert_spawning(self)
|
||||||
|
@ -380,10 +381,10 @@ class SimpleQueue(object):
|
||||||
def _make_methods(self):
|
def _make_methods(self):
|
||||||
recv = self._reader.recv
|
recv = self._reader.recv
|
||||||
racquire, rrelease = self._rlock.acquire, self._rlock.release
|
racquire, rrelease = self._rlock.acquire, self._rlock.release
|
||||||
def get(*, sentinels=None):
|
def get():
|
||||||
racquire()
|
racquire()
|
||||||
try:
|
try:
|
||||||
return recv(sentinels)
|
return recv()
|
||||||
finally:
|
finally:
|
||||||
rrelease()
|
rrelease()
|
||||||
self.get = get
|
self.get = get
|
||||||
|
|
|
@ -1811,6 +1811,84 @@ class _TestListenerClient(BaseTestCase):
|
||||||
p.join()
|
p.join()
|
||||||
l.close()
|
l.close()
|
||||||
|
|
||||||
|
class _TestPoll(unittest.TestCase):
|
||||||
|
|
||||||
|
ALLOWED_TYPES = ('processes', 'threads')
|
||||||
|
|
||||||
|
def test_empty_string(self):
|
||||||
|
a, b = self.Pipe()
|
||||||
|
self.assertEqual(a.poll(), False)
|
||||||
|
b.send_bytes(b'')
|
||||||
|
self.assertEqual(a.poll(), True)
|
||||||
|
self.assertEqual(a.poll(), True)
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def _child_strings(cls, conn, strings):
|
||||||
|
for s in strings:
|
||||||
|
time.sleep(0.1)
|
||||||
|
conn.send_bytes(s)
|
||||||
|
conn.close()
|
||||||
|
|
||||||
|
def test_strings(self):
|
||||||
|
strings = (b'hello', b'', b'a', b'b', b'', b'bye', b'', b'lop')
|
||||||
|
a, b = self.Pipe()
|
||||||
|
p = self.Process(target=self._child_strings, args=(b, strings))
|
||||||
|
p.start()
|
||||||
|
|
||||||
|
for s in strings:
|
||||||
|
for i in range(200):
|
||||||
|
if a.poll(0.01):
|
||||||
|
break
|
||||||
|
x = a.recv_bytes()
|
||||||
|
self.assertEqual(s, x)
|
||||||
|
|
||||||
|
p.join()
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def _child_boundaries(cls, r):
|
||||||
|
# Polling may "pull" a message in to the child process, but we
|
||||||
|
# don't want it to pull only part of a message, as that would
|
||||||
|
# corrupt the pipe for any other processes which might later
|
||||||
|
# read from it.
|
||||||
|
r.poll(5)
|
||||||
|
|
||||||
|
def test_boundaries(self):
|
||||||
|
r, w = self.Pipe(False)
|
||||||
|
p = self.Process(target=self._child_boundaries, args=(r,))
|
||||||
|
p.start()
|
||||||
|
time.sleep(2)
|
||||||
|
L = [b"first", b"second"]
|
||||||
|
for obj in L:
|
||||||
|
w.send_bytes(obj)
|
||||||
|
w.close()
|
||||||
|
p.join()
|
||||||
|
self.assertIn(r.recv_bytes(), L)
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def _child_dont_merge(cls, b):
|
||||||
|
b.send_bytes(b'a')
|
||||||
|
b.send_bytes(b'b')
|
||||||
|
b.send_bytes(b'cd')
|
||||||
|
|
||||||
|
def test_dont_merge(self):
|
||||||
|
a, b = self.Pipe()
|
||||||
|
self.assertEqual(a.poll(0.0), False)
|
||||||
|
self.assertEqual(a.poll(0.1), False)
|
||||||
|
|
||||||
|
p = self.Process(target=self._child_dont_merge, args=(b,))
|
||||||
|
p.start()
|
||||||
|
|
||||||
|
self.assertEqual(a.recv_bytes(), b'a')
|
||||||
|
self.assertEqual(a.poll(1.0), True)
|
||||||
|
self.assertEqual(a.poll(1.0), True)
|
||||||
|
self.assertEqual(a.recv_bytes(), b'b')
|
||||||
|
self.assertEqual(a.poll(1.0), True)
|
||||||
|
self.assertEqual(a.poll(1.0), True)
|
||||||
|
self.assertEqual(a.poll(0.0), True)
|
||||||
|
self.assertEqual(a.recv_bytes(), b'cd')
|
||||||
|
|
||||||
|
p.join()
|
||||||
|
|
||||||
#
|
#
|
||||||
# Test of sending connection and socket objects between processes
|
# Test of sending connection and socket objects between processes
|
||||||
#
|
#
|
||||||
|
@ -2404,8 +2482,163 @@ class TestStdinBadfiledescriptor(unittest.TestCase):
|
||||||
flike.flush()
|
flike.flush()
|
||||||
assert sio.getvalue() == 'foo'
|
assert sio.getvalue() == 'foo'
|
||||||
|
|
||||||
|
|
||||||
|
class TestWait(unittest.TestCase):
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def _child_test_wait(cls, w, slow):
|
||||||
|
for i in range(10):
|
||||||
|
if slow:
|
||||||
|
time.sleep(random.random()*0.1)
|
||||||
|
w.send((i, os.getpid()))
|
||||||
|
w.close()
|
||||||
|
|
||||||
|
def test_wait(self, slow=False):
|
||||||
|
from multiprocessing import Pipe, Process
|
||||||
|
from multiprocessing.connection import wait
|
||||||
|
readers = []
|
||||||
|
procs = []
|
||||||
|
messages = []
|
||||||
|
|
||||||
|
for i in range(4):
|
||||||
|
r, w = Pipe(duplex=False)
|
||||||
|
p = Process(target=self._child_test_wait, args=(w, slow))
|
||||||
|
p.daemon = True
|
||||||
|
p.start()
|
||||||
|
w.close()
|
||||||
|
readers.append(r)
|
||||||
|
procs.append(p)
|
||||||
|
|
||||||
|
while readers:
|
||||||
|
for r in wait(readers):
|
||||||
|
try:
|
||||||
|
msg = r.recv()
|
||||||
|
except EOFError:
|
||||||
|
readers.remove(r)
|
||||||
|
r.close()
|
||||||
|
else:
|
||||||
|
messages.append(msg)
|
||||||
|
|
||||||
|
messages.sort()
|
||||||
|
expected = sorted((i, p.pid) for i in range(10) for p in procs)
|
||||||
|
self.assertEqual(messages, expected)
|
||||||
|
|
||||||
|
@classmethod
|
||||||
|
def _child_test_wait_socket(cls, address, slow):
|
||||||
|
s = socket.socket()
|
||||||
|
s.connect(address)
|
||||||
|
for i in range(10):
|
||||||
|
if slow:
|
||||||
|
time.sleep(random.random()*0.1)
|
||||||
|
s.sendall(('%s\n' % i).encode('ascii'))
|
||||||
|
s.close()
|
||||||
|
|
||||||
|
def test_wait_socket(self, slow=False):
|
||||||
|
from multiprocessing import Process
|
||||||
|
from multiprocessing.connection import wait
|
||||||
|
l = socket.socket()
|
||||||
|
l.bind(('', 0))
|
||||||
|
l.listen(4)
|
||||||
|
addr = ('localhost', l.getsockname()[1])
|
||||||
|
readers = []
|
||||||
|
procs = []
|
||||||
|
dic = {}
|
||||||
|
|
||||||
|
for i in range(4):
|
||||||
|
p = Process(target=self._child_test_wait_socket, args=(addr, slow))
|
||||||
|
p.daemon = True
|
||||||
|
p.start()
|
||||||
|
procs.append(p)
|
||||||
|
|
||||||
|
for i in range(4):
|
||||||
|
r, _ = l.accept()
|
||||||
|
readers.append(r)
|
||||||
|
dic[r] = []
|
||||||
|
l.close()
|
||||||
|
|
||||||
|
while readers:
|
||||||
|
for r in wait(readers):
|
||||||
|
msg = r.recv(32)
|
||||||
|
if not msg:
|
||||||
|
readers.remove(r)
|
||||||
|
r.close()
|
||||||
|
else:
|
||||||
|
dic[r].append(msg)
|
||||||
|
|
||||||
|
expected = ''.join('%s\n' % i for i in range(10)).encode('ascii')
|
||||||
|
for v in dic.values():
|
||||||
|
self.assertEqual(b''.join(v), expected)
|
||||||
|
|
||||||
|
def test_wait_slow(self):
|
||||||
|
self.test_wait(True)
|
||||||
|
|
||||||
|
def test_wait_socket_slow(self):
|
||||||
|
self.test_wait(True)
|
||||||
|
|
||||||
|
def test_wait_timeout(self):
|
||||||
|
from multiprocessing.connection import wait
|
||||||
|
|
||||||
|
expected = 1
|
||||||
|
a, b = multiprocessing.Pipe()
|
||||||
|
|
||||||
|
start = time.time()
|
||||||
|
res = wait([a, b], 1)
|
||||||
|
delta = time.time() - start
|
||||||
|
|
||||||
|
self.assertEqual(res, [])
|
||||||
|
self.assertLess(delta, expected + 0.2)
|
||||||
|
self.assertGreater(delta, expected - 0.2)
|
||||||
|
|
||||||
|
b.send(None)
|
||||||
|
|
||||||
|
start = time.time()
|
||||||
|
res = wait([a, b], 1)
|
||||||
|
delta = time.time() - start
|
||||||
|
|
||||||
|
self.assertEqual(res, [a])
|
||||||
|
self.assertLess(delta, 0.2)
|
||||||
|
|
||||||
|
def test_wait_integer(self):
|
||||||
|
from multiprocessing.connection import wait
|
||||||
|
|
||||||
|
expected = 5
|
||||||
|
a, b = multiprocessing.Pipe()
|
||||||
|
p = multiprocessing.Process(target=time.sleep, args=(expected,))
|
||||||
|
|
||||||
|
p.start()
|
||||||
|
self.assertIsInstance(p.sentinel, int)
|
||||||
|
|
||||||
|
start = time.time()
|
||||||
|
res = wait([a, p.sentinel, b], expected + 20)
|
||||||
|
delta = time.time() - start
|
||||||
|
|
||||||
|
self.assertEqual(res, [p.sentinel])
|
||||||
|
self.assertLess(delta, expected + 1)
|
||||||
|
self.assertGreater(delta, expected - 1)
|
||||||
|
|
||||||
|
a.send(None)
|
||||||
|
|
||||||
|
start = time.time()
|
||||||
|
res = wait([a, p.sentinel, b], 20)
|
||||||
|
delta = time.time() - start
|
||||||
|
|
||||||
|
self.assertEqual(res, [p.sentinel, b])
|
||||||
|
self.assertLess(delta, 0.2)
|
||||||
|
|
||||||
|
b.send(None)
|
||||||
|
|
||||||
|
start = time.time()
|
||||||
|
res = wait([a, p.sentinel, b], 20)
|
||||||
|
delta = time.time() - start
|
||||||
|
|
||||||
|
self.assertEqual(res, [a, p.sentinel, b])
|
||||||
|
self.assertLess(delta, 0.2)
|
||||||
|
|
||||||
|
p.join()
|
||||||
|
|
||||||
|
|
||||||
testcases_other = [OtherTest, TestInvalidHandle, TestInitializers,
|
testcases_other = [OtherTest, TestInvalidHandle, TestInitializers,
|
||||||
TestStdinBadfiledescriptor]
|
TestStdinBadfiledescriptor, TestWait]
|
||||||
|
|
||||||
#
|
#
|
||||||
#
|
#
|
||||||
|
|
|
@ -13,6 +13,10 @@ Core and Builtins
|
||||||
Library
|
Library
|
||||||
-------
|
-------
|
||||||
|
|
||||||
|
- Issue #12328: Fix multiprocessing's use of overlapped I/O on Windows.
|
||||||
|
Also, add a multiprocessing.connection.wait(rlist, timeout=None) function
|
||||||
|
for polling multiple objects at once. Patch by sbt.
|
||||||
|
|
||||||
- Issue #13719: Make the distutils and packaging upload commands aware of
|
- Issue #13719: Make the distutils and packaging upload commands aware of
|
||||||
bdist_msi products.
|
bdist_msi products.
|
||||||
|
|
||||||
|
|
|
@ -60,16 +60,18 @@ typedef struct {
|
||||||
static void
|
static void
|
||||||
overlapped_dealloc(OverlappedObject *self)
|
overlapped_dealloc(OverlappedObject *self)
|
||||||
{
|
{
|
||||||
|
DWORD bytes;
|
||||||
int err = GetLastError();
|
int err = GetLastError();
|
||||||
if (self->pending) {
|
if (self->pending) {
|
||||||
if (check_CancelIoEx())
|
/* make it a programming error to deallocate while operation
|
||||||
Py_CancelIoEx(self->handle, &self->overlapped);
|
is pending, even if we can safely cancel it */
|
||||||
else {
|
if (check_CancelIoEx() &&
|
||||||
PyErr_SetString(PyExc_RuntimeError,
|
Py_CancelIoEx(self->handle, &self->overlapped))
|
||||||
"I/O operations still in flight while destroying "
|
GetOverlappedResult(self->handle, &self->overlapped, &bytes, TRUE);
|
||||||
"Overlapped object, the process may crash");
|
PyErr_SetString(PyExc_RuntimeError,
|
||||||
PyErr_WriteUnraisable(NULL);
|
"I/O operations still in flight while destroying "
|
||||||
}
|
"Overlapped object, the process may crash");
|
||||||
|
PyErr_WriteUnraisable(NULL);
|
||||||
}
|
}
|
||||||
CloseHandle(self->overlapped.hEvent);
|
CloseHandle(self->overlapped.hEvent);
|
||||||
SetLastError(err);
|
SetLastError(err);
|
||||||
|
@ -85,6 +87,7 @@ overlapped_GetOverlappedResult(OverlappedObject *self, PyObject *waitobj)
|
||||||
int wait;
|
int wait;
|
||||||
BOOL res;
|
BOOL res;
|
||||||
DWORD transferred = 0;
|
DWORD transferred = 0;
|
||||||
|
DWORD err;
|
||||||
|
|
||||||
wait = PyObject_IsTrue(waitobj);
|
wait = PyObject_IsTrue(waitobj);
|
||||||
if (wait < 0)
|
if (wait < 0)
|
||||||
|
@ -94,23 +97,27 @@ overlapped_GetOverlappedResult(OverlappedObject *self, PyObject *waitobj)
|
||||||
wait != 0);
|
wait != 0);
|
||||||
Py_END_ALLOW_THREADS
|
Py_END_ALLOW_THREADS
|
||||||
|
|
||||||
if (!res) {
|
err = res ? ERROR_SUCCESS : GetLastError();
|
||||||
int err = GetLastError();
|
switch (err) {
|
||||||
if (err == ERROR_IO_INCOMPLETE)
|
case ERROR_SUCCESS:
|
||||||
Py_RETURN_NONE;
|
case ERROR_MORE_DATA:
|
||||||
if (err != ERROR_MORE_DATA) {
|
case ERROR_OPERATION_ABORTED:
|
||||||
|
self->completed = 1;
|
||||||
|
self->pending = 0;
|
||||||
|
break;
|
||||||
|
case ERROR_IO_INCOMPLETE:
|
||||||
|
break;
|
||||||
|
default:
|
||||||
self->pending = 0;
|
self->pending = 0;
|
||||||
return PyErr_SetExcFromWindowsErr(PyExc_IOError, err);
|
return PyErr_SetExcFromWindowsErr(PyExc_IOError, err);
|
||||||
}
|
|
||||||
}
|
}
|
||||||
self->pending = 0;
|
if (self->completed && self->read_buffer != NULL) {
|
||||||
self->completed = 1;
|
|
||||||
if (self->read_buffer) {
|
|
||||||
assert(PyBytes_CheckExact(self->read_buffer));
|
assert(PyBytes_CheckExact(self->read_buffer));
|
||||||
if (_PyBytes_Resize(&self->read_buffer, transferred))
|
if (transferred != PyBytes_GET_SIZE(self->read_buffer) &&
|
||||||
|
_PyBytes_Resize(&self->read_buffer, transferred))
|
||||||
return NULL;
|
return NULL;
|
||||||
}
|
}
|
||||||
return Py_BuildValue("lN", (long) transferred, PyBool_FromLong(res));
|
return Py_BuildValue("II", (unsigned) transferred, (unsigned) err);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *
|
static PyObject *
|
||||||
|
@ -522,9 +529,10 @@ win32_WriteFile(PyObject *self, PyObject *args, PyObject *kwds)
|
||||||
HANDLE handle;
|
HANDLE handle;
|
||||||
Py_buffer _buf, *buf;
|
Py_buffer _buf, *buf;
|
||||||
PyObject *bufobj;
|
PyObject *bufobj;
|
||||||
int written;
|
DWORD written;
|
||||||
BOOL ret;
|
BOOL ret;
|
||||||
int use_overlapped = 0;
|
int use_overlapped = 0;
|
||||||
|
DWORD err;
|
||||||
OverlappedObject *overlapped = NULL;
|
OverlappedObject *overlapped = NULL;
|
||||||
static char *kwlist[] = {"handle", "buffer", "overlapped", NULL};
|
static char *kwlist[] = {"handle", "buffer", "overlapped", NULL};
|
||||||
|
|
||||||
|
@ -553,8 +561,9 @@ win32_WriteFile(PyObject *self, PyObject *args, PyObject *kwds)
|
||||||
overlapped ? &overlapped->overlapped : NULL);
|
overlapped ? &overlapped->overlapped : NULL);
|
||||||
Py_END_ALLOW_THREADS
|
Py_END_ALLOW_THREADS
|
||||||
|
|
||||||
|
err = ret ? 0 : GetLastError();
|
||||||
|
|
||||||
if (overlapped) {
|
if (overlapped) {
|
||||||
int err = GetLastError();
|
|
||||||
if (!ret) {
|
if (!ret) {
|
||||||
if (err == ERROR_IO_PENDING)
|
if (err == ERROR_IO_PENDING)
|
||||||
overlapped->pending = 1;
|
overlapped->pending = 1;
|
||||||
|
@ -563,13 +572,13 @@ win32_WriteFile(PyObject *self, PyObject *args, PyObject *kwds)
|
||||||
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return (PyObject *) overlapped;
|
return Py_BuildValue("NI", (PyObject *) overlapped, err);
|
||||||
}
|
}
|
||||||
|
|
||||||
PyBuffer_Release(buf);
|
PyBuffer_Release(buf);
|
||||||
if (!ret)
|
if (!ret)
|
||||||
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
||||||
return PyLong_FromLong(written);
|
return Py_BuildValue("II", written, err);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *
|
static PyObject *
|
||||||
|
@ -581,6 +590,7 @@ win32_ReadFile(PyObject *self, PyObject *args, PyObject *kwds)
|
||||||
PyObject *buf;
|
PyObject *buf;
|
||||||
BOOL ret;
|
BOOL ret;
|
||||||
int use_overlapped = 0;
|
int use_overlapped = 0;
|
||||||
|
DWORD err;
|
||||||
OverlappedObject *overlapped = NULL;
|
OverlappedObject *overlapped = NULL;
|
||||||
static char *kwlist[] = {"handle", "size", "overlapped", NULL};
|
static char *kwlist[] = {"handle", "size", "overlapped", NULL};
|
||||||
|
|
||||||
|
@ -607,8 +617,9 @@ win32_ReadFile(PyObject *self, PyObject *args, PyObject *kwds)
|
||||||
overlapped ? &overlapped->overlapped : NULL);
|
overlapped ? &overlapped->overlapped : NULL);
|
||||||
Py_END_ALLOW_THREADS
|
Py_END_ALLOW_THREADS
|
||||||
|
|
||||||
|
err = ret ? 0 : GetLastError();
|
||||||
|
|
||||||
if (overlapped) {
|
if (overlapped) {
|
||||||
int err = GetLastError();
|
|
||||||
if (!ret) {
|
if (!ret) {
|
||||||
if (err == ERROR_IO_PENDING)
|
if (err == ERROR_IO_PENDING)
|
||||||
overlapped->pending = 1;
|
overlapped->pending = 1;
|
||||||
|
@ -617,16 +628,16 @@ win32_ReadFile(PyObject *self, PyObject *args, PyObject *kwds)
|
||||||
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return (PyObject *) overlapped;
|
return Py_BuildValue("NI", (PyObject *) overlapped, err);
|
||||||
}
|
}
|
||||||
|
|
||||||
if (!ret && GetLastError() != ERROR_MORE_DATA) {
|
if (!ret && err != ERROR_MORE_DATA) {
|
||||||
Py_DECREF(buf);
|
Py_DECREF(buf);
|
||||||
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
return PyErr_SetExcFromWindowsErr(PyExc_IOError, 0);
|
||||||
}
|
}
|
||||||
if (_PyBytes_Resize(&buf, nread))
|
if (_PyBytes_Resize(&buf, nread))
|
||||||
return NULL;
|
return NULL;
|
||||||
return Py_BuildValue("NN", buf, PyBool_FromLong(ret));
|
return Py_BuildValue("NI", buf, err);
|
||||||
}
|
}
|
||||||
|
|
||||||
static PyObject *
|
static PyObject *
|
||||||
|
@ -783,7 +794,11 @@ create_win32_namespace(void)
|
||||||
|
|
||||||
WIN32_CONSTANT(F_DWORD, ERROR_ALREADY_EXISTS);
|
WIN32_CONSTANT(F_DWORD, ERROR_ALREADY_EXISTS);
|
||||||
WIN32_CONSTANT(F_DWORD, ERROR_BROKEN_PIPE);
|
WIN32_CONSTANT(F_DWORD, ERROR_BROKEN_PIPE);
|
||||||
|
WIN32_CONSTANT(F_DWORD, ERROR_IO_PENDING);
|
||||||
|
WIN32_CONSTANT(F_DWORD, ERROR_MORE_DATA);
|
||||||
|
WIN32_CONSTANT(F_DWORD, ERROR_NETNAME_DELETED);
|
||||||
WIN32_CONSTANT(F_DWORD, ERROR_NO_SYSTEM_RESOURCES);
|
WIN32_CONSTANT(F_DWORD, ERROR_NO_SYSTEM_RESOURCES);
|
||||||
|
WIN32_CONSTANT(F_DWORD, ERROR_OPERATION_ABORTED);
|
||||||
WIN32_CONSTANT(F_DWORD, ERROR_PIPE_BUSY);
|
WIN32_CONSTANT(F_DWORD, ERROR_PIPE_BUSY);
|
||||||
WIN32_CONSTANT(F_DWORD, ERROR_PIPE_CONNECTED);
|
WIN32_CONSTANT(F_DWORD, ERROR_PIPE_CONNECTED);
|
||||||
WIN32_CONSTANT(F_DWORD, ERROR_SEM_TIMEOUT);
|
WIN32_CONSTANT(F_DWORD, ERROR_SEM_TIMEOUT);
|
||||||
|
|
Loading…
Reference in New Issue