Skip to content

Conversation

@xhochy
Copy link
Member

@xhochy xhochy commented Feb 15, 2019

Minimal reproducing example:

import dask
import pandas as pd
import pyarrow as pa
import numpy as np

def segfault_me(df):
    pa.Table.from_pandas(df, nthreads=1)


while True:
    df = pd.DataFrame(
        {"P": np.arange(0, 10), "L": np.arange(0, 10), "TARGET": np.arange(10, 20)}
    )
    dask.compute([
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
    ])

Segfaults are more likely when run in AddressSanitizer or otherwise slow system with many cores. It is important that always the same df is passed into the functions.

The issue was that the reference count of the underlying NumPy array was increased at the same time by multiple threads. The decrease happend then with a GIL, so the array was sometimes destroyed while still used.

@kszucs
Copy link
Member

kszucs commented Feb 15, 2019

Would You mind adding a test for it?

@xhochy
Copy link
Member Author

xhochy commented Feb 15, 2019

@kszucs I have no idea on how to test this. It's a race condition that occured really seldomly.

@codecov-io
Copy link

Codecov Report

Merging #3655 into master will increase coverage by 0.02%.
The diff coverage is 85.71%.

Impacted file tree graph

@@            Coverage Diff             @@
##           master    #3655      +/-   ##
==========================================
+ Coverage   87.76%   87.79%   +0.02%     
==========================================
  Files         689      689              
  Lines       83984    84014      +30     
  Branches     1081     1081              
==========================================
+ Hits        73712    73759      +47     
+ Misses      10157    10144      -13     
+ Partials      115      111       -4
Impacted Files Coverage Δ
cpp/src/arrow/python/numpy_convert.cc 90.54% <85.71%> (+0.06%) ⬆️
go/arrow/math/uint64_sse4_amd64.go 0% <0%> (-100%) ⬇️
go/arrow/math/float64_sse4_amd64.go 0% <0%> (-100%) ⬇️
go/arrow/memory/memory_sse4_amd64.go 0% <0%> (-100%) ⬇️
go/arrow/math/int64_sse4_amd64.go 0% <0%> (-100%) ⬇️
cpp/src/plasma/thirdparty/ae/ae.c 71.09% <0%> (-0.95%) ⬇️
go/arrow/math/float64_amd64.go 33.33% <0%> (ø) ⬆️
go/arrow/math/int64_amd64.go 33.33% <0%> (ø) ⬆️
go/arrow/math/uint64_amd64.go 33.33% <0%> (ø) ⬆️
cpp/src/arrow/util/bit-util-test.cc 98.88% <0%> (+0.03%) ⬆️
... and 11 more

Continue to review full report at Codecov.

Legend - Click here to learn more
Δ = absolute <relative> (impact), ø = not affected, ? = missing data
Powered by Codecov. Last update bf138a8...7f9838d. Read the comment docs.

@xhochy
Copy link
Member Author

xhochy commented Feb 15, 2019

@xhochy xhochy requested review from pitrou and wesm February 15, 2019 14:48
@wesm
Copy link
Member

wesm commented Feb 15, 2019

@xhochy nice catch!! Taking a closer look

Copy link
Member

@wesm wesm left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1

@wesm wesm closed this in bdbf630 Feb 15, 2019
@wesm wesm deleted the ARROW-4582 branch February 15, 2019 15:47
xhochy added a commit to xhochy/arrow that referenced this pull request Feb 19, 2019
Minimal reproducing example:

```
import dask
import pandas as pd
import pyarrow as pa
import numpy as np

def segfault_me(df):
    pa.Table.from_pandas(df, nthreads=1)

while True:
    df = pd.DataFrame(
        {"P": np.arange(0, 10), "L": np.arange(0, 10), "TARGET": np.arange(10, 20)}
    )
    dask.compute([
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
        dask.delayed(segfault_me)(df),
    ])
```

Segfaults are more likely when run in AddressSanitizer or otherwise slow system with many cores. It is important that always the same df is passed into the functions.

The issue was that the reference count of the underlying NumPy array was increased at the same time by multiple threads. The decrease happend then with a GIL, so the array was sometimes destroyed while still used.

Author: Korn, Uwe <[email protected]>

Closes apache#3655 from xhochy/ARROW-4582 and squashes the following commits:

7f9838d <Korn, Uwe> docker-compose run clang-format
3d6e5ee <Korn, Uwe> ARROW-4582:  Acquire the GIL on Py_INCREF
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

Successfully merging this pull request may close these issues.

4 participants