Skip to content

Conversation

@thequackdaddy
Copy link
Contributor

Hello,
#1057

So I don't think divisions are broken for HDF5, but it would be nice to have divisions with the accurate index. This enables that.

Thanks for your consideration.

for i in range(start, stop, chunksize):
kwargs = {'start': i, 'stop': i + 1}
a = _pd_read_hdf(path, key, lock, kwargs)
divisions.append(a.index[0])
Copy link
Member

Choose a reason for hiding this comment

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

Is the index of the HDF dataset guaranteed to be sorted?

Copy link
Member

Choose a reason for hiding this comment

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

Or perhaps we ask the user to make this guarantee?

Also, #955

@thequackdaddy
Copy link
Contributor Author

Thanks for the helpful comments. You really know your stuff.

I've updated my PR with your comments. Let me know if there's anything else I can help with.

simultaneous reads. If a lock object, will use that to acquire locks.
If False, will not use locks.
sorted_index_column : bool or str, default False
If True, use the sorted index as the key. If True, use the column
Copy link
Member

Choose a reason for hiding this comment

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

If True used twice.

Copy link
Member

Choose a reason for hiding this comment

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

Some users might assume that we're sorting the data for them. If there is a concise way to avoid this misconception that would be ideal.

@mrocklin
Copy link
Member

@jcrist brought up a good comment in #955 (comment) about repeated values in the index. We need to find a location where we can cleanly split the dataframes by the division value. Many dask.dataframe algorithms like joins and groupbys assume that the divisions values cleanly divide the dataset.

with tmpfile('h5') as fn:
df.to_hdf(fn, '/data', format='table')
a = dd.read_hdf(fn, '/data', chunksize=2, sorted_index_column='y')
eq(a.npartitions, 2)
Copy link
Member

Choose a reason for hiding this comment

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

