Skip to content

Commit

Permalink
Improve messaging around sorted parquet columns for index (#5265)
Browse files Browse the repository at this point in the history
* Improve messaging around sorted parquet columns for index
  • Loading branch information
martindurant authored and TomAugspurger committed Aug 13, 2019
1 parent ab6acc5 commit 266c314
Show file tree
Hide file tree
Showing 2 changed files with 21 additions and 1 deletion.
6 changes: 5 additions & 1 deletion dask/dataframe/io/parquet/core.py
Expand Up @@ -217,7 +217,11 @@ def read_parquet(
else:
# Multiple sorted columns found, cannot autodetect the index
warnings.warn(
"Multiple sorted columns found, cannot autodetect index",
"Multiple sorted columns found %s, cannot\n "
"autodetect index. Will continue without an index.\n"
"To pick an index column, use the index= keyword; to \n"
"silence this warning use index=False."
"" % [o["name"] for o in out],
RuntimeWarning,
)
index = False
Expand Down
16 changes: 16 additions & 0 deletions dask/dataframe/io/tests/test_parquet.py
Expand Up @@ -1699,6 +1699,22 @@ def test_arrow_partitioning(tmpdir):
ddf.astype({"b": np.float32}).compute()


def test_sorted_warnings(tmpdir, engine):
tmpdir = str(tmpdir)
df = dd.from_pandas(
pd.DataFrame({"cola": range(10), "colb": range(10)}), npartitions=2
)
df.to_parquet(tmpdir, engine=engine, write_index=False)
with pytest.warns(RuntimeWarning) as record:
out = dd.read_parquet(tmpdir, engine=engine)
assert "['cola', 'colb']" in str(record[-1].message)
warnings = len(record)
assert out.columns.tolist() == ["cola", "colb"]
with pytest.warns(None) as record:
dd.read_parquet(tmpdir, engine=engine, index=False)
assert len(record) < warnings # still may have some arrow warnings


def test_informative_error_messages():
with pytest.raises(ValueError) as info:
dd.read_parquet("foo", engine="foo")
Expand Down

0 comments on commit 266c314

Please sign in to comment.