-
-
Notifications
You must be signed in to change notification settings - Fork 719
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
Tackle "ValueError: buffer source array is read-only" #1978
Comments
Hrm, interesting. @shoyer or @jakirkham might have thoughts on this.
…On Sat, May 12, 2018 at 6:35 PM, crusaderky ***@***.***> wrote:
One of the biggest pitfalls of running dask.array with a distributed
scheduler is the dreaded ValueError: buffer source array is read-only.
This error is typical when one runs a memoryview-based Cython kernel in
distributed, and it's particularly insidious as it will never show up in
unit tests performed with the local dask multithreaded scheduler. It might
even not show up when you run your problem on distributed and the arrays
just happen to never transit across nodes or to the disk cache (which is
exactly what the scheduler will try to achieve if enough RAM and CPU power
are available).
In other words, this is the classical issue that risks appearing in
production for the first time!
I can see a few ways of tackling the problem:
- in Cython: if you ain't writing to an array, it should just work. As
this issue has been around for the longest time, I suspect it might not be
trivial?
- in distributed, making sure that all arrays passed to all kernels
are writeable
- in dask.array, making sure that all arrays passed to all kernels are
NOT writeable, which actually makes a lot of sense regardless of
distributed. This will make the error crop up immediately in any naive unit
tests. It will also wreak havoc for many existing dask.array users though.
Possibly a opt-in settings?
- in the distributed docs, with a thorough tutorial on how to
reproduce the problem in unit testing and how to change your kernels to fix
it, so that it becomes the first result when anybody googles the exception.
On the last point, I personally solved the problem as follows:
In the kernels:
def _memoryview_safe(x):
"""Make array safe to run in a Cython memoryview-based kernel. These
kernels typically break down with the error ``ValueError: buffer source
array is read-only`` when running in dask distributed.
"""
if not x.flags.writeable:
if not x.flags.owndata:
x = x.copy(order='C')
x.setflags(write=True)
return x
def splev(x_new, t, c, k=3, extrapolate=True):
x_new = _memoryview_safe(x_new)
t = _memoryview_safe(t)
c = _memoryview_safe(c)
spline = scipy.interpolate.BSpline.construct_fast(t, c, k, axis=0, extrapolate=extrapolate)
return spline(x_new)
In the unit test:
def test_distributed():
def ro_array(a):
a = np.array(a)
a.setflags(write=False)
# Return a view of a, so that setting the write flag on the view is not enough
return a[:]
t = ro_array([1, 2])
c = ro_array([10, 20])
x_new = ro([1.5, 1.8])
splev(x_new, t, c, k=1)
If you comment out any of those calls to _memoryview_safe, the test falls
over.
Above I'm calling the kernel directly, but a similar thing can also be
invoked from the dask wrapper (probably a more robust design).
—
You are receiving this because you are subscribed to this thread.
Reply to this email directly, view it on GitHub
<#1978>, or mute the thread
<https://github.com/notifications/unsubscribe-auth/AASszAlUR5U7UPHV8lX4Yc8G3XRfAmBMks5tx2OygaJpZM4T8lhv>
.
|
Support for read-only memoryviews was only recently added into Cython: cython/cython#1869. If I understand that PR correctly, this error should no longer be raised if the Cythonized function does not try to override any data. So fixing this may be as simple as recompiling all your dependencies with the latest version of Cython. |
Thanks for raising this @crusaderky. Sorry for the slow reply. Agree this is not a fun issue and it needs a fix.
We can't do too much about how Cython behaves. Though it sounds like they have solved this in 0.28+, which is good to hear. That said, this can be a problem in vanilla Python if an assignment occurs under the hood in some wrapped function. IOW let's ignore the Cython context for a moment.
These seem like the most reasonable options IMHO. We could discuss documenting it more (and certainly that could be reasonable while we figure out how to solve this), but that is ultimately a workaround when we want a fix. First and foremost, this is a consistency problem. So can see value in both solutions just by making things more consistent. Second the value of Dask IMHO is that it provides a dataflow model. As such, it doesn't really make sense to write to some upstream array in the graph because it breaks this model or "goes against the flow" (couldn't resist 😄). IOW would consider the fact Dask Array allows this currently a bug and one needing a fix. So would be inclined to start forcing arrays in the Dask Array graph to be read-only. To summarize, we need consistency in this area across schedulers. Having read-only arrays is reasonable in Dask's dataflow model. So making all arrays in Dask read-only would be a reasonable fix to this problem. |
Issue ( dask/dask#3674 ) appears somewhat related. |
Just a quick comment on this since I was involved in providing feed-back about the Cython PR (this read-only problem happens quite often too in a scikit-learn context, or more precisely in a joblib context which automatically memmaps inputs in read-only mode, and we were quite interested by the functionality). To benefit from the cython feature you need to add a cpdef func_that_can_take_read_only_array(const double[:] input_array):
... There is a limitation of const memoryview at the moment: you can not use const memoryview with fused types, see cython/cython#1772 for more details. As far as scikit-learn is concerned this is the main reason we have not moved to using const memoryviews. |
I've just encountered this working with scikit-allel (cggh/scikit-allel#206). The function that causes this uses a fused type, so +1 for cython supporting const on fused types. Thanks @crusaderky for the workaround. |
I am facing this issue at times when running dask-ml PCA! Sometimes it runs smoothly, sometimes it throws this error. This is the code that I am submitting:
This is the output that I get:
|
I just ran into this for the first time. I was running on a fresh environment with the latest dask (1.2.2) and (1.28.0). As I hadn't seen it before, I downgraded to dask (1.2.0) and distributed (1.27.0) and the issue stopped. @jcrist I was wondering if maybe it was to do with the upgrade to series.isin, but maybe it's just a coincidence. Anyway, I thought I would report just in case. My code was very simple:
|
@birdsarah if you could provide a reproducible example causing this issue that would be much appreciated. |
I'm afraid I don't have time to work up a minimum reproducible example and I can't share my data right now. Sorry. |
I have no experience with Cython. How can I do this? |
I'd recommend checking your dependencies using Cython to confirm that it's
even possibly the cause. I believe that Cython includes the version used in
a comment at the top of the C file it generates.
…On Thu, Dec 12, 2019 at 12:13 PM demaheim ***@***.***> wrote:
So fixing this *may* be as simple as recompiling all your dependencies
with the latest version of Cython.
I have no experience with Cython. How can I do this?
—
You are receiving this because you are subscribed to this thread.
Reply to this email directly, view it on GitHub
<#1978?email_source=notifications&email_token=AAKAOIWOWBCO4PHQCDU5FULQYJ5NRA5CNFSM4E7SLBX2YY3PNVWWK3TUL52HS4DFVREXG43VMVBW63LNMVXHJKTDN5WW2ZLOORPWSZGOEGXRG4A#issuecomment-565121904>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AAKAOITEYLHZNBKD7WKQ3JTQYJ5NRANCNFSM4E7SLBXQ>
.
|
To check if using the latest version of Cython fixes my error, I used
and cloned my dependencies
In each dependency I ran
but I still get
|
About the PCA issue, I would guess it is unlikely that this one is related to Cython. The line that the traceback points to is
If I try to sum up my understanding of this issue:
Clarifications on point 3. by dask developers could potentially make it easier to reproduce the problem with a |
Just to confirm, is this still an issue? I believe I am running into it using a combination of |
Reproduced with stack as of May 2018; cannot reproduce with latest stack as of Feb 2020. Note how downgrading Cython was not enough to reproduce the issue; I did not investigate which package/version fixed the problem exactly. POCdemo.pyximport numpy
cpdef f(double[:] x):
return numpy.array(x)
cpdef g(const double[:] x):
return numpy.array(x) main.pyimport dask
import dask.array as da
import dask.threaded
import distributed
import numpy
import pyximport
pyximport.install()
from demo import f, g
def main():
a1 = da.ones(4, chunks=4)
a2 = da.from_array(numpy.ones(4), chunks=4)
client = distributed.Client()
for scheduler in ('threads', 'distributed'):
if dask.__version__ < '2':
kwargs = {"get": client.get if scheduler == "distributed" else dask.threaded.get}
else:
kwargs = {"scheduler": scheduler}
for a in (a1, a2):
for func in (f, g):
try:
b = a.map_blocks(func, dtype=a.dtype).compute(**kwargs)
assert b.tolist() == [1, 1, 1, 1]
out = "OK"
except Exception as e:
out = f"{type(e).__name__}: {e}"
print(f"{scheduler}, {func.__name__}, {a.name.split('-')[0]}: {out}")
if __name__ == "__main__":
main() With legacy stack$ conda create -n legacy python=3.6 cython=0.28.1 distributed=1.21.1 dask=0.17.3 numpy=1.14.3 tornado=5.0.2 clang_osx-64
$ conda activate legacy
$ python main.py 2>/dev/null
threads, f, wrapped: OK
threads, g, wrapped: OK
threads, f, array: OK
threads, g, array: OK
distributed, f, wrapped: OK
distributed, g, wrapped: OK
distributed, f, array: ValueError: buffer source array is read-only
distributed, g, array: OK With latest stack$ conda create -n latest python=3.6 cython dask distributed clang_osx-64
$ conda activate latest
$ python main.py 2>/dev/null
threads, f, ones: OK
threads, g, ones: OK
threads, f, array: OK
threads, g, array: OK
distributed, f, ones: OK
distributed, g, ones: OK
distributed, f, array: OK
distributed, g, array: OK |
Thanks for checking in and following up @crusaderky .
…On Fri, Feb 28, 2020 at 6:28 AM crusaderky ***@***.***> wrote:
Closed #1978 <#1978>.
—
You are receiving this because you commented.
Reply to this email directly, view it on GitHub
<#1978?email_source=notifications&email_token=AACKZTHVHWB6BQGFSPURZDLRFENRZA5CNFSM4E7SLBX2YY3PNVWWK3TUL52HS4DFWZEXG43VMVCXMZLOORHG65DJMZUWGYLUNFXW5KTDN5WW2ZLOORPWSZGOW6255WQ#event-3082149594>,
or unsubscribe
<https://github.com/notifications/unsubscribe-auth/AACKZTEPZWGBFLVTHN7NM6LRFENRZANCNFSM4E7SLBXQ>
.
|
I observed this error again today with latest versions of everything. I'm following this old tutorial by @mrocklin and doing something like: df = dd.read_csv("../data/yellow_tripdata_2019-*.csv", parse_dates=["tpep_pickup_datetime", "tpep_dropoff_datetime"])
payment_types = {
1: "Credit Card",
2: "Cash",
3: "No Charge",
4: "Dispute",
5: "Unknown",
6: "Voided trip"
}
payment_names = pd.Series(
payment_types, name="payment_name"
).to_frame()
df2 = df.merge(
payment_names, left_on="payment_type", right_index=True
)
op = df2.groupby("payment_name")["tip_amount"].mean()
client.compute(op2) Interestingly, when I use 2
and this traceback when trying to retrieve the result of the future:
Linux Mint 19.3, Python 3.8.5, fresh |
@astrojuanlu it's possible that's a separate issue. Pandas has had a decent number of issues (pandas-dev/pandas#34857 and linked issues) where arrays end up being readonly and then blow up when they get to a Cython routine that doesn't expect them. Or it could be the same issue reported here. |
Thanks for the blazing fast response @TomAugspurger! I will keep an eye on this. |
At this point, we are allocating |
It seems to work with python 3.7 for me with my own data. I tried latest pandas (1.0.5) and dask (2.19) in both python 3.7 and python 3.8. The error happens just with the python 3.8 version. This test was done with a Windows 7 and a Windows 10 machine working in parallel. Hope this helps. |
Could someone please file a new issue with an MRE? |
My guess is we will want something like PR ( #3918 ). Edit: Would be good if some people could test it and report back over there 🙂 |
That said, I've not managed to get a reproducer together. So am unable to test or develop on this further. @astrojuanlu or @djuarezr, if you are able to supply a reproducer, that would really help us get to some fix here quickly 🙂 |
My attempt to submit a reproducer at #3943, hope it helps! |
Upstream pandas-dev/pandas#34857 was fixed |
Interesting, well it shouldn't be an issue for Dask in either case as we track which buffers are writable and ensure that is preserved when reconstituting objects in deserialization. |
One of the biggest pitfalls of running dask.array with a distributed scheduler is the dreaded
ValueError: buffer source array is read-only
.This error is typical when one runs a memoryview-based Cython kernel in distributed, and it's particularly insidious as it will never show up in unit tests performed with the local dask multithreaded scheduler. It might even not show up when you run your problem on distributed and the arrays just happen to never transit across nodes or to the disk cache (which is exactly what the scheduler will try to achieve if enough RAM and CPU power are available).
In other words, this is a textbook example of an issue that risks appearing in production for the first time!
I can see a few ways of tackling the problem:
On the last point, I personally solved the problem as follows:
In the kernels:
In the unit test:
If you comment out any of those calls to
_memoryview_safe
, the test falls over.Above I'm calling the kernel directly, but a similar thing can also be invoked from the dask wrapper (probably a more robust design).
The text was updated successfully, but these errors were encountered: