Skip to content

Conversation

@hendrikmakait
Copy link
Member

@hendrikmakait hendrikmakait commented Dec 15, 2022

Closes #7400

  • Tests added / passed
  • Passes pre-commit run --all-files

@github-actions
Copy link
Contributor

github-actions bot commented Dec 15, 2022

Unit Test Results

See test report for an extended history of previous test failures. This is useful for diagnosing flaky tests.

       22 files  +       10         22 suites  +10   9h 23m 22s ⏱️ + 5h 3m 45s
  3 276 tests +         5    3 188 ✔️ +       13       86 💤  -     9  2 +1 
35 967 runs  +15 687  34 447 ✔️ +14 937  1 516 💤 +747  4 +3 

For more details on these failures, see this check.

Results for commit 5fb858d. ± Comparison against base commit 3ac8631.

♻️ This comment has been updated with latest results.

@hendrikmakait
Copy link
Member Author

TODO: I still need to improve testing, e.g. by extending test_processing_chain, to ensure that we caught all issues with pandas dtypes.

Comment on lines 711 to 714
# f"col{next(counter)}": pd.array(
# [np.nan, np.nan, 1.0, np.nan, np.nan] * 20,
# dtype="Sparse[float64]",
# ),
Copy link
Member Author

Choose a reason for hiding this comment

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

raises TypeError: Sparse pandas data (column col27) not supported.

Copy link
Member

Choose a reason for hiding this comment

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

f"col{next(counter)}": pd.array(range(100), dtype="float16"),
f"col{next(counter)}": pd.array(range(100), dtype="float32"),
f"col{next(counter)}": pd.array(range(100), dtype="float64"),
# f"col{next(counter)}": pd.array(range(100), dtype="csingle"),
Copy link
Member Author

Choose a reason for hiding this comment

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

raises pyarrow.lib.ArrowNotImplementedError: Unsupported numpy type 14

Copy link
Member

Choose a reason for hiding this comment

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

Looks like arrow doesn't support complex dtypes https://issues.apache.org/jira/browse/ARROW-638

f"col{next(counter)}": pd.array(range(100), dtype="float32"),
f"col{next(counter)}": pd.array(range(100), dtype="float64"),
# f"col{next(counter)}": pd.array(range(100), dtype="csingle"),
# f"col{next(counter)}": pd.array(range(100), dtype="cdouble"),
Copy link
Member Author

Choose a reason for hiding this comment

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

raises pyarrow.lib.ArrowNotImplementedError: Unsupported numpy type 15

f"col{next(counter)}": pd.array(range(100), dtype="float64"),
# f"col{next(counter)}": pd.array(range(100), dtype="csingle"),
# f"col{next(counter)}": pd.array(range(100), dtype="cdouble"),
# f"col{next(counter)}": pd.array(range(100), dtype="clongdouble"),
Copy link
Member Author

Choose a reason for hiding this comment

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

raises pyarrow.lib.ArrowNotImplementedError: Unsupported numpy type 16

@hendrikmakait hendrikmakait marked this pull request as ready for review December 16, 2022 14:34
@hendrikmakait hendrikmakait marked this pull request as draft December 16, 2022 18:36
Copy link
Member

@jrbourbeau jrbourbeau left a comment

Choose a reason for hiding this comment

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

Thanks @hendrikmakait -- I agree that complex, sparse, and object dtypes aren't supported by pyarrow, so raising an informative error message in that case makes sense.

It's be good to also test that pyarrow-backed dtypes also work (e.g. int64[pyarrow], string[pyarrow], etc.)

f"col{next(counter)}": pd.array(range(100), dtype="float16"),
f"col{next(counter)}": pd.array(range(100), dtype="float32"),
f"col{next(counter)}": pd.array(range(100), dtype="float64"),
# f"col{next(counter)}": pd.array(range(100), dtype="csingle"),
Copy link
Member

Choose a reason for hiding this comment

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

Looks like arrow doesn't support complex dtypes https://issues.apache.org/jira/browse/ARROW-638

Comment on lines 711 to 714
# f"col{next(counter)}": pd.array(
# [np.nan, np.nan, 1.0, np.nan, np.nan] * 20,
# dtype="Sparse[float64]",
# ),
Copy link
Member

Choose a reason for hiding this comment

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

Comment on lines +733 to +741
# FIXME: distributed#7420
# f"col{next(counter)}": pd.array(
# ["lorem ipsum"] * 100,
# dtype="string[pyarrow]",
# ),
# f"col{next(counter)}": pd.array(
# ["lorem ipsum"] * 100,
# dtype=pd.StringDtype("pyarrow"),
# ),
Copy link
Member

Choose a reason for hiding this comment

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

My guess is we're running into pandas-dev/pandas#50074 here

Copy link
Member

Choose a reason for hiding this comment

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

Do we have access to, or could easily keep track of, the original input dtypes at the point when we convert the pa.Table to a pd.DataFrame?

Copy link
Member Author

Choose a reason for hiding this comment

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

Left for follow-up work.

@hendrikmakait
Copy link
Member Author

complex, sparse, and object dtypes aren't supported by pyarrow, so raising an informative error message in that case makes sense.

I leave this to a follow-up PR since we currently cast all objects to string[python]. I would like to isolate changes to that logic.

@hendrikmakait hendrikmakait marked this pull request as ready for review December 20, 2022 10:37
@hendrikmakait
Copy link
Member Author

It's be good to also test that pyarrow-backed dtypes also work (e.g. int64[pyarrow], string[pyarrow], etc.)

I added a bunch of arrow-based types, hopefully didn't miss any.

Copy link
Member

@jrbourbeau jrbourbeau left a comment

Choose a reason for hiding this comment

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

Comment on lines 54 to 55
if str(e) == "Tried reading schema message, was null or length 0":
return pa.concat_tables(shards)
Copy link
Member

Choose a reason for hiding this comment

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

Parsing this error message seems brittle. Is there some other check we can do on file to make sure we're read it all?

Copy link
Member Author

Choose a reason for hiding this comment

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

I forgot that we can seek the end of the open file object, I'm using that instead now.

Comment on lines 809 to 810
assert sum(map(len, out.values())) == len(df)
assert all(v.to_pandas().dtypes.equals(df.dtypes) for v in out.values())
Copy link
Member

Choose a reason for hiding this comment

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

(Not meant as a blocking comment) We're checking lengths and dtypes here. Maybe we could just use pd.testing.assert_frame_equal instead?

Copy link
Member Author

Choose a reason for hiding this comment

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

Good point!

hendrikmakait and others added 3 commits December 21, 2022 09:25
Co-authored-by: James Bourbeau <jrbourbeau@users.noreply.github.com>
@hendrikmakait
Copy link
Member Author

I have incorporated all feedback and CI looks good, this is ready for another review.

@jrbourbeau jrbourbeau changed the title Fix P2PShuffle serialization for categorical data Fix P2PShuffle serialization for categorical data Dec 21, 2022
Copy link
Member

@jrbourbeau jrbourbeau left a comment

Choose a reason for hiding this comment

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

Thanks @hendrikmakait!

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.

p2p shuffle - ArrowInvalid error

2 participants