We generally only use eq to compare dasky things. I recommend sticking to assert x == y for normal Python values.

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 want to test the values of the dask dataframe. Presumably a should be equal to something like df.set_index('y', drop_index=False) (I haven't verified that, just suggesting things.)

@mrocklin
Copy link
Member

So if we consider the following example:

In [1]: import pandas as pd
In [2]: df = pd.DataFrame({'x': [1, 2, 2, 3], 'y': ['alice', 'bob', 'alice', 'bob']})
In [3]: df
Out[3]: 
   x      y
0  1  alice
1  2    bob
2  2  alice
3  3    bob

In [4]: df.set_index('x', drop=True)
Out[4]: 
       y
x       
1  alice
2    bob
2  alice
3    bob

I think we would probably expect a dask dataframe that, when computed looked like Out[4].

df.to_hdf(...)
ddf = dd.read_hdf(..., sorted_index_column='x')
eq(ddf, df.set_index('x', drop=True))  # do we want to drop by default?  I'm not sure

Although actually maybe we keep the index around. Dropping might be a bad idea, thoughts?

I think we would expect divisions [1, 3, 3] and lengths to match

assert list(ddf.divisions) == [1, 3, 3]
assert ddf.map_partitions(len).compute() == [3, 1]

Not sure that this all is best though, thoughts welcome.

@thequackdaddy
Copy link
Contributor Author

@mrocklin After thinking about this for a second, I'm thinking divisions and index columns can be quite different. To me, an index should unique. Divisions, on the other hand, are just convenient ways we split large datasets into bite sized pieces. Indices don't necessarily have to follow an order or be sorted, but divisions must be. Otherwise, divisions should be None.

Perhaps I change the keyword to sorted_division_column. I think I can make the documentation cleaner that way too.

@mrocklin
Copy link
Member

Correct, if the index is not sorted then divisions should be [None] * (npartitions + 1)

Stepping back a bit, divisions is useful for certain algorithms like joins, groupbys, loc, and datetime managing where, if we know how our data is sorted we can act much much more efficiently. We often have backup algorithms in the case of [None] divisions, but these are much less efficient.

@thequackdaddy
Copy link
Contributor Author

I put some code together that I think should handle unclean splits. I probably needs a few more tests. One caveat, I am defining chunksize to be maximum permissible chunksize. If the chunk is too small to cleanly divide, my code will raise an error.

I imagine there will be some back-and-forth on this.

The other nice things is that once this code makes everyone happy, I'm thinking we could apply this same division logic to read_csv relatively easily.

Thanks for your consideration.

else:
buffer = _pd_read_hdf(path, key, lock, {'chunksize': chunksize,
'start': start,
'stop': stop})
Copy link
Member

Choose a reason for hiding this comment

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

Does this read the entire dataset into ram?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Not per the documentaiton of read_hdf http://pandas.pydata.org/pandas-docs/version/0.17.0/generated/pandas.read_hdf.html this will return an iterator as I provide the chunksize argument. Only 2 chunks (at most) should be loaded into RAM at a time.

@mrocklin
Copy link
Member

So, I think it's important that we be able to construct the dask.dataframe without reading through the full dataset. My suggested approach to wrap a dask dataframe around a single HDF5 dataset would be the following:

  1. Choose a good approximate chunksize (or perhaps arbitrarily set to a million rows or whatever)
  2. Seek to those rows in the HDF dataset to find values for divisions
  3. Walk forward from those rows to find the next transition point, use these locations to determine new start/stops

Example

HDF dataset in ram

a, 1
a, 2
b, 3
b, 3
c, 1
c, 2
c, 3
c, 4
d, 5
e, 6

We'll work with chunksize=3

divisions = [dset[0, index]]
starts = [0]
for i in range(chunksize, len(dset), chunksize):
    while dset[i, index] == dset[i + 1, index]:  # scan forward while in the same value
        i += 1

starts.append(i + 1)
divisions.append(dset[i + 1, index])

This has the advantage of only reading a very small number of bytes when we first create the dask.dataframe. It should complete fairly quickly.

Then we can create the dask.dataframe by slicing using starts and then probably calling pandas.DataFrame.{re,set}index on each partition if there is an index column that is not just the index.

@mrocklin
Copy link
Member

Also, I apologize for my sluggish response on this. It's a very important change and I really appreciate that you're handling it. This week (and part of next) are particularly swamped for me.

@thequackdaddy
Copy link
Contributor Author

No problem. I find this tool amazing, and you are charging an unbeatable price for it.

@mrocklin
Copy link
Member

Well, if we can get this merged in then you'll definitely get a share in the profits :)

@thequackdaddy
Copy link
Contributor Author

Here's attempt number 4. For the first division and last division, I pull 1 record. For the in-between divisions, I pull 2 records and check that those 2 records have the same value in the division column. If so, increment by 1 until they aren't equal.

divisions.append(_div_col(data).iloc[0])
for row in range(start + chunksize, stop, chunksize):
data = _pd_read_hdf(path, key, lock,
{'start': row - 1, 'stop': row + 1})
Copy link
Member

Choose a reason for hiding this comment

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

Does pandas let us access the values directly from the HDF file? How expensive is it to make a new pandas dataframe every time like this? There will be many cases where we need to scan thousands of elements before we find a change.

Also, feel free to not use the internal functions here like _pd_read_hdf , they're a bit janky. I would use the standard Pandas API directly. The internal functions solve annoying problems that come up when computing dask graphs.

@thequackdaddy
Copy link
Contributor Author

I guess what I could do to make this less expensive is a few things...

  1. I can limit the columns to be pulled so that we are only going to pull the column we actually intend to put the divisions on. (I'm not sure how to do this for an index, but it seems trivial enough...)

  2. I can think of 3 ways of reading an HDF file...
    a. pd.read_hdf - What we use now.

    b. pd.HDFStore.select - Another function I've played with. I have no clue if its more/less expensive.

    c. pytables directly - Unfortunately, I've never really played with pytables, so I personally know nothing about it.

You can do a select statement with a where clause in pandas on HDFStores, but the division column must be set as a data_column. So I don't think that's really a good idea.

Thoughs?

@mrocklin
Copy link
Member

The question of "is this expensive" was a genuine question. It might very well be cheap.

@mrocklin
Copy link
Member

Or, in other words. Feel free to ignore many of my comments. Often I'm just thinking out loud.

@thequackdaddy
Copy link
Contributor Author

Oh that's fine. I have a big HDF file at work. Later this week, I'll try to run it with this logic vs without this logic to see what the speed is.

That should get us a nice benchmark.

@mrocklin
Copy link
Member

Great. I should also be more responsive and will have more time to look at this late this week as well.

@thequackdaddy
Copy link
Contributor Author

So a couple days off become 2 weeks sorry... end-of-fiscal year stuff hit and was more problematic than I thought.

So this seems pretty fast to me... I've got a 1.4 GB HDF5 file. It has just one table with 1.2 million rows at 135 columns. The columns are a smorgasbord of floats, ints, str, and datetimes.

In [21]: %timeit ddf.read_hdf('c:/workspace/myfile.hdf5', key='/MyTable', chunksize=10000, sorted_division_column=True)
1 loop, best of 3: 3.87 s per loop

In [22]: %timeit ddf.read_hdf('c:/workspace/myfile.hdf5', key='/MyTable', chunksize=50000, sorted_division_column=True)
1 loop, best of 3: 874 ms per loop

In [23]: %timeit ddf.read_hdf('c:/workspace/myfile.hdf5', key='/MyTable', chunksize=500000, sorted_division_column=True)
1 loop, best of 3: 186 ms per loop

Its maybe a little slow when you have lots of reads... but my first example has well over 100 partitions.

@thequackdaddy
Copy link
Contributor Author

I don't disagree.

I know what my data looks like. You probably have a better handle on what other people's data looks like.

There's probably a better way to do this. I'm new at this, so there's gonna be problems.

@thequackdaddy
Copy link
Contributor Author

thequackdaddy commented Apr 15, 2016

It's fairly common to see datasets with hundreds of thousands of identical values, depending on the column.

Based on this comment, maybe I should figure out a way to get pandas HDFStore select_column feature to work?

@mrocklin
Copy link
Member

I suggest that we move this particular subproblem over to stackoverflow and see if we can't get the broader community so solve it for us. I'll write something up.

@mrocklin
Copy link
Member

Part of this problem would also be useful for other data stores like bcolz. I'm not sure if it's worth generalizing, but it's probably worth thinking about.

def _read_hdf_column(path, key, column, start, rows):
if isinstance(column, string_types):
return pd.HDFStore(path).select(key, start=start, stop=start + rows,
columns=[column])[column].values
Copy link
Contributor

Choose a reason for hiding this comment

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

if you are selecting a single columns, use .select_column, you cannot use a where clauses, but start/stop will work. This will be MUCH faster as the columns kw selects ALL columns then does a reindex (its how PyTables works).

Copy link
Contributor Author

Choose a reason for hiding this comment

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

I think you have to specify those columns as data_columns upon file creation. Is that a safe assumption? (Not really sure what best practice is here.)

Copy link
Contributor

Choose a reason for hiding this comment

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

yes you do, unless it is the index. By definition you cannot index directly a non-data_column. They are stored in blocks otherwise are inacessible in a columnar way.

@nirizr
Copy link
Contributor

nirizr commented Jul 3, 2016

Is there any particular reason this PR is no longer active?
can I help with anything?

@thequackdaddy
Copy link
Contributor Author

Well, the main reason was I was playing with dask to deal with a large-ish dataset I have, but I ended up going with bcolz. Bcolz ended up being much faster for my purposes. I'm probably not going to continue working on this as finding those efficient places to split is not trivial, especially with lots of repeated values and my needs from dask are now satisfied.

@nirizr
Copy link
Contributor

nirizr commented Jul 9, 2016

Thanks for clarifying that, would you object to someone picking this improvement up where you left it and work on it in a different branch and PR?

@thequackdaddy
Copy link
Contributor Author

Not at all. Enjoy.

On Jul 9, 2016, at 9:27 AM, Nir [email protected] wrote:

Thanks for clarifying that, would you object to someone picking this improvement up where you left it and work on it in a different branch and PR?


You are receiving this because you authored the thread.
Reply to this email directly, view it on GitHub, or mute the thread.

@nirizr
Copy link
Contributor

nirizr commented Jul 17, 2016

When reading multiple files (or datasets) for clean split positions, _find_divisions needs to consider adjacent files (or datasets), as the same index value might continue in the next file (or dataset).
I don't think this case was pointed out previously in this discussion.

@mrocklin
Copy link
Member

Good point. I wonder if it would be convenient to make a general method to
take an unclean DataFrame with overlapping partitions and perform the
necessary repartition work to make it clean again. This might be useful in
a few different places.

On Sun, Jul 17, 2016 at 12:43 PM, Nir [email protected] wrote:

When reading multiple files (or datasets) for clean split positions,
_find_divisions needs to consider adjacent files (or datasets), as the
same index value might continue in the next file (or dataset).
I don't think this case was pointed out previously in this discussion.


You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
#1058 (comment), or mute
the thread
https://github.com/notifications/unsubscribe-auth/AASszOXjoiD1RRc08lSLFXGdf1xBT8Q_ks5qWlu-gaJpZM4H0u3o
.

@mrocklin
Copy link
Member

That way we could be fairly dumb in picking out locations from which to
slice the HDF file and then just call this function which would look at the
(possibly improper) divisions, and then call the right loc and concat
calls to split out the errant partitions.

On Mon, Jul 18, 2016 at 9:03 AM, Matthew Rocklin [email protected]
wrote:

Good point. I wonder if it would be convenient to make a general method
to take an unclean DataFrame with overlapping partitions and perform the
necessary repartition work to make it clean again. This might be useful in
a few different places.

On Sun, Jul 17, 2016 at 12:43 PM, Nir [email protected] wrote:

When reading multiple files (or datasets) for clean split positions,
_find_divisions needs to consider adjacent files (or datasets), as the
same index value might continue in the next file (or dataset).
I don't think this case was pointed out previously in this discussion.


You are receiving this because you were mentioned.
Reply to this email directly, view it on GitHub
#1058 (comment), or mute
the thread
https://github.com/notifications/unsubscribe-auth/AASszOXjoiD1RRc08lSLFXGdf1xBT8Q_ks5qWlu-gaJpZM4H0u3o
.

@nirizr
Copy link
Contributor

nirizr commented Jul 18, 2016

Yes, I agree it would be better to separate this solution from read_hdf, but corner case such as reading the next file might be a bit tricky in certain cases and redundant in others.
I'm still trying to wrap my mind around the specifics of this change,, so i'm mostly commenting out issues atm.

@mrocklin
Copy link
Member

My hope is that if we have a function to "heal" slightly overlapping partitions then we can be very dumb about these corner cases and just clean them up afterwards. Maybe this function takes dask.dataframe without known divisions (divisions=[None] * (npartitions + 1)) and a list of known min/maxes for each partition. It then creates divisions normally for the nicely behaving partitions that don't overlap and does a bit of cleanup with loc and concat for the partitions that do overlap, returning a nicely divided dask.dataframe with clean divisions and a bit of stitching together work.

This was referenced Jul 24, 2016
@thequackdaddy
Copy link
Contributor Author

I think this is being handled in another PR. Closing.

@mrocklin
Copy link
Member

@thequackdaddy you might also be interested in the new Parquet support, which would give you division storage by default (as well as column access.)

@thequackdaddy thequackdaddy deleted the hdf_index branch January 11, 2017 22:36
@thequackdaddy
Copy link
Contributor Author

@mrocklin Thanks for the recommendation!

I've heard of Parquet, but haven't played with it.

Honestly, dask+bcolz has been working great for my needs. My data is large-ish (in the 10-20 million rows and 250 columns) but I refuse to use the word "big" to describe it. (And definitely not "bigly").

Bcolz is simple (which is nice) and a little limited (e.g., unicode doesn't work in numexpr, no support for categoricals) but I can rapidly create numpy/pandas dataframe slices from a selection of rows and columns... that that keeps me happy enough.

@mrocklin
Copy link
Member

As you like. Parquet probably won't get to the speed of BColz, but it does support things like text and divisions and it has a pretty similar layout (a bunch of files on disk). It will probably be my recommendation to use Parquet instead of HDF for Dask users going forward.

http://dask.pydata.org/en/latest/dataframe-api.html#dask.dataframe.to_parquet
http://dask.pydata.org/en/latest/dataframe-api.html#dask.dataframe.read_parquet

phofl added a commit to phofl/dask that referenced this pull request Dec 23, 2024
phofl added a commit to phofl/dask that referenced this pull request Dec 23, 2024
* Make column projections stricter (dask#881)

* Simplify again after lowering (dask#884)

* Visual EXPLAIN (dask#885)

* Fix merge predicate pushdowns with weird predicates (dask#888)

* Handle futures that are put into map_partitions (dask#892)

* Remove eager divisions from indexing (dask#891)

* Add shuffle if objects are not aligned and partitions are unknown in assign (dask#887)

Co-authored-by: Hendrik Makait <[email protected]>

* Add support for dd.Aggregation (dask#893)

* Fix random_split for series (dask#894)

* Update dask version

* Use Aggregation from dask/dask (dask#895)

* Fix meta calculation error in groupby (dask#897)

* Revert "Use Aggregation from dask/dask" (dask#898)

* Parquet reader using Pyarrow FileSystem (dask#882)

Co-authored-by: Patrick Hoefler <[email protected]>

* Fix assign for empty indexer (dask#901)

* Add dask.dataframe import at start (dask#903)

* Add indicator support to merge (dask#902)

* Fix detection of parquet filter pushdown (dask#910)

* Speedup init of `ReadParquetPyarrowFS` (dask#909)

* Don't rely on sets in are_co_aligned (dask#908)

* Implement more efficient GroupBy.mean (dask#906)

* Refactor GroupByReduction (dask#920)

* Implement array inference in new_collection (dask#922)

* Add support for convert string option (dask#912)

* P2P shuffle drops partitioning column early (dask#899)

* Avoid culling for SetIndexBlockwise with divisions (dask#925)

* Re-run versioneer install to fix version number (tag_prefix) (dask#926)

* Sort if split_out=1 in value_counts (dask#924)

* Wrap fragments (dask#911)

* Ensure that columns are copied in projection (dask#927)

* Raise in map if pandas < 2.1 (dask#929)

* Add _repr_html_ and updated __repr__ for FrameBase (dask#930)

* Support token for map_partitions (dask#931)

* Fix Copy-on-Write related bug in groupby.transform (dask#933)

* Fix to_dask_dataframe test after switching to dask-expr by default (dask#935)

* Use multi-column assign in groupby apply (dask#934)

* Enable copy on write by default (dask#932)

Co-authored-by: Patrick Hoefler <[email protected]>

* Avoid fusing from_pandas ops to avoid duplicating data (dask#938)

* Adjust automatic split_out parameter (dask#940)

* Revert "Add _repr_html_ and updated __repr__ for FrameBase (dask#930)" (dask#941)

* Remove repartition from P2P shuffle (dask#942)

* [Parquet] Calculate divisions from statistics (dask#917)

* Accept user arguments for arrow_to_pandas (dask#936)

* Add _repr_html_ and prettier __repr__ w/o graph materialization (dask#943)

* Add dask tokenize for fragment wrapper (dask#948)

* Warn if annotations are ignored (dask#947)

* Require `pyarrow>=7` (dask#949)

* Implement string conversion for from_array (dask#950)

* Add dtype and columns type check for shuffle (dask#951)

* Concat arrow tables before converting to pandas (dask#928)

* MINOR: Avoid confusion around shuffle method (dask#956)

Co-authored-by: Patrick Hoefler <[email protected]>

* Set pa cpu count (dask#954)

Co-authored-by: Patrick Hoefler <[email protected]>

* Update for pandas nighlies (dask#953)

* Fix bug with split_out in groupby aggregate (dask#957)

* Fix default observed value (dask#960)

* Ensure that we respect shuffle in context manager (dask#958)

Co-authored-by: Hendrik Makait <[email protected]>

* Fix 'Empty' prefix to non-empty Series repr (dask#963)

* Update README.md (dask#964)

* Adjust split_out values to be consistent with other methods (dask#961)

* bump version to 1.0

* Raise an error if the optimizer cannot terminate (dask#966)

* Fix non-converging optimizer (dask#967)

* Fixup filter pushdown through merges with ands and column reuse (dask#969)

* Fix unique with shuffle and strings (dask#971)

* Implement custom reductions (dask#970)

* Fixup set_index with one partition but more divisions by user (dask#972)

* Fixup predicate pushdown for query 19 (dask#973)

Co-authored-by: Miles <[email protected]>

* Revert enabling pandas cow (dask#974)

* Update changelog for 1.0.2

* Fix set-index preserving divisions for presorted (dask#977)

* Fixup reduction with split_every=False (dask#978)

* Release for dask 2024.3.1

* Raise better error for repartition on divisions with unknown divisions (dask#980)

* Fix concat of series objects with column projection (dask#981)

* Fix some reset_index optimization issues (dask#982)

* Remove keys() (dask#983)

* Ensure wrapping an array when comparing to Series works if columns are empty (dask#984)

* Version v1.0.4

* Visual ANALYZE (dask#889)

Co-authored-by: fjetter <[email protected]>

* Support ``prefix`` argument in  ``from_delayed`` (dask#991)

* Ensure drop matches column names exactly (dask#992)

* Fix SettingWithCopyWarning in _merge.py (dask#990)

* Update pyproject.toml (dask#994)

* Allow passing of boolean index for column index in loc (dask#995)

* Ensure that repr doesn't raise if an operand is a pandas object (dask#996)

* Version v1.0.5

* Reduce coverage target a little bit (dask#999)

* Nicer read_parquet prefix (dask#998)

Co-authored-by: Patrick Hoefler <[email protected]>

* Set divisions with divisions already known (dask#997)

* Start building and publishing conda nightlies (dask#986)

* Fix zero division error when reading index from parquet (dask#1000)

* Rename overloaded `to/from_dask_dataframe` API (dask#987)

* Register json and orc APIs for "pandas" dispatch (dask#1004)

* Fix pyarrow fs reads for list of directories (dask#1006)

* Release for dask 2024.4.0

* Fix meta caclulation in drop_duplicates (dask#1007)

* Release 1.0.7

* Support named aggregations in groupby.aggregate (dask#1009)

* Make release 1.0.9

* Adjust version number in changes

* Make setattr work (dask#1011)

* Release for dask 2024.4.1

* Fix head for npartitions=-1 and optimizer step (dask#1014)

* Deprecate ``to/from_dask_dataframe`` API (dask#1001)

* Fix projection for rename if projection isn't renamed (dask#1016)

* Fix unique with numeric columns (dask#1017)

* Add changes for new version

* Fix column projections in merge when suffixes are relevant (dask#1019)

* Simplify dtype casting logic for shuffle (dask#1012)

* Use implicit knowledge about divisions for efficient grouping (dask#946)

Co-authored-by: Patrick Hoefler <[email protected]>
Co-authored-by: Hendrik Makait <[email protected]>

* Fix assign after set index incorrect projections (dask#1020)

* Fix read_parquet if directory is empty (dask#1023)

* Rename uniuqe_partition_mapping property and add docs (dask#1022)

* Add docs for usefule optimizer methods (dask#1025)

* Fix doc build error (dask#1026)

* Fix error in analyze for scalar (dask#1027)

* Add nr of columns to explain output for projection (dask#1030)

Co-authored-by: Hendrik Makait <[email protected]>

* Fuse more aggressively if parquet files are tiny (dask#1029)

* Move IO docstrings over (dask#1033)

* Release for dask 2024.4.2

* Add cudf support to ``to_datetime`` and ``_maybe_from_pandas`` (dask#1035)

* Fix backend dispatching for `read_csv` (dask#1028)

* Fix loc accessing index for element wise op (dask#1037)

* Fix loc slicing with Datetime Index (dask#1039)

* Fix shuffle after set_index from 1 partition df (dask#1040)

* Bugfix release

* Fix bug in ``Series`` reductions (dask#1041)

* Fix shape returning integer (dask#1043)

* Fix xarray integration with scalar columns (dask#1046)

* Fix None min/max statistics and missing statistics generally (dask#1045)

* Fix drop with set (dask#1047)

* Fix delayed in fusing with multipled dependencies (dask#1038)

* Add bugfix release

* Optimize when from-delayed is called (dask#1048)

* Fix default name conversion in `ToFrame` (dask#1044)

Co-authored-by: Patrick Hoefler <[email protected]>

* Add support for ``DataFrame.melt`` (dask#1049)

* Fixup failing test (dask#1052)

* Generalize ``get_dummies`` (dask#1053)

* reduce pickle size of parquet fragments (dask#1050)

* Add a bunch of docs (dask#1051)

Co-authored-by: Hendrik Makait <[email protected]>

* Release for dask 2024.5.0

* Fix to_parquet in append mode (dask#1057)

* Fix sort_values for unordered categories (dask#1058)

* Fix dropna before merge (dask#1062)

* Fix non-integer divisions in FusedIO (dask#1063)

* Add cache  argument to ``lower_once`` (dask#1059)

* Use ensure_deterministic kwarg instead of config (dask#1064)

* Fix isin with strings (dask#1067)

* Fix isin for head computation (dask#1068)

* Fix read_csv with positional usecols (dask#1069)

* Release for dask 2024.5.1

* Use `is_categorical_dtype` dispatch for `sort_values` (dask#1070)

* Fix meta for string accessors (dask#1071)

* Fix projection to empty from_pandas (dask#1072)

* Release for dask 2024.5.2

* Fix categorize if columns are dropped (dask#1074)

* Fix resample divisions propagation (dask#1075)

* Release for dask 2024.6.0

* Fix get_group for multiple keys (dask#1080)

* Skip distributed tests (dask#1081)

* Fix cumulative aggregations for empty partitions (dask#1082)

* Move another test to distributed folder (dask#1085)

* Release 1.1.4

* Release for dask 2024.6.2

* Add minimal subset of interchange protocol (dask#1087)

* Add from_map docstring (dask#1088)

* Ensure 1 task group per from_delayed (dask#1084)

* Advise against using from_delayed (dask#1089)

* Refactor shuffle method to handle invalid columns (dask#1091)

* Fix freq behavior on  ci (dask#1092)

* Add first array draft (dask#1090)

* Fix array import stuff (dask#1094)

* Add asarray (dask#1095)

* Implement arange (dask#1097)

* Implement linspace (dask#1098)

* Implement zeros and ones (dask#1099)

* Remvoe pandas 2 checks (dask#1100)

* Add unify-chunks draft to arrays (dask#1101)

Co-authored-by: Patrick Hoefler <[email protected]>

* Release for dask 2024.7.0

* Skip test if optional xarray cannot be imported (dask#1104)

* Fix deepcopying FromPandas class (dask#1105)

* Fix from_pandas with chunksize and empty df (dask#1106)

* Link fix in readme (dask#1107)

* Fix shuffle blowing up the task graph (dask#1108)

Co-authored-by: Hendrik Makait <[email protected]>

* Release for dask 2024.7.1

* Fix some things for pandas 3 (dask#1110)

* Fixup remaining upstream failures (dask#1111)

* Release for dask 2024.8.0

* Drop support for Python 3.9 (dask#1109)

Co-authored-by: James Bourbeau <[email protected]>

* Fix tuples as on argument in merge (dask#1117)

* Fix merging when index name in meta missmatches actual name (dask#1119)

Co-authored-by: Hendrik Makait <[email protected]>

* Register `read_parquet` and `read_csv` as "dispatchable" (dask#1114)

* Fix projection for Index class in read_parquet (dask#1120)

* Fix result index of merge (dask#1121)

* Introduce `ToBackend` expression (dask#1115)

* Avoid calling ``array`` attribute on ``cudf.Series`` (dask#1122)

* Make split_out for categorical default smarter (dask#1124)

* Release for dask 2024.8.1

* Fix scalar detection of columns coming from sql (dask#1125)

* Bump `pyarrow>=14.0.1` minimum versions (dask#1127)

Co-authored-by: Patrick Hoefler <[email protected]>

* Fix concat axis 1 bug in divisions (dask#1128)

* Release for dask 2024.8.2

* Use task-based rechunking as default (dask#1131)

* Improve performance of `DelayedsExpr` through caching (dask#1132)

* Import from tokenize (dask#1133)

* Release for dask 2024.9.0

* Add concatenate flag to .compute() (dask#1138)

* Release for dask 2024.9.1

* Fix displaying timestamp scalar (dask#1141)

* Fix alignment issue with groupby index accessors (dask#1142)

* Improve handling of optional dependencies in `analyze` and `explain` (dask#1146)

* Switch from mambaforge to miniforge in CI (dask#1147)

* Fix merge_asof for single partition (dask#1145)

* Raise exception when calculating divisons (dask#1149)

* Fix binary operations with scalar on the left (dask#1150)

* Explicitly list setuptools as a build dependency in conda recipe (dask#1151)

* Version v1.1.16

* Fix ``Merge`` divisions after filtering partitions (dask#1152)

* Fix meta calculation for to_datetime (dask#1153)

* Internal cleanup of P2P code (dask#1154)

* Migrate P2P shuffle and merge to TaskSpec (dask#1155)

* Improve Aggregation docstring explicitly mentionning SeriesGroupBy (dask#1156)

* Migrate shuffle and merge to `P2PBarrierTask` (dask#1157)

* Migrate Blockwise to use taskspec (dask#1159)

* Add support for Python 3.13 (dask#1160)

* Release for dask 2024.11.0

* Fix fusion calling things multiple times (dask#1161)

* Version 1.1.18

* Version 1.1.19

* Fix orphaned dependencies in Fused expression (dask#1163)

* Use Taskspec fuse implementation (dask#1162)

Co-authored-by: Patrick Hoefler <[email protected]>

* Introduce more caching when walking the expression (dask#1165)

* Avoid exponentially growing graph for Assign-Projection combinations (dask#1164)

* Remove ``from_dask_dataframe`` (dask#1167)

* Deprecated and remove from_legacy_dataframe usage (dask#1168)

Co-authored-by: James Bourbeau <[email protected]>

* Remove recursion in task spec (dask#1158)

* Fix value_counts with split_out != 1 (dask#1170)

* Release 2024.12.0

* Use new blockwise unpack collection in array (dask#1173)

* Propagate group_keys in DataFrameGroupBy (dask#1174)

* Fix assign optimization when overwriting columns (dask#1176)

* Remove custom read-csv stuff (dask#1178)

* Fixup install paths (dask#1179)

* Version 1.1.21

* Remove legacy conversion functions (dask#1177)

* Remove duplicated files

* Move repository

* Clean up docs and imports

* Clean up docs and imports

---------

Co-authored-by: Hendrik Makait <[email protected]>
Co-authored-by: Florian Jetter <[email protected]>
Co-authored-by: Miles <[email protected]>
Co-authored-by: Joris Van den Bossche <[email protected]>
Co-authored-by: Richard (Rick) Zamora <[email protected]>
Co-authored-by: Charles Blackmon-Luca <[email protected]>
Co-authored-by: James Bourbeau <[email protected]>
Co-authored-by: alex-rakowski <[email protected]>
Co-authored-by: Matthew Rocklin <[email protected]>
Co-authored-by: Sandro <[email protected]>
Co-authored-by: Ben <[email protected]>
Co-authored-by: James Bourbeau <[email protected]>
Co-authored-by: Guillaume Eynard-Bontemps <[email protected]>
Co-authored-by: Tom Augspurger <[email protected]>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

5 participants