From b0b8fb0527b5b8060c577d50c27e658ea2f6c194 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 11:08:45 -0800 Subject: [PATCH 01/61] first steps to new sort_values approach --- python/dask_cudf/dask_cudf/batcher_sortnet.py | 184 ++++++++++++++++++ python/dask_cudf/dask_cudf/core.py | 17 +- 2 files changed, 195 insertions(+), 6 deletions(-) diff --git a/python/dask_cudf/dask_cudf/batcher_sortnet.py b/python/dask_cudf/dask_cudf/batcher_sortnet.py index 3cdc6ca4a10..513719306ca 100644 --- a/python/dask_cudf/dask_cudf/batcher_sortnet.py +++ b/python/dask_cudf/dask_cudf/batcher_sortnet.py @@ -3,8 +3,19 @@ Adapted from https://en.wikipedia.org/wiki/Batcher_odd%E2%80%93even_mergesort """ import math +from operator import getitem + +import numpy as np +import toolz from dask import compute, delayed +from dask.base import tokenize +from dask.dataframe.core import DataFrame, _concat +from dask.dataframe.partitionquantiles import partition_quantiles +from dask.dataframe.shuffle import set_partitions_pre, shuffle_group_get +from dask.dataframe.utils import group_split_dispatch +from dask.highlevelgraph import HighLevelGraph +from dask.utils import M, digit, insert import cudf as gd @@ -137,3 +148,176 @@ def sort_delayed_frame(parts, by): valid = compute(valid_ct)[0] validparts = parts[:valid] return validparts + + +def shuffle_group_divs(df, divisions, col, stage, k, npartitions): + dtype = df[col].dtype + c = set_partitions_pre( + df[col], divisions=df._constructor_sliced(divisions, dtype=dtype) + ) + typ = np.min_scalar_type(npartitions * 2) + c = np.mod(c, npartitions).astype(typ, copy=False) + np.floor_divide(c, k ** stage, out=c) + np.mod(c, k, out=c) + return group_split_dispatch(df, c.astype(np.int64), k) + + +def shuffle_group_divs_2(df, divisions, col): + if not len(df): + return {}, df + ind = set_partitions_pre( + df[col], divisions=df._constructor_sliced(divisions) + ).astype(np.int64) + n = ind.max() + 1 + result2 = group_split_dispatch(df, ind.values.view(np.int64), n) + return result2, df.iloc[:0] + + +def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): + npartitions = len(divisions) - 1 + max_branch = max_branch or 32 + n = df.npartitions + + stages = int(math.ceil(math.log(n) / math.log(max_branch))) + if stages > 1: + k = int(math.ceil(n ** (1 / stages))) + else: + k = n + + groups = [] + splits = [] + joins = [] + + inputs = [ + tuple(digit(i, j, k) for j in range(stages)) + for i in range(k ** stages) + ] + + token = tokenize(df, column, max_branch) + + start = { + ("shuffle-join-" + token, 0, inp): (df._name, i) + if i < df.npartitions + else df._meta + for i, inp in enumerate(inputs) + } + + for stage in range(1, stages + 1): + group = { # Convert partition into dict of dataframe pieces + ("shuffle-group-divs-" + token, stage, inp): ( + shuffle_group_divs, + ("shuffle-join-" + token, stage - 1, inp), + divisions, + column, + stage - 1, + k, + n, + ) + for inp in inputs + } + + split = { # Get out each individual dataframe piece from the dicts + ("shuffle-split-" + token, stage, i, inp): ( + getitem, + ("shuffle-group-divs-" + token, stage, inp), + i, + ) + for i in range(k) + for inp in inputs + } + + join = { # concatenate those pieces together, with their friends + ("shuffle-join-" + token, stage, inp): ( + _concat, + [ + ( + "shuffle-split-" + token, + stage, + inp[stage - 1], + insert(inp, stage - 1, j), + ) + for j in range(k) + ], + ) + for inp in inputs + } + groups.append(group) + splits.append(split) + joins.append(join) + + end = { + ("shuffle-" + token, i): ("shuffle-join-" + token, stages, inp) + for i, inp in enumerate(inputs) + } + + dsk = toolz.merge(start, end, *(groups + splits + joins)) + graph = HighLevelGraph.from_collections( + "shuffle-" + token, dsk, dependencies=[df] + ) + df2 = DataFrame(graph, "shuffle-" + token, df, df.divisions) + + if npartitions != df.npartitions: + parts = [i % df.npartitions for i in range(npartitions)] + token = tokenize(df2, npartitions) + + dsk = { + ("repartition-group-" + token, i): ( + shuffle_group_divs_2, + k, + divisions, + column, + ) + for i, k in enumerate(df2.__dask_keys__()) + } + for p in range(npartitions): + dsk[("repartition-get-" + token, p)] = ( + shuffle_group_get, + ("repartition-group-" + token, parts[p]), + p, + ) + + graph2 = HighLevelGraph.from_collections( + "repartition-get-" + token, dsk, dependencies=[df2] + ) + df3 = DataFrame( + graph2, "repartition-get-" + token, df2, [None] * (npartitions + 1) + ) + else: + df3 = df2 + df3.divisions = (None,) * (df.npartitions + 1) + + return df3 + + +def sort_values_new(df, by, ignore_index=False): + + if isinstance(by, str): + by = [by] + elif isinstance(by, tuple): + by = list(by) + + # Only handle single column (for now) + # Note: How can we map multiple columns onto + # a single `partitions` column? + if len(by) > 1: + return df.sort_values(by, ignore_index=ignore_index, legacy=True) + index = by[0] + + # Step 1 - Pre-sort each partition + df2 = df.map_partitions(M.sort_values, index) + + # Step 2 - Calculate new divisions + npartitions = df.npartitions + divisions = ( + partition_quantiles(df2[index], npartitions, upsample=1.0) + .compute() + .to_list() + ) + + # Step 3 - Perform shuffle + df3 = rearrange_by_divisions(df2, index, divisions) + df3.divisions = (None,) * (npartitions + 1) + + # Step 4 - Return final sorted df + # (No sort needed after k-way merging parts) + return df3.map_partitions(M.sort_values, index) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 2c950967a59..784272e52f9 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -202,18 +202,23 @@ def fix_index(df, startpos): else: return self.map_partitions(M.reset_index, drop=drop) - def sort_values(self, by, ignore_index=False): + def sort_values(self, by, ignore_index=False, legacy=True): """Sort by the given column Parameter --------- by : str """ - parts = self.to_delayed() - sorted_parts = batcher_sortnet.sort_delayed_frame(parts, by) - return from_delayed(sorted_parts, meta=self._meta).reset_index( - force=not ignore_index - ) + if legacy: + parts = self.to_delayed() + sorted_parts = batcher_sortnet.sort_delayed_frame(parts, by) + return from_delayed(sorted_parts, meta=self._meta).reset_index( + force=not ignore_index + ) + else: + return batcher_sortnet.sort_values_new( + self, by, ignore_index=ignore_index + ) def sort_values_binned(self, by): """Sorty by the given column and ensure that the same key From 26f69253237dcdf9f6402827e5bfdc4899f35e57 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 12:36:01 -0800 Subject: [PATCH 02/61] further renaming and reorg for sort_values_experimental --- python/dask_cudf/dask_cudf/core.py | 29 +++++++---- .../{batcher_sortnet.py => sorting.py} | 51 ++++++++++++------- 2 files changed, 51 insertions(+), 29 deletions(-) rename python/dask_cudf/dask_cudf/{batcher_sortnet.py => sorting.py} (91%) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 784272e52f9..3b51219de92 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -1,4 +1,4 @@ -# Copyright (c) 2018, NVIDIA CORPORATION. +# Copyright (c) 2020, NVIDIA CORPORATION. import warnings import numpy as np @@ -23,7 +23,7 @@ import cudf import cudf._lib as libcudf -from dask_cudf import batcher_sortnet +from dask_cudf import sorting from dask_cudf.accessor import ( CachedAccessor, CategoricalAccessor, @@ -202,23 +202,30 @@ def fix_index(df, startpos): else: return self.map_partitions(M.reset_index, drop=drop) - def sort_values(self, by, ignore_index=False, legacy=True): + def sort_values(self, by, ignore_index=False, experimental=False): """Sort by the given column Parameter --------- by : str """ - if legacy: - parts = self.to_delayed() - sorted_parts = batcher_sortnet.sort_delayed_frame(parts, by) - return from_delayed(sorted_parts, meta=self._meta).reset_index( - force=not ignore_index - ) - else: - return batcher_sortnet.sort_values_new( + if experimental: + # Experimental aglorithm (mostly) matches + # the set_index sorting procedure used in Dask. + # Note that, if len(by)>1, only the first column + # is used for repartitioning. All columns are + # used for intra-partition sorting. + df = sorting.sort_values_experimental( self, by, ignore_index=ignore_index ) + if ignore_index: + return df.reset_index(drop=True) + else: + # Legacy sorting algorithm based on "batcher-sortnet" + parts = self.to_delayed() + sorted_parts = sorting.sort_delayed_frame(parts, by) + df = from_delayed(sorted_parts, meta=self._meta) + return df def sort_values_binned(self, by): """Sorty by the given column and ensure that the same key diff --git a/python/dask_cudf/dask_cudf/batcher_sortnet.py b/python/dask_cudf/dask_cudf/sorting.py similarity index 91% rename from python/dask_cudf/dask_cudf/batcher_sortnet.py rename to python/dask_cudf/dask_cudf/sorting.py index 513719306ca..78e27531215 100644 --- a/python/dask_cudf/dask_cudf/batcher_sortnet.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -1,8 +1,6 @@ -""" -Batcher's Odd-even sorting network -Adapted from https://en.wikipedia.org/wiki/Batcher_odd%E2%80%93even_mergesort -""" +# Copyright (c) 2020, NVIDIA CORPORATION. import math +import warnings from operator import getitem import numpy as np @@ -11,7 +9,6 @@ from dask import compute, delayed from dask.base import tokenize from dask.dataframe.core import DataFrame, _concat -from dask.dataframe.partitionquantiles import partition_quantiles from dask.dataframe.shuffle import set_partitions_pre, shuffle_group_get from dask.dataframe.utils import group_split_dispatch from dask.highlevelgraph import HighLevelGraph @@ -20,6 +17,12 @@ import cudf as gd +""" +Batcher's Odd-even sorting network +Adapted from https://en.wikipedia.org/wiki/Batcher_odd%E2%80%93even_mergesort +""" + + def get_oversized(length): """ The oddeven network requires a power-of-2 length. @@ -289,35 +292,47 @@ def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): return df3 -def sort_values_new(df, by, ignore_index=False): +def sort_values_experimental(df, by, ignore_index=False): + """ Experimental sort_values implementation. + Sort by the given column name or list/tuple of column names. + + Parameter + --------- + by : list, tuple, str + """ + npartitions = df.npartitions if isinstance(by, str): by = [by] elif isinstance(by, tuple): by = list(by) - # Only handle single column (for now) - # Note: How can we map multiple columns onto - # a single `partitions` column? + # Step 1 - Pre-sort each partition + df2 = df.map_partitions(M.sort_values, by) + if npartitions == 1: + return df2 + + # Only handle single-column partitioning (for now) + # TODO: Handle partitioning on multiple columns? if len(by) > 1: - return df.sort_values(by, ignore_index=ignore_index, legacy=True) + warnings.warn( + "Using experimental version of sort_values." + " Only `by[0]` will be used for partitioning." + ) index = by[0] - # Step 1 - Pre-sort each partition - df2 = df.map_partitions(M.sort_values, index) - # Step 2 - Calculate new divisions - npartitions = df.npartitions divisions = ( - partition_quantiles(df2[index], npartitions, upsample=1.0) + df2[index] + ._repartition_quantiles(npartitions, upsample=1.0) .compute() .to_list() ) - # Step 3 - Perform shuffle + # Step 3 - Perform repartitioning shuffle df3 = rearrange_by_divisions(df2, index, divisions) df3.divisions = (None,) * (npartitions + 1) # Step 4 - Return final sorted df - # (No sort needed after k-way merging parts) - return df3.map_partitions(M.sort_values, index) + # (Can remove after k-way merge added) + return df3.map_partitions(M.sort_values, by) From 00eb15d849e6c5f8316c2504626d3e042b317395 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 12:53:15 -0800 Subject: [PATCH 03/61] test corrections --- python/dask_cudf/dask_cudf/core.py | 4 ++-- .../dask_cudf/tests/test_batcher_sortnet.py | 16 ++++++++-------- python/dask_cudf/dask_cudf/tests/test_sort.py | 4 +++- 3 files changed, 13 insertions(+), 11 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 3b51219de92..3ba31881a15 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -218,13 +218,13 @@ def sort_values(self, by, ignore_index=False, experimental=False): df = sorting.sort_values_experimental( self, by, ignore_index=ignore_index ) - if ignore_index: - return df.reset_index(drop=True) else: # Legacy sorting algorithm based on "batcher-sortnet" parts = self.to_delayed() sorted_parts = sorting.sort_delayed_frame(parts, by) df = from_delayed(sorted_parts, meta=self._meta) + if ignore_index: + return df.reset_index() return df def sort_values_binned(self, by): diff --git a/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py b/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py index 317c6d88cd4..7cbd06084ea 100644 --- a/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py +++ b/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py @@ -3,15 +3,15 @@ import cudf -from dask_cudf import batcher_sortnet +from dask_cudf import sorting @pytest.mark.parametrize("n", list(range(1, 40))) def test_padding(n): data = list(range(n)) - padded, valid = batcher_sortnet._pad_data_to_length(data) + padded, valid = sorting._pad_data_to_length(data) assert len(data) == valid - assert batcher_sortnet.is_power_of_2(len(padded)) + assert sorting.is_power_of_2(len(padded)) assert valid > len(padded) / 2 assert all(x is not None for x in padded[:valid]) assert all(x is None for x in padded[valid:]) @@ -33,7 +33,7 @@ def test_compare_frame(seed, nelem): rhs["b"] = rhs_b = np.random.random(nelem) # Sort by column "a" - got_a = batcher_sortnet._compare_frame(lhs, rhs, max_part_size, by="a") + got_a = sorting._compare_frame(lhs, rhs, max_part_size, by="a") # Check expect_a = np.hstack([lhs_a, rhs_a]) expect_a.sort() @@ -41,7 +41,7 @@ def test_compare_frame(seed, nelem): np.testing.assert_array_equal(got_a[1].a.to_array(), expect_a[nelem:]) # Sort by column "b" - got_b = batcher_sortnet._compare_frame(lhs, rhs, max_part_size, by="b") + got_b = sorting._compare_frame(lhs, rhs, max_part_size, by="b") # Check expect_b = np.hstack([lhs_b, rhs_b]) expect_b.sort() @@ -53,9 +53,9 @@ def test_compare_frame_with_none(): df = cudf.DataFrame() max_part_size = 1 df["a"] = [0] - res = batcher_sortnet._compare_frame(df, None, max_part_size, by="a") + res = sorting._compare_frame(df, None, max_part_size, by="a") assert res[0] is not None, res[1] is None - res = batcher_sortnet._compare_frame(None, df, max_part_size, by="a") + res = sorting._compare_frame(None, df, max_part_size, by="a") assert res[0] is not None, res[1] is None - res = batcher_sortnet._compare_frame(None, None, max_part_size, by="a") + res = sorting._compare_frame(None, None, max_part_size, by="a") assert res == (None, None) diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index c2e20f20601..fe1ef83e5d1 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -18,7 +18,9 @@ def test_sort_values(nelem, nparts, by): ddf = dd.from_pandas(df, npartitions=nparts) with dask.config.set(scheduler="single-threaded"): - got = ddf.sort_values(by=by).compute().to_pandas() + got = ( + ddf.sort_values(by=by).compute().to_pandas().reset_index(drop=True) + ) expect = df.sort_values(by=by).to_pandas().reset_index(drop=True) pd.util.testing.assert_frame_equal(got, expect) From 06d48ce5837b312fca95fb63121e23218502b58c Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 18:42:07 -0800 Subject: [PATCH 04/61] adding explicit comms support --- python/dask_cudf/dask_cudf/core.py | 6 +- .../dask_cudf/dask_cudf/explicit_shuffle.py | 124 ++++++++++++++++++ python/dask_cudf/dask_cudf/sorting.py | 23 +++- 3 files changed, 148 insertions(+), 5 deletions(-) create mode 100644 python/dask_cudf/dask_cudf/explicit_shuffle.py diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 6f032bdfd42..5dae328eaa9 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -202,7 +202,9 @@ def fix_index(df, startpos): else: return self.map_partitions(M.reset_index, drop=drop) - def sort_values(self, by, ignore_index=False, experimental=False): + def sort_values( + self, by, ignore_index=False, experimental=False, **kwargs + ): """Sort by the given column Parameter @@ -219,7 +221,7 @@ def sort_values(self, by, ignore_index=False, experimental=False): # is used for repartitioning. All columns are # used for intra-partition sorting. df = sorting.sort_values_experimental( - self, by, ignore_index=ignore_index + self, by, ignore_index=ignore_index, **kwargs ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py new file mode 100644 index 00000000000..895fd20a910 --- /dev/null +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -0,0 +1,124 @@ +import asyncio + +from distributed.protocol import to_serialize + +import cudf + +from dask_cudf.explicit_comms import comms + + +async def send_df(ep, df): + if df is None: + return await ep.write("empty") + else: + return await ep.write([to_serialize(df)]) + + +async def recv_df(ep): + ret = await ep.read() + if ret == "empty": + return None + else: + return ret[0] + + +async def send_parts(eps, parts): + futures = [] + for rank, ep in eps.items(): + futures.append(send_df(ep, parts[rank])) + await asyncio.gather(*futures) + + +async def recv_parts(eps, parts): + futures = [] + for ep in eps.values(): + futures.append(recv_df(ep)) + parts.extend(await asyncio.gather(*futures)) + + +async def exchange_and_concat_parts(rank, eps, parts, sort=False): + ret = [parts[rank]] + await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) + df = concat(list(filter(None, ret))) + if sort: + return df.sort_values(sort) + return df + + +def concat(df_list): + if len(df_list) == 0: + return None + return cudf.concat(df_list) + + +def partition_by_column(df, column, n_chunks): + if df is None: + return [None] * n_chunks + elif hasattr(df, "scatter_by_map"): + return df.scatter_by_map(column, map_size=n_chunks) + else: + raise NotImplementedError( + "partition_by_column not yet implemented for pandas backend.\n" + ) + + +async def distributed_shuffle( + n_chunks, rank, eps, table, partitions, index, sorted_split +): + if sorted_split: + parts = [ + table.iloc[partitions[i] : partitions[i + 1]] + for i in range(0, len(partitions) - 1) + ] + else: + parts = partition_by_column(table, partitions, n_chunks) + return await exchange_and_concat_parts(rank, eps, parts, sort=index) + + +async def _aggregated_shuffle( + s, df_nparts, df_parts, index, divisions, sorted_split +): + def df_concat(df_parts): + """Making sure df_parts is a single dataframe or None""" + if len(df_parts) == 0: + return None + elif len(df_parts) == 1: + return df_parts[0] + else: + return concat(df_parts) + + # Concatenate all parts owned by this worker into + # a single cudf DataFrame + df = df_concat(df_parts[0]) + + divisions = cudf.Series(divisions) + if sorted_split: + # Avoid `scatter_by_map` by sorting the dataframe here + # (Can just use iloc to split into groups) + if len(df_parts) > 1: + # Need to sort again after concatenation + df = df.sort_values(index) + splits = df[index].searchsorted(divisions, side="left") + splits[-1] = len(df[index]) + partitions = splits.tolist() + else: + partitions = divisions.searchsorted(df[index], side="right") - 1 + partitions[(df[index] >= divisions.iloc[-1]).values] = ( + len(divisions) - 2 + ) + + # Run distributed shuffle and set_index algorithm + return await distributed_shuffle( + s["nworkers"], s["rank"], s["eps"], df, partitions, index, sorted_split + ) + + +def explicit_shuffle(df, index, divisions, sorted_split=False, **kwargs): + + # Explict-comms shuffle and local set_index + # TODO: Fast repartition back to df.npartitions using views... + return comms.default_comms().dataframe_operation( + _aggregated_shuffle, + df_list=(df,), + extra_args=(index, divisions, sorted_split), + ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 78e27531215..0e19921c384 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -16,6 +16,13 @@ import cudf as gd +try: + from .explicit_sorting import explicit_shuffle + + explicit_comms = True +except ImportError: + explicit_comms = False + """ Batcher's Odd-even sorting network @@ -292,7 +299,9 @@ def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): return df3 -def sort_values_experimental(df, by, ignore_index=False): +def sort_values_experimental( + df, by, ignore_index=False, client=None, explicit_workers=None +): """ Experimental sort_values implementation. Sort by the given column name or list/tuple of column names. @@ -321,6 +330,11 @@ def sort_values_experimental(df, by, ignore_index=False): ) index = by[0] + # Check if we are using explicit comms + use_explicit = explicit_comms and explicit_workers and client + if use_explicit: + npartitions = explicit_workers + # Step 2 - Calculate new divisions divisions = ( df2[index] @@ -330,8 +344,11 @@ def sort_values_experimental(df, by, ignore_index=False): ) # Step 3 - Perform repartitioning shuffle - df3 = rearrange_by_divisions(df2, index, divisions) - df3.divisions = (None,) * (npartitions + 1) + if use_explicit: + df3 = explicit_shuffle(df2, index, divisions, sorted_split=False) + else: + df3 = rearrange_by_divisions(df2, index, divisions) + df3.divisions = (None,) * (npartitions + 1) # Step 4 - Return final sorted df # (Can remove after k-way merge added) From 23ce44aa997053f169a14ccef874b0874f343790 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 19:01:28 -0800 Subject: [PATCH 05/61] simple sort_values working for dynamic and explicit comms --- python/dask_cudf/dask_cudf/sorting.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 0e19921c384..783814a953f 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -300,7 +300,7 @@ def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): def sort_values_experimental( - df, by, ignore_index=False, client=None, explicit_workers=None + df, by, ignore_index=False, client=None, explicit=False ): """ Experimental sort_values implementation. @@ -331,9 +331,9 @@ def sort_values_experimental( index = by[0] # Check if we are using explicit comms - use_explicit = explicit_comms and explicit_workers and client + use_explicit = explicit_comms and explicit and client if use_explicit: - npartitions = explicit_workers + npartitions = len(client.cluster.workers) # Step 2 - Calculate new divisions divisions = ( @@ -345,10 +345,11 @@ def sort_values_experimental( # Step 3 - Perform repartitioning shuffle if use_explicit: + warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_shuffle(df2, index, divisions, sorted_split=False) else: df3 = rearrange_by_divisions(df2, index, divisions) - df3.divisions = (None,) * (npartitions + 1) + df3.divisions = (None,) * (npartitions + 1) # Step 4 - Return final sorted df # (Can remove after k-way merge added) From 7ec0cf2afe0f66fb741d1b14ddeea80ab741c56c Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 19:23:10 -0800 Subject: [PATCH 06/61] cleanup --- .../dask_cudf/dask_cudf/explicit_shuffle.py | 31 ++++++++++++------- python/dask_cudf/dask_cudf/sorting.py | 16 ++++++---- 2 files changed, 30 insertions(+), 17 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 895fd20a910..2268c3d7c0d 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -63,7 +63,7 @@ def partition_by_column(df, column, n_chunks): async def distributed_shuffle( - n_chunks, rank, eps, table, partitions, index, sorted_split + n_chunks, rank, eps, table, partitions, index, sort_by, sorted_split ): if sorted_split: parts = [ @@ -72,11 +72,11 @@ async def distributed_shuffle( ] else: parts = partition_by_column(table, partitions, n_chunks) - return await exchange_and_concat_parts(rank, eps, parts, sort=index) + return await exchange_and_concat_parts(rank, eps, parts, sort=sort_by) -async def _aggregated_shuffle( - s, df_nparts, df_parts, index, divisions, sorted_split +async def _explicit_shuffle( + s, df_nparts, df_parts, index, divisions, sort_by, sorted_split ): def df_concat(df_parts): """Making sure df_parts is a single dataframe or None""" @@ -97,7 +97,7 @@ def df_concat(df_parts): # (Can just use iloc to split into groups) if len(df_parts) > 1: # Need to sort again after concatenation - df = df.sort_values(index) + df = df.sort_values(sort_by) splits = df[index].searchsorted(divisions, side="left") splits[-1] = len(df[index]) partitions = splits.tolist() @@ -109,16 +109,25 @@ def df_concat(df_parts): # Run distributed shuffle and set_index algorithm return await distributed_shuffle( - s["nworkers"], s["rank"], s["eps"], df, partitions, index, sorted_split + s["nworkers"], + s["rank"], + s["eps"], + df, + partitions, + index, + sort_by, + sorted_split, ) -def explicit_shuffle(df, index, divisions, sorted_split=False, **kwargs): - - # Explict-comms shuffle and local set_index +def explicit_sorted_shuffle( + df, index, divisions, sort_by, sorted_split=False, **kwargs +): + # Explict-comms shuffle # TODO: Fast repartition back to df.npartitions using views... + df.persist() return comms.default_comms().dataframe_operation( - _aggregated_shuffle, + _explicit_shuffle, df_list=(df,), - extra_args=(index, divisions, sorted_split), + extra_args=(index, divisions, sort_by, sorted_split), ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 783814a953f..0b022b4ad5f 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -17,7 +17,7 @@ import cudf as gd try: - from .explicit_sorting import explicit_shuffle + from .explicit_sorting import explicit_sorted_shuffle explicit_comms = True except ImportError: @@ -346,11 +346,15 @@ def sort_values_experimental( # Step 3 - Perform repartitioning shuffle if use_explicit: warnings.warn("Using explicit comms - This is an advanced feature.") - df3 = explicit_shuffle(df2, index, divisions, sorted_split=False) + df3 = explicit_sorted_shuffle( + df2, index, divisions, sort_by=by, sorted_split=False + ) + df3.divisions = (None,) * (npartitions + 1) + return df3 else: df3 = rearrange_by_divisions(df2, index, divisions) - df3.divisions = (None,) * (npartitions + 1) + df3.divisions = (None,) * (npartitions + 1) - # Step 4 - Return final sorted df - # (Can remove after k-way merge added) - return df3.map_partitions(M.sort_values, by) + # Step 4 - Return final sorted df + # (Can remove after k-way merge added) + return df3.map_partitions(M.sort_values, by) From 66729b5d611bcbf3e69c9d892056425fecad4d2b Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 29 Jan 2020 19:56:56 -0800 Subject: [PATCH 07/61] fix import typo --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 3 +-- python/dask_cudf/dask_cudf/sorting.py | 2 +- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 2268c3d7c0d..eea43f204df 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -1,11 +1,10 @@ import asyncio +from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize import cudf -from dask_cudf.explicit_comms import comms - async def send_df(ep, df): if df is None: diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 0b022b4ad5f..d62859bdb7a 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -17,7 +17,7 @@ import cudf as gd try: - from .explicit_sorting import explicit_sorted_shuffle + from .explicit_shuffle import explicit_sorted_shuffle explicit_comms = True except ImportError: From f6328b509bc4669aaa104c1c1fa6bf98e6bfcf41 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 30 Jan 2020 08:23:25 -0800 Subject: [PATCH 08/61] add experimental set_index option (which wraps experimental sort_values code path) --- python/dask_cudf/dask_cudf/core.py | 20 +++++++++++++++++++- python/dask_cudf/dask_cudf/sorting.py | 12 +++++++++--- 2 files changed, 28 insertions(+), 4 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 5dae328eaa9..c4ac44bb79a 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -171,7 +171,25 @@ def join(self, other, **kwargs): on = list(on) return super().join(other, how=how, on=on, shuffle="tasks", **kwargs) - def set_index(self, other, **kwargs): + def set_index( + self, + other, + experimental=False, + sorted=False, + client=None, + explicit=False, + **kwargs, + ): + if experimental and not sorted: + # Use sort values for the shuffle + df = self.sort_values( + [other], + experimental=experimental, + client=client, + explicit=explicit, + ) + # Use sorted=True with usual `set_index` API + return df.map_partitions(M.set_index, other) if kwargs.pop("shuffle", "tasks") != "tasks": raise ValueError( "Dask-cudf only supports task based shuffling, got %s" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index d62859bdb7a..8d7c2a6fce8 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -316,10 +316,16 @@ def sort_values_experimental( elif isinstance(by, tuple): by = list(by) + # Make sure first column is numeric + # (Cannot handle string column here yet) + if isinstance(df[by[0]]._meta._column, gd.core.column.string.StringColumn): + return df.sort_values( + by, ignore_index=ignore_index, experimental=False + ) + # Step 1 - Pre-sort each partition - df2 = df.map_partitions(M.sort_values, by) - if npartitions == 1: - return df2 + # df2 = df.map_partitions(M.sort_values, by) + df2 = df # TODO: Use Presort when k-way merge/concat is avilable. # Only handle single-column partitioning (for now) # TODO: Handle partitioning on multiple columns? From 024a1ce52bb22a4c3b75bcd0eebc4be70d616fb6 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 30 Jan 2020 08:31:09 -0800 Subject: [PATCH 09/61] add repartitioning to experimental set_index --- python/dask_cudf/dask_cudf/core.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index c4ac44bb79a..6176f9cad77 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -188,8 +188,11 @@ def set_index( client=client, explicit=explicit, ) - # Use sorted=True with usual `set_index` API - return df.map_partitions(M.set_index, other) + # Set index and repartition + npartitions = kwargs.get("npartitions", self.npartitions) + return df.map_partitions(M.set_index, other).repartition( + npartitions=npartitions + ) if kwargs.pop("shuffle", "tasks") != "tasks": raise ValueError( "Dask-cudf only supports task based shuffling, got %s" From 16b25a5e1cb2d02187e78a2dc0a3d336637921eb Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 30 Jan 2020 12:29:11 -0800 Subject: [PATCH 10/61] use explicit_client kwarg instead of both client and explicit --- python/dask_cudf/dask_cudf/core.py | 18 ++++++++++++------ python/dask_cudf/dask_cudf/sorting.py | 8 +++----- 2 files changed, 15 insertions(+), 11 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 6176f9cad77..4cae33cf3b9 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -176,8 +176,7 @@ def set_index( other, experimental=False, sorted=False, - client=None, - explicit=False, + explicit_client=None, **kwargs, ): if experimental and not sorted: @@ -185,8 +184,7 @@ def set_index( df = self.sort_values( [other], experimental=experimental, - client=client, - explicit=explicit, + explicit_client=explicit_client, ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) @@ -224,7 +222,12 @@ def fix_index(df, startpos): return self.map_partitions(M.reset_index, drop=drop) def sort_values( - self, by, ignore_index=False, experimental=False, **kwargs + self, + by, + ignore_index=False, + experimental=False, + explicit_client=None, + **kwargs, ): """Sort by the given column @@ -242,7 +245,10 @@ def sort_values( # is used for repartitioning. All columns are # used for intra-partition sorting. df = sorting.sort_values_experimental( - self, by, ignore_index=ignore_index, **kwargs + self, + by, + ignore_index=ignore_index, + explicit_client=explicit_client, ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 8d7c2a6fce8..2e93ced4126 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -299,9 +299,7 @@ def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): return df3 -def sort_values_experimental( - df, by, ignore_index=False, client=None, explicit=False -): +def sort_values_experimental(df, by, ignore_index=False, explicit_client=None): """ Experimental sort_values implementation. Sort by the given column name or list/tuple of column names. @@ -337,9 +335,9 @@ def sort_values_experimental( index = by[0] # Check if we are using explicit comms - use_explicit = explicit_comms and explicit and client + use_explicit = explicit_comms and explicit_client if use_explicit: - npartitions = len(client.cluster.workers) + npartitions = len(explicit_client.cluster.workers) # Step 2 - Calculate new divisions divisions = ( From 0612bef3291649089cf5965f38dc6153d0bc6812 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 30 Jan 2020 12:59:11 -0800 Subject: [PATCH 11/61] add test coverage for non-explicit code path (not sure how to test explicit algorithm here) --- python/dask_cudf/dask_cudf/core.py | 4 +++- python/dask_cudf/dask_cudf/tests/test_core.py | 5 +++-- python/dask_cudf/dask_cudf/tests/test_sort.py | 12 +++++------- 3 files changed, 11 insertions(+), 10 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 4cae33cf3b9..46a1799c148 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -196,7 +196,9 @@ def set_index( "Dask-cudf only supports task based shuffling, got %s" % kwargs["shuffle"] ) - return super().set_index(other, shuffle="tasks", **kwargs) + return super().set_index( + other, sorted=sorted, shuffle="tasks", **kwargs + ) def reset_index(self, force=False, drop=False): """Reset index to range based diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index d21b8368740..29eac70a4bf 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -122,7 +122,8 @@ def test_from_dask_dataframe(): @pytest.mark.parametrize("nelem", [10, 200, 1333]) -def test_set_index(nelem): +@pytest.mark.parametrize("experimental", [True, False]) +def test_set_index(nelem, experimental): with dask.config.set(scheduler="single-threaded"): np.random.seed(0) # Use unique index range as the sort may not be stable-ordering @@ -135,7 +136,7 @@ def test_set_index(nelem): dgdf = ddf.map_partitions(cudf.from_pandas) expect = ddf.set_index("x") - got = dgdf.set_index("x") + got = dgdf.set_index("x", experimental=experimental) dd.assert_eq(expect, got, check_index=False, check_divisions=False) diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index 7e63f7a7e87..8235bb905e9 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -1,5 +1,4 @@ import numpy as np -import pandas as pd import pytest import dask @@ -11,18 +10,17 @@ @pytest.mark.parametrize("by", ["a", "b", ["a", "b"]]) @pytest.mark.parametrize("nelem", [10, 100, 1000]) @pytest.mark.parametrize("nparts", [1, 2, 5, 10]) -def test_sort_values(nelem, nparts, by): +@pytest.mark.parametrize("experimental", [True, False]) +def test_sort_values(nelem, nparts, by, experimental): df = cudf.DataFrame() df["a"] = np.ascontiguousarray(np.arange(nelem)[::-1]) df["b"] = np.arange(100, nelem + 100) ddf = dd.from_pandas(df, npartitions=nparts) with dask.config.set(scheduler="single-threaded"): - got = ( - ddf.sort_values(by=by).compute().to_pandas().reset_index(drop=True) - ) - expect = df.sort_values(by=by).to_pandas().reset_index(drop=True) - pd.util.testing.assert_frame_equal(got, expect) + got = ddf.sort_values(by=by, experimental=experimental) + expect = df.sort_values(by=by) + dd.assert_eq(got, expect, check_index=False) def test_sort_values_binned(): From ac4acc4f143dad4037b871764cd872fdebcc3a87 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 30 Jan 2020 14:23:23 -0800 Subject: [PATCH 12/61] expose max_branch kwarg for experimental set_index/sort_values --- python/dask_cudf/dask_cudf/core.py | 4 ++++ python/dask_cudf/dask_cudf/sorting.py | 12 +++++++++--- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 46a1799c148..2e503cc1700 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -177,6 +177,7 @@ def set_index( experimental=False, sorted=False, explicit_client=None, + max_branch=None, **kwargs, ): if experimental and not sorted: @@ -185,6 +186,7 @@ def set_index( [other], experimental=experimental, explicit_client=explicit_client, + max_branch=max_branch, ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) @@ -229,6 +231,7 @@ def sort_values( ignore_index=False, experimental=False, explicit_client=None, + max_branch=None, **kwargs, ): """Sort by the given column @@ -251,6 +254,7 @@ def sort_values( by, ignore_index=ignore_index, explicit_client=explicit_client, + max_branch=max_branch, ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 2e93ced4126..d8b94342051 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -183,7 +183,9 @@ def shuffle_group_divs_2(df, divisions, col): return result2, df.iloc[:0] -def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): +def rearrange_by_division_list( + df, column: str, divisions: list, max_branch=None +): npartitions = len(divisions) - 1 max_branch = max_branch or 32 n = df.npartitions @@ -299,7 +301,9 @@ def rearrange_by_divisions(df, column: str, divisions: list, max_branch=None): return df3 -def sort_values_experimental(df, by, ignore_index=False, explicit_client=None): +def sort_values_experimental( + df, by, ignore_index=False, explicit_client=None, max_branch=None +): """ Experimental sort_values implementation. Sort by the given column name or list/tuple of column names. @@ -356,7 +360,9 @@ def sort_values_experimental(df, by, ignore_index=False, explicit_client=None): df3.divisions = (None,) * (npartitions + 1) return df3 else: - df3 = rearrange_by_divisions(df2, index, divisions) + df3 = rearrange_by_division_list( + df2, index, divisions, max_branch=max_branch + ) df3.divisions = (None,) * (npartitions + 1) # Step 4 - Return final sorted df From 6f04b2b762f58c5b5f5424db636d9193d3b0b16d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 30 Jan 2020 15:05:12 -0800 Subject: [PATCH 13/61] changelog --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index bc142870480..000d7a25740 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -11,7 +11,7 @@ - PR #3975 Simplify and generalize data handling in `Buffer` - PR #3911 Adding null boolean handling for copy_if_else - PR #4002 Adding to_frame and fix for categorical column issue - +- PR #4016 Adding experimental code path for dask_cudf sorting - PR #3897 Port cuIO JSON reader to cudf::column types ## Bug Fixes From 2ef215a9fb486f949fabee3ba50188081f813f5f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 3 Feb 2020 14:26:48 -0800 Subject: [PATCH 14/61] simplifying some logic and adding client.rebalance before shuffle --- .../dask_cudf/dask_cudf/explicit_shuffle.py | 67 +++++-------------- python/dask_cudf/dask_cudf/sorting.py | 2 +- 2 files changed, 18 insertions(+), 51 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index eea43f204df..e5a4f864830 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -35,13 +35,10 @@ async def recv_parts(eps, parts): parts.extend(await asyncio.gather(*futures)) -async def exchange_and_concat_parts(rank, eps, parts, sort=False): +async def exchange_and_concat_parts(rank, eps, parts): ret = [parts[rank]] await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) - df = concat(list(filter(None, ret))) - if sort: - return df.sort_values(sort) - return df + return concat(list(filter(None, ret))) def concat(df_list): @@ -53,30 +50,17 @@ def concat(df_list): def partition_by_column(df, column, n_chunks): if df is None: return [None] * n_chunks - elif hasattr(df, "scatter_by_map"): - return df.scatter_by_map(column, map_size=n_chunks) else: - raise NotImplementedError( - "partition_by_column not yet implemented for pandas backend.\n" - ) + return df.scatter_by_map(column, map_size=n_chunks) -async def distributed_shuffle( - n_chunks, rank, eps, table, partitions, index, sort_by, sorted_split -): - if sorted_split: - parts = [ - table.iloc[partitions[i] : partitions[i + 1]] - for i in range(0, len(partitions) - 1) - ] - else: - parts = partition_by_column(table, partitions, n_chunks) - return await exchange_and_concat_parts(rank, eps, parts, sort=sort_by) +async def distributed_shuffle(n_chunks, rank, eps, table, partitions): + parts = partition_by_column(table, partitions, n_chunks) + del table + return await exchange_and_concat_parts(rank, eps, parts) -async def _explicit_shuffle( - s, df_nparts, df_parts, index, divisions, sort_by, sorted_split -): +async def _explicit_shuffle(s, df_nparts, df_parts, index, divisions): def df_concat(df_parts): """Making sure df_parts is a single dataframe or None""" if len(df_parts) == 0: @@ -90,43 +74,26 @@ def df_concat(df_parts): # a single cudf DataFrame df = df_concat(df_parts[0]) - divisions = cudf.Series(divisions) - if sorted_split: - # Avoid `scatter_by_map` by sorting the dataframe here - # (Can just use iloc to split into groups) - if len(df_parts) > 1: - # Need to sort again after concatenation - df = df.sort_values(sort_by) - splits = df[index].searchsorted(divisions, side="left") - splits[-1] = len(df[index]) - partitions = splits.tolist() - else: + # Calculate new partition mapping + if df: + divisions = cudf.Series(divisions) partitions = divisions.searchsorted(df[index], side="right") - 1 partitions[(df[index] >= divisions.iloc[-1]).values] = ( len(divisions) - 2 ) + else: + partitions = None # Run distributed shuffle and set_index algorithm return await distributed_shuffle( - s["nworkers"], - s["rank"], - s["eps"], - df, - partitions, - index, - sort_by, - sorted_split, + s["nworkers"], s["rank"], s["eps"], df, partitions ) -def explicit_sorted_shuffle( - df, index, divisions, sort_by, sorted_split=False, **kwargs -): +def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): # Explict-comms shuffle # TODO: Fast repartition back to df.npartitions using views... - df.persist() + client.rebalance(futures=df.to_delayed()) return comms.default_comms().dataframe_operation( - _explicit_shuffle, - df_list=(df,), - extra_args=(index, divisions, sort_by, sorted_split), + _explicit_shuffle, df_list=(df,), extra_args=(index, divisions) ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index d8b94342051..d52aedb3aed 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -355,7 +355,7 @@ def sort_values_experimental( if use_explicit: warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_sorted_shuffle( - df2, index, divisions, sort_by=by, sorted_split=False + df2, index, divisions, by, explicit_client, sorted_split=False ) df3.divisions = (None,) * (npartitions + 1) return df3 From 76d13a23643f30aa436401475effc7282bc2ea34 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 3 Feb 2020 15:18:30 -0800 Subject: [PATCH 15/61] ensure cupy uses rmm in explicit_shuffle --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index e5a4f864830..627cbc403bc 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -1,9 +1,14 @@ import asyncio +import cupy + from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize import cudf +import rmm + +cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) async def send_df(ep, df): From d2aeeaf1631784b54805e2d0dc89456e154c4887 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 3 Feb 2020 15:29:37 -0800 Subject: [PATCH 16/61] trying to reduce memory further --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 1 + 1 file changed, 1 insertion(+) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 627cbc403bc..115a4e53e44 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -78,6 +78,7 @@ def df_concat(df_parts): # Concatenate all parts owned by this worker into # a single cudf DataFrame df = df_concat(df_parts[0]) + del df_parts # Calculate new partition mapping if df: From 09423d6f9ba99767817d09b35b5fb5d5ec336591 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 4 Feb 2020 08:06:05 -0800 Subject: [PATCH 17/61] make sure cupy uses rmm in sorting.py --- python/dask_cudf/dask_cudf/core.py | 1 + python/dask_cudf/dask_cudf/sorting.py | 26 ++++++++++++++++++-------- 2 files changed, 19 insertions(+), 8 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 2e503cc1700..bbf3e49ef8a 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -187,6 +187,7 @@ def set_index( experimental=experimental, explicit_client=explicit_client, max_branch=max_branch, + divisions=kwargs.get("divisions", None), ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index d52aedb3aed..46bc430b817 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -3,6 +3,7 @@ import warnings from operator import getitem +import cupy import numpy as np import toolz @@ -15,6 +16,9 @@ from dask.utils import M, digit, insert import cudf as gd +import rmm + +cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) try: from .explicit_shuffle import explicit_sorted_shuffle @@ -302,7 +306,12 @@ def rearrange_by_division_list( def sort_values_experimental( - df, by, ignore_index=False, explicit_client=None, max_branch=None + df, + by, + ignore_index=False, + explicit_client=None, + max_branch=None, + divisions=None, ): """ Experimental sort_values implementation. @@ -343,13 +352,14 @@ def sort_values_experimental( if use_explicit: npartitions = len(explicit_client.cluster.workers) - # Step 2 - Calculate new divisions - divisions = ( - df2[index] - ._repartition_quantiles(npartitions, upsample=1.0) - .compute() - .to_list() - ) + # Step 2 - Calculate new divisions (if ) + if not divisions or len(divisions) != npartitions + 1: + divisions = ( + df2[index] + ._repartition_quantiles(npartitions, upsample=1.0) + .compute() + .to_list() + ) # Step 3 - Perform repartitioning shuffle if use_explicit: From b289c9d65e7d84fc54719b922ae13e8cac4c9c8a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 4 Feb 2020 09:27:58 -0800 Subject: [PATCH 18/61] remove sorted_split for now --- python/dask_cudf/dask_cudf/sorting.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 46bc430b817..3d312944450 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -365,7 +365,7 @@ def sort_values_experimental( if use_explicit: warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_sorted_shuffle( - df2, index, divisions, by, explicit_client, sorted_split=False + df2, index, divisions, by, explicit_client ) df3.divisions = (None,) * (npartitions + 1) return df3 From 717733fe845d37fe8e6d9d2f53414c4d8603c665 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 4 Feb 2020 21:23:19 -0800 Subject: [PATCH 19/61] efforts to reduce memory --- .../dask_cudf/dask_cudf/explicit_shuffle.py | 7 +- python/dask_cudf/dask_cudf/sorting.py | 72 +++++++++++++++---- 2 files changed, 64 insertions(+), 15 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 115a4e53e44..949987016fb 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -78,7 +78,9 @@ def df_concat(df_parts): # Concatenate all parts owned by this worker into # a single cudf DataFrame df = df_concat(df_parts[0]) - del df_parts + for part in df_parts: + if part: + del part # Calculate new partition mapping if df: @@ -87,6 +89,7 @@ def df_concat(df_parts): partitions[(df[index] >= divisions.iloc[-1]).values] = ( len(divisions) - 2 ) + del divisions else: partitions = None @@ -99,7 +102,7 @@ def df_concat(df_parts): def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): # Explict-comms shuffle # TODO: Fast repartition back to df.npartitions using views... - client.rebalance(futures=df.to_delayed()) + # client.rebalance(futures=df.to_delayed()) return comms.default_comms().dataframe_operation( _explicit_shuffle, df_list=(df,), extra_args=(index, divisions) ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 3d312944450..b804af41495 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -10,7 +10,7 @@ from dask import compute, delayed from dask.base import tokenize from dask.dataframe.core import DataFrame, _concat -from dask.dataframe.shuffle import set_partitions_pre, shuffle_group_get +from dask.dataframe.shuffle import shuffle_group_get from dask.dataframe.utils import group_split_dispatch from dask.highlevelgraph import HighLevelGraph from dask.utils import M, digit, insert @@ -164,6 +164,31 @@ def sort_delayed_frame(parts, by): return validparts +def set_partitions_pre(s, divisions): + partitions = divisions.searchsorted(s, side="right") - 1 + partitions[(s >= divisions.iloc[-1]).values] = len(divisions) - 2 + return partitions + + +def sorted_split_divs(df, divisions, col, stage, k, npartitions): + if stage > 1: + raise ValueError("Use of sorted_split_divs with multiple stages.") + + # First, get partitions + dtype = df[col].dtype + splits = df[col].searchsorted( + df._constructor_sliced(divisions, dtype=dtype), side="left" + ) + splits[-1] = len(df[col]) + partitions = splits.tolist() + + # Second, create splits + return { + i: df.iloc[partitions[i] : partitions[i + 1]] + for i in range(0, len(partitions) - 1) + } + + def shuffle_group_divs(df, divisions, col, stage, k, npartitions): dtype = df[col].dtype c = set_partitions_pre( @@ -173,7 +198,9 @@ def shuffle_group_divs(df, divisions, col, stage, k, npartitions): c = np.mod(c, npartitions).astype(typ, copy=False) np.floor_divide(c, k ** stage, out=c) np.mod(c, k, out=c) - return group_split_dispatch(df, c.astype(np.int64), k) + return dict( + zip(range(k), df.scatter_by_map(c.astype(np.int32), map_size=k)) + ) def shuffle_group_divs_2(df, divisions, col): @@ -181,20 +208,25 @@ def shuffle_group_divs_2(df, divisions, col): return {}, df ind = set_partitions_pre( df[col], divisions=df._constructor_sliced(divisions) - ).astype(np.int64) + ).astype(np.int32) n = ind.max() + 1 - result2 = group_split_dispatch(df, ind.values.view(np.int64), n) + result2 = group_split_dispatch(df, ind.values.view(np.int32), n) return result2, df.iloc[:0] def rearrange_by_division_list( - df, column: str, divisions: list, max_branch=None + df, column: str, divisions: list, max_branch=None, sorted_split=None ): + npartitions = len(divisions) - 1 - max_branch = max_branch or 32 n = df.npartitions + max_branch = max_branch or 32 + if sorted_split: + # Only do single stage if data is already sorted + stages = 1 + else: + stages = int(math.ceil(math.log(n) / math.log(max_branch))) - stages = int(math.ceil(math.log(n) / math.log(max_branch))) if stages > 1: k = int(math.ceil(n ** (1 / stages))) else: @@ -218,10 +250,17 @@ def rearrange_by_division_list( for i, inp in enumerate(inputs) } + if sorted_split: + _split_func = sorted_split_divs + _agg_func = _concat + else: + _split_func = shuffle_group_divs + _agg_func = _concat + for stage in range(1, stages + 1): group = { # Convert partition into dict of dataframe pieces ("shuffle-group-divs-" + token, stage, inp): ( - shuffle_group_divs, + _split_func, ("shuffle-join-" + token, stage - 1, inp), divisions, column, @@ -244,7 +283,7 @@ def rearrange_by_division_list( join = { # concatenate those pieces together, with their friends ("shuffle-join-" + token, stage, inp): ( - _concat, + _agg_func, [ ( "shuffle-split-" + token, @@ -312,6 +351,7 @@ def sort_values_experimental( explicit_client=None, max_branch=None, divisions=None, + sorted_split=True, ): """ Experimental sort_values implementation. @@ -335,8 +375,10 @@ def sort_values_experimental( ) # Step 1 - Pre-sort each partition - # df2 = df.map_partitions(M.sort_values, by) - df2 = df # TODO: Use Presort when k-way merge/concat is avilable. + if sorted_split and not explicit_client: + df2 = df.map_partitions(M.sort_values, by) + else: + df2 = df # Only handle single-column partitioning (for now) # TODO: Handle partitioning on multiple columns? @@ -352,7 +394,7 @@ def sort_values_experimental( if use_explicit: npartitions = len(explicit_client.cluster.workers) - # Step 2 - Calculate new divisions (if ) + # Step 2 - Calculate new divisions (if necessary) if not divisions or len(divisions) != npartitions + 1: divisions = ( df2[index] @@ -371,7 +413,11 @@ def sort_values_experimental( return df3 else: df3 = rearrange_by_division_list( - df2, index, divisions, max_branch=max_branch + df2, + index, + divisions, + max_branch=max_branch, + sorted_split=sorted_split, ) df3.divisions = (None,) * (npartitions + 1) From d55e56dbc3d009e4db8e7fb3cd60b2991916f954 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 5 Feb 2020 06:50:25 -0800 Subject: [PATCH 20/61] adding sorted_split back - but explicit code still needs work to avoid OOM --- python/dask_cudf/dask_cudf/core.py | 1 + .../dask_cudf/dask_cudf/explicit_shuffle.py | 45 ++++++++++++++----- python/dask_cudf/dask_cudf/sorting.py | 6 +-- 3 files changed, 39 insertions(+), 13 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index bbf3e49ef8a..bbccdd721c1 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -188,6 +188,7 @@ def set_index( explicit_client=explicit_client, max_branch=max_branch, divisions=kwargs.get("divisions", None), + sorted_split=kwargs.get("sorted_split", False), ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 949987016fb..854e2fed3e1 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -43,6 +43,7 @@ async def recv_parts(eps, parts): async def exchange_and_concat_parts(rank, eps, parts): ret = [parts[rank]] await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) + # Parts may be sorted - k-way merge should improve performance here return concat(list(filter(None, ret))) @@ -59,13 +60,23 @@ def partition_by_column(df, column, n_chunks): return df.scatter_by_map(column, map_size=n_chunks) -async def distributed_shuffle(n_chunks, rank, eps, table, partitions): - parts = partition_by_column(table, partitions, n_chunks) +async def distributed_shuffle( + n_chunks, rank, eps, table, partitions, index, sorted_split +): + if sorted_split: + parts = [ + table.iloc[partitions[i] : partitions[i + 1]] + for i in range(0, len(partitions) - 1) + ] + else: + parts = partition_by_column(table, partitions, n_chunks) del table return await exchange_and_concat_parts(rank, eps, parts) -async def _explicit_shuffle(s, df_nparts, df_parts, index, divisions): +async def _explicit_shuffle( + s, df_nparts, df_parts, index, divisions, sorted_split +): def df_concat(df_parts): """Making sure df_parts is a single dataframe or None""" if len(df_parts) == 0: @@ -78,6 +89,7 @@ def df_concat(df_parts): # Concatenate all parts owned by this worker into # a single cudf DataFrame df = df_concat(df_parts[0]) + nparts = len(df_parts) for part in df_parts: if part: del part @@ -85,24 +97,37 @@ def df_concat(df_parts): # Calculate new partition mapping if df: divisions = cudf.Series(divisions) - partitions = divisions.searchsorted(df[index], side="right") - 1 - partitions[(df[index] >= divisions.iloc[-1]).values] = ( - len(divisions) - 2 - ) + if sorted_split: + if nparts > 1: + # Need to sort again after concatenation + # (Should be faster with k-way merge) + df = df.sort_values(index) + splits = df[index].searchsorted(divisions, side="left") + splits[-1] = len(df[index]) + partitions = splits.tolist() + else: + partitions = divisions.searchsorted(df[index], side="right") - 1 + partitions[(df[index] >= divisions.iloc[-1]).values] = ( + len(divisions) - 2 + ) del divisions else: partitions = None # Run distributed shuffle and set_index algorithm return await distributed_shuffle( - s["nworkers"], s["rank"], s["eps"], df, partitions + s["nworkers"], s["rank"], s["eps"], df, partitions, index, sorted_split ) -def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): +def explicit_sorted_shuffle( + df, index, divisions, sort_by, client, sorted_split, **kwargs +): # Explict-comms shuffle # TODO: Fast repartition back to df.npartitions using views... # client.rebalance(futures=df.to_delayed()) return comms.default_comms().dataframe_operation( - _explicit_shuffle, df_list=(df,), extra_args=(index, divisions) + _explicit_shuffle, + df_list=(df,), + extra_args=(index, divisions, sorted_split), ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index b804af41495..53ba19d5bbd 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -351,7 +351,7 @@ def sort_values_experimental( explicit_client=None, max_branch=None, divisions=None, - sorted_split=True, + sorted_split=False, ): """ Experimental sort_values implementation. @@ -375,7 +375,7 @@ def sort_values_experimental( ) # Step 1 - Pre-sort each partition - if sorted_split and not explicit_client: + if sorted_split: df2 = df.map_partitions(M.sort_values, by) else: df2 = df @@ -407,7 +407,7 @@ def sort_values_experimental( if use_explicit: warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_sorted_shuffle( - df2, index, divisions, by, explicit_client + df2, index, divisions, by, explicit_client, sorted_split ) df3.divisions = (None,) * (npartitions + 1) return df3 From a67957efcabb38a5cb7b93d4a0ab9dabde9971e7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 5 Feb 2020 10:09:05 -0800 Subject: [PATCH 21/61] remove unnecessary cupy setup --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 5 ----- python/dask_cudf/dask_cudf/sorting.py | 4 ---- 2 files changed, 9 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 854e2fed3e1..08c320062c4 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -1,14 +1,9 @@ import asyncio -import cupy - from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize import cudf -import rmm - -cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) async def send_df(ep, df): diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 53ba19d5bbd..55375bf6e62 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -3,7 +3,6 @@ import warnings from operator import getitem -import cupy import numpy as np import toolz @@ -16,9 +15,6 @@ from dask.utils import M, digit, insert import cudf as gd -import rmm - -cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) try: from .explicit_shuffle import explicit_sorted_shuffle From b2ea67b51a01970f4d191892dd1d94099cbfb220 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 5 Feb 2020 11:56:30 -0800 Subject: [PATCH 22/61] use 10.0 upsampling and add to_cpu version of explicit algo --- python/dask_cudf/dask_cudf/core.py | 2 ++ .../dask_cudf/dask_cudf/explicit_shuffle.py | 35 ++++++++++++++----- python/dask_cudf/dask_cudf/sorting.py | 12 +++---- 3 files changed, 33 insertions(+), 16 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index bbccdd721c1..3f21a121ae9 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -234,6 +234,7 @@ def sort_values( experimental=False, explicit_client=None, max_branch=None, + sorted_split=False, **kwargs, ): """Sort by the given column @@ -257,6 +258,7 @@ def sort_values( ignore_index=ignore_index, explicit_client=explicit_client, max_branch=max_branch, + sorted_split=sorted_split, ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 08c320062c4..6daca16a86f 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -1,5 +1,9 @@ import asyncio +import warnings +import pandas as pd + +from dask.dataframe.utils import group_split_dispatch from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize @@ -39,20 +43,23 @@ async def exchange_and_concat_parts(rank, eps, parts): ret = [parts[rank]] await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) # Parts may be sorted - k-way merge should improve performance here - return concat(list(filter(None, ret))) + return concat([df for df in ret if df is not None]) def concat(df_list): if len(df_list) == 0: return None - return cudf.concat(df_list) + if isinstance(df_list[0], cudf.DataFrame): + return cudf.concat(df_list) + else: + return pd.concat(df_list) def partition_by_column(df, column, n_chunks): if df is None: return [None] * n_chunks else: - return df.scatter_by_map(column, map_size=n_chunks) + return group_split_dispatch(df, column, n_chunks) async def distributed_shuffle( @@ -70,7 +77,7 @@ async def distributed_shuffle( async def _explicit_shuffle( - s, df_nparts, df_parts, index, divisions, sorted_split + s, df_nparts, df_parts, index, divisions, sorted_split, to_cpu ): def df_concat(df_parts): """Making sure df_parts is a single dataframe or None""" @@ -83,15 +90,18 @@ def df_concat(df_parts): # Concatenate all parts owned by this worker into # a single cudf DataFrame - df = df_concat(df_parts[0]) + if to_cpu: + df = df_concat([dfp.to_pandas() for dfp in df_parts[0]]) + else: + df = df_concat(df_parts[0]) nparts = len(df_parts) for part in df_parts: if part: del part # Calculate new partition mapping - if df: - divisions = cudf.Series(divisions) + if df is not None: + divisions = df._constructor_sliced(divisions, dtype=df[index].dtype) if sorted_split: if nparts > 1: # Need to sort again after concatenation @@ -110,9 +120,13 @@ def df_concat(df_parts): partitions = None # Run distributed shuffle and set_index algorithm - return await distributed_shuffle( + new_df = await distributed_shuffle( s["nworkers"], s["rank"], s["eps"], df, partitions, index, sorted_split ) + del df + if to_cpu: + return cudf.from_pandas(new_df) + return new_df def explicit_sorted_shuffle( @@ -121,8 +135,11 @@ def explicit_sorted_shuffle( # Explict-comms shuffle # TODO: Fast repartition back to df.npartitions using views... # client.rebalance(futures=df.to_delayed()) + to_cpu = kwargs.get("to_cpu", False) + if to_cpu: + warnings.warn("Using CPU for shuffling. Performance will suffer!") return comms.default_comms().dataframe_operation( _explicit_shuffle, df_list=(df,), - extra_args=(index, divisions, sorted_split), + extra_args=(index, divisions, sorted_split, to_cpu), ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 55375bf6e62..c122af10ce8 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -394,7 +394,7 @@ def sort_values_experimental( if not divisions or len(divisions) != npartitions + 1: divisions = ( df2[index] - ._repartition_quantiles(npartitions, upsample=1.0) + ._repartition_quantiles(npartitions, upsample=10.0) .compute() .to_list() ) @@ -405,8 +405,6 @@ def sort_values_experimental( df3 = explicit_sorted_shuffle( df2, index, divisions, by, explicit_client, sorted_split ) - df3.divisions = (None,) * (npartitions + 1) - return df3 else: df3 = rearrange_by_division_list( df2, @@ -415,8 +413,8 @@ def sort_values_experimental( max_branch=max_branch, sorted_split=sorted_split, ) - df3.divisions = (None,) * (npartitions + 1) + df3.divisions = (None,) * (npartitions + 1) - # Step 4 - Return final sorted df - # (Can remove after k-way merge added) - return df3.map_partitions(M.sort_values, by) + # Step 4 - Return final sorted df + # (Can remove after k-way merge added) + return df3.map_partitions(M.sort_values, by) From 56a472a3f1383819492148d06a5a60219acc0daf Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 5 Feb 2020 12:57:06 -0800 Subject: [PATCH 23/61] allow user to specify and use 2x divisions heuristic to avoid super-small partitions --- python/dask_cudf/dask_cudf/core.py | 3 +++ python/dask_cudf/dask_cudf/sorting.py | 7 +++++-- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 3f21a121ae9..fe1ccb945f1 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -189,6 +189,7 @@ def set_index( max_branch=max_branch, divisions=kwargs.get("divisions", None), sorted_split=kwargs.get("sorted_split", False), + upsample=kwargs.get("upsample", 1.0), ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) @@ -235,6 +236,7 @@ def sort_values( explicit_client=None, max_branch=None, sorted_split=False, + upsample=1.0, **kwargs, ): """Sort by the given column @@ -259,6 +261,7 @@ def sort_values( explicit_client=explicit_client, max_branch=max_branch, sorted_split=sorted_split, + upsample=upsample, ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index c122af10ce8..259fda8abc8 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -348,6 +348,7 @@ def sort_values_experimental( max_branch=None, divisions=None, sorted_split=False, + upsample=1.0, ): """ Experimental sort_values implementation. @@ -392,12 +393,14 @@ def sort_values_experimental( # Step 2 - Calculate new divisions (if necessary) if not divisions or len(divisions) != npartitions + 1: - divisions = ( + doubledivs = ( df2[index] - ._repartition_quantiles(npartitions, upsample=10.0) + ._repartition_quantiles(npartitions * 2, upsample=upsample) .compute() .to_list() ) + # Heuristic: Start with 2x divisions and coarsening + divisions = [doubledivs[i] for i in range(0, len(doubledivs), 2)] # Step 3 - Perform repartitioning shuffle if use_explicit: From 9f0db5a63db183e5b98d5f41f2ca176d061db0f1 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 5 Feb 2020 13:19:24 -0800 Subject: [PATCH 24/61] allow user to specify divisions to set_index and sort_values --- python/dask_cudf/dask_cudf/core.py | 1 + python/dask_cudf/dask_cudf/sorting.py | 3 ++- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index fe1ccb945f1..4d7f5903a95 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -262,6 +262,7 @@ def sort_values( max_branch=max_branch, sorted_split=sorted_split, upsample=upsample, + divisions=kwargs.get("divisions", None), ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 259fda8abc8..2811fd70492 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -392,7 +392,8 @@ def sort_values_experimental( npartitions = len(explicit_client.cluster.workers) # Step 2 - Calculate new divisions (if necessary) - if not divisions or len(divisions) != npartitions + 1: + if not divisions or (use_explicit and len(divisions) != npartitions + 1): + # TODO: Use input divisions for use_explicit==True doubledivs = ( df2[index] ._repartition_quantiles(npartitions * 2, upsample=upsample) From c6e2c13f9bebf29434ca6895f9dc63c1b70389b7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 5 Feb 2020 14:31:56 -0800 Subject: [PATCH 25/61] adding some repartitioning logic, but not using it yet (might not help much) --- python/dask_cudf/dask_cudf/core.py | 5 ++- python/dask_cudf/dask_cudf/sorting.py | 44 ++++++++++++++++++++++++--- 2 files changed, 42 insertions(+), 7 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 4d7f5903a95..2d9d06f86cd 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -193,9 +193,8 @@ def set_index( ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) - return df.map_partitions(M.set_index, other).repartition( - npartitions=npartitions - ) + df2 = df.map_partitions(M.set_index, other) + return df2.repartition(npartitions=npartitions) if kwargs.pop("shuffle", "tasks") != "tasks": raise ValueError( "Dask-cudf only supports task based shuffling, got %s" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 2811fd70492..80d22ab7962 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -6,6 +6,7 @@ import numpy as np import toolz +import dask from dask import compute, delayed from dask.base import tokenize from dask.dataframe.core import DataFrame, _concat @@ -205,8 +206,7 @@ def shuffle_group_divs_2(df, divisions, col): ind = set_partitions_pre( df[col], divisions=df._constructor_sliced(divisions) ).astype(np.int32) - n = ind.max() + 1 - result2 = group_split_dispatch(df, ind.values.view(np.int32), n) + result2 = group_split_dispatch(df, ind.view(np.int32), len(divisions) - 1) return result2, df.iloc[:0] @@ -394,15 +394,51 @@ def sort_values_experimental( # Step 2 - Calculate new divisions (if necessary) if not divisions or (use_explicit and len(divisions) != npartitions + 1): # TODO: Use input divisions for use_explicit==True + + partition_size = None # 10e6 + repartition = False + if partition_size and not use_explicit: + repartition = True + index2 = df2[index] + if repartition: + index2, df2 = dask.base.optimize(index2, df2) + parts = df2.to_delayed(optimize_graph=False) + sizes = [delayed(dask.sizeof.sizeof)(part) for part in parts] + else: + (index2,) = dask.base.optimize(index2) + sizes = [] + doubledivs = ( - df2[index] - ._repartition_quantiles(npartitions * 2, upsample=upsample) + index2._repartition_quantiles(npartitions * 2, upsample=upsample) .compute() .to_list() ) # Heuristic: Start with 2x divisions and coarsening divisions = [doubledivs[i] for i in range(0, len(doubledivs), 2)] + if repartition: + iparts = index2.to_delayed(optimize_graph=False) + mins = [ipart.min() for ipart in iparts] + maxes = [ipart.max() for ipart in iparts] + sizes, mins, maxes = dask.base.optimize(sizes, mins, maxes) + sizes, mins, maxes = dask.base.compute( + sizes, mins, maxes, optimize_graph=False + ) + + total = sum(sizes) + npartitions = max(math.ceil(total / partition_size), 1) + npartitions = min(npartitions, df2.npartitions) + n = len(divisions) + try: + divisions = np.interp( + x=np.linspace(0, n - 1, npartitions + 1), + xp=np.linspace(0, n - 1, n), + fp=divisions, + ).tolist() + except (TypeError, ValueError): # str type + indexes = np.linspace(0, n - 1, npartitions + 1).astype(int) + divisions = [divisions[i] for i in indexes] + # Step 3 - Perform repartitioning shuffle if use_explicit: warnings.warn("Using explicit comms - This is an advanced feature.") From 29e295b7c179bc5b5942fc97aded3f0181f5c438 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 6 Feb 2020 08:23:00 -0800 Subject: [PATCH 26/61] expose partition_size --- python/dask_cudf/dask_cudf/core.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 2d9d06f86cd..4a158d6a292 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -193,7 +193,10 @@ def set_index( ) # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) + partition_size = kwargs.get("partition_size", None) df2 = df.map_partitions(M.set_index, other) + if partition_size: + return df2.repartition(partition_size=partition_size) return df2.repartition(npartitions=npartitions) if kwargs.pop("shuffle", "tasks") != "tasks": raise ValueError( From dac9cdab082857581a61ef2ae0df596cd5b1b7fe Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 6 Feb 2020 08:48:32 -0800 Subject: [PATCH 27/61] fix some division logic --- python/dask_cudf/dask_cudf/sorting.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 80d22ab7962..484b7ab8e7d 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -457,4 +457,6 @@ def sort_values_experimental( # Step 4 - Return final sorted df # (Can remove after k-way merge added) - return df3.map_partitions(M.sort_values, by) + df4 = df3.map_partitions(M.sort_values, by) + df4.divisions = tuple(divisions) + return df4 From 8221feff44b60d52f4c90a93cf061aa5f02d8b55 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 6 Feb 2020 09:28:32 -0800 Subject: [PATCH 28/61] avoid repartition if divisions provided --- python/dask_cudf/dask_cudf/core.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 4a158d6a292..3c55570da56 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -182,12 +182,13 @@ def set_index( ): if experimental and not sorted: # Use sort values for the shuffle + divisions = divisions = kwargs.get("divisions", None) df = self.sort_values( [other], experimental=experimental, explicit_client=explicit_client, max_branch=max_branch, - divisions=kwargs.get("divisions", None), + divisions=divisions, sorted_split=kwargs.get("sorted_split", False), upsample=kwargs.get("upsample", 1.0), ) @@ -197,7 +198,9 @@ def set_index( df2 = df.map_partitions(M.set_index, other) if partition_size: return df2.repartition(partition_size=partition_size) - return df2.repartition(npartitions=npartitions) + if not divisions and df2.npartitions != npartitions: + return df2.repartition(npartitions=npartitions) + return df2 if kwargs.pop("shuffle", "tasks") != "tasks": raise ValueError( "Dask-cudf only supports task based shuffling, got %s" From 4398316d236d18beee239ec66096693e85cb9a70 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 12 Feb 2020 11:48:49 -0800 Subject: [PATCH 29/61] using merge_sorted in experimental -- not working well (yet) --- python/dask_cudf/dask_cudf/core.py | 2 + .../dask_cudf/dask_cudf/explicit_shuffle.py | 57 ++++----- python/dask_cudf/dask_cudf/sorting.py | 112 +++++++++++++----- thirdparty/jitify | 2 +- 4 files changed, 115 insertions(+), 58 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 3c55570da56..67287186ab7 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -200,6 +200,8 @@ def set_index( return df2.repartition(partition_size=partition_size) if not divisions and df2.npartitions != npartitions: return df2.repartition(npartitions=npartitions) + if divisions and df2.npartitions != len(divisions) - 1: + return df2.repartition(divisions=divisions) return df2 if kwargs.pop("shuffle", "tasks") != "tasks": raise ValueError( diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 6daca16a86f..ce6e0430882 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -10,6 +10,12 @@ import cudf +def _cleanup_parts(df_parts): + for part in df_parts: + if part: + del part + + async def send_df(ep, df): if df is None: return await ep.write("empty") @@ -39,20 +45,27 @@ async def recv_parts(eps, parts): parts.extend(await asyncio.gather(*futures)) -async def exchange_and_concat_parts(rank, eps, parts): +async def exchange_and_concat_parts(rank, eps, parts, sort_by): ret = [parts[rank]] await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) - # Parts may be sorted - k-way merge should improve performance here - return concat([df for df in ret if df is not None]) + return concat( + [df for df in ret if df is not None and len(df)], + # TODO: USe `sort_by=sort_by` here (not working right now) + sort_by=None, + ) -def concat(df_list): +def concat(df_list, sort_by=None): if len(df_list) == 0: return None if isinstance(df_list[0], cudf.DataFrame): - return cudf.concat(df_list) + if sort_by: + new_df = cudf.merge_sorted(df_list, keys=sort_by) + new_df = cudf.concat(df_list) else: - return pd.concat(df_list) + new_df = pd.concat(df_list) + _cleanup_parts(df_list) + return new_df def partition_by_column(df, column, n_chunks): @@ -63,9 +76,9 @@ def partition_by_column(df, column, n_chunks): async def distributed_shuffle( - n_chunks, rank, eps, table, partitions, index, sorted_split + n_chunks, rank, eps, table, partitions, index, sort_by ): - if sorted_split: + if sort_by: parts = [ table.iloc[partitions[i] : partitions[i + 1]] for i in range(0, len(partitions) - 1) @@ -73,40 +86,32 @@ async def distributed_shuffle( else: parts = partition_by_column(table, partitions, n_chunks) del table - return await exchange_and_concat_parts(rank, eps, parts) + return await exchange_and_concat_parts(rank, eps, parts, sort_by) async def _explicit_shuffle( - s, df_nparts, df_parts, index, divisions, sorted_split, to_cpu + s, df_nparts, df_parts, index, sort_by, divisions, to_cpu ): - def df_concat(df_parts): + def df_concat(df_parts, sort_by=None): """Making sure df_parts is a single dataframe or None""" if len(df_parts) == 0: return None elif len(df_parts) == 1: return df_parts[0] else: - return concat(df_parts) + return concat(df_parts, sort_by=sort_by) # Concatenate all parts owned by this worker into # a single cudf DataFrame if to_cpu: df = df_concat([dfp.to_pandas() for dfp in df_parts[0]]) else: - df = df_concat(df_parts[0]) - nparts = len(df_parts) - for part in df_parts: - if part: - del part + df = df_concat(df_parts[0], sort_by=sort_by) # Calculate new partition mapping if df is not None: divisions = df._constructor_sliced(divisions, dtype=df[index].dtype) - if sorted_split: - if nparts > 1: - # Need to sort again after concatenation - # (Should be faster with k-way merge) - df = df.sort_values(index) + if sort_by: splits = df[index].searchsorted(divisions, side="left") splits[-1] = len(df[index]) partitions = splits.tolist() @@ -121,7 +126,7 @@ def df_concat(df_parts): # Run distributed shuffle and set_index algorithm new_df = await distributed_shuffle( - s["nworkers"], s["rank"], s["eps"], df, partitions, index, sorted_split + s["nworkers"], s["rank"], s["eps"], df, partitions, index, sort_by ) del df if to_cpu: @@ -129,9 +134,7 @@ def df_concat(df_parts): return new_df -def explicit_sorted_shuffle( - df, index, divisions, sort_by, client, sorted_split, **kwargs -): +def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): # Explict-comms shuffle # TODO: Fast repartition back to df.npartitions using views... # client.rebalance(futures=df.to_delayed()) @@ -141,5 +144,5 @@ def explicit_sorted_shuffle( return comms.default_comms().dataframe_operation( _explicit_shuffle, df_list=(df,), - extra_args=(index, divisions, sorted_split, to_cpu), + extra_args=(index, sort_by, divisions, to_cpu), ) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 484b7ab8e7d..1a8cf0a2d25 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -168,24 +168,50 @@ def set_partitions_pre(s, divisions): def sorted_split_divs(df, divisions, col, stage, k, npartitions): - if stage > 1: - raise ValueError("Use of sorted_split_divs with multiple stages.") - - # First, get partitions + divs = divisions + if k > npartitions: + # Aggregate divisions for "k" partitions + divs = list(divisions)[0::2] + if not len(divisions) % len(divs): + divs += [divisions[-1]] + + # Get partitions dtype = df[col].dtype splits = df[col].searchsorted( - df._constructor_sliced(divisions, dtype=dtype), side="left" + df._constructor_sliced(divs, dtype=dtype), side="left" ) splits[-1] = len(df[col]) partitions = splits.tolist() - # Second, create splits + # Create splits return { - i: df.iloc[partitions[i] : partitions[i + 1]] - for i in range(0, len(partitions) - 1) + i: df.iloc[partitions[i] : partitions[i + 1]].copy() + for i in range(len(divs) - 1) } +def sorted_split_divs_2(df, divisions, col): + if not len(df): + return {}, df + + # Get partitions + dtype = df[col].dtype + splits = df[col].searchsorted( + df._constructor_sliced(divisions, dtype=dtype), side="left" + ) + splits[-1] = len(df[col]) + partitions = splits.tolist() + + # Create splits + return ( + { + i: df.iloc[partitions[i] : partitions[i + 1]].copy() + for i in range(len(divisions) - 1) + }, + df.iloc[:0], + ) + + def shuffle_group_divs(df, divisions, col, stage, k, npartitions): dtype = df[col].dtype c = set_partitions_pre( @@ -210,18 +236,24 @@ def shuffle_group_divs_2(df, divisions, col): return result2, df.iloc[:0] +def _concat_wrapper(df_list, sort_by): + if sort_by: + return gd.merge_sorted(df_list, keys=sort_by) + else: + return _concat(df_list) + + def rearrange_by_division_list( - df, column: str, divisions: list, max_branch=None, sorted_split=None + df, column: str, divisions: list, max_branch=None, sort_by=None ): npartitions = len(divisions) - 1 n = df.npartitions max_branch = max_branch or 32 - if sorted_split: - # Only do single stage if data is already sorted + stages = int(math.ceil(math.log(n) / math.log(max_branch))) + + if sort_by: stages = 1 - else: - stages = int(math.ceil(math.log(n) / math.log(max_branch))) if stages > 1: k = int(math.ceil(n ** (1 / stages))) @@ -246,17 +278,17 @@ def rearrange_by_division_list( for i, inp in enumerate(inputs) } - if sorted_split: - _split_func = sorted_split_divs - _agg_func = _concat + if sort_by: + _split_func_1 = sorted_split_divs + _split_func_2 = sorted_split_divs_2 else: - _split_func = shuffle_group_divs - _agg_func = _concat + _split_func_1 = shuffle_group_divs + _split_func_2 = shuffle_group_divs_2 for stage in range(1, stages + 1): group = { # Convert partition into dict of dataframe pieces ("shuffle-group-divs-" + token, stage, inp): ( - _split_func, + _split_func_1, ("shuffle-join-" + token, stage - 1, inp), divisions, column, @@ -279,7 +311,7 @@ def rearrange_by_division_list( join = { # concatenate those pieces together, with their friends ("shuffle-join-" + token, stage, inp): ( - _agg_func, + _concat_wrapper, [ ( "shuffle-split-" + token, @@ -289,6 +321,7 @@ def rearrange_by_division_list( ) for j in range(k) ], + sort_by, ) for inp in inputs } @@ -313,7 +346,7 @@ def rearrange_by_division_list( dsk = { ("repartition-group-" + token, i): ( - shuffle_group_divs_2, + _split_func_2, k, divisions, column, @@ -440,23 +473,42 @@ def sort_values_experimental( divisions = [divisions[i] for i in indexes] # Step 3 - Perform repartitioning shuffle + sort_by = None + if sorted_split: + sort_by = by if use_explicit: warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_sorted_shuffle( - df2, index, divisions, by, explicit_client, sorted_split + df2, index, divisions, sort_by, explicit_client ) else: + if sorted_split: + # sorted_split does not support multiple stages (yet) + # Lets aggregate divisions here... + npartitions = len(divisions) - 1 + n = df.npartitions + max_branch = max_branch or 32 + stages = int(math.ceil(math.log(n) / math.log(max_branch))) + k = n + if stages > 1: + k = int(math.ceil(n ** (1 / stages))) + if k < n: + # Aggregate divisions for "k" partitions + divs = list(divisions)[0::2] + if not len(divisions) % len(divs): + divs += [divisions[-1]] + divisions = divs + df3 = rearrange_by_division_list( - df2, - index, - divisions, - max_branch=max_branch, - sorted_split=sorted_split, + df2, index, divisions, max_branch=max_branch, sort_by=sort_by ) - df3.divisions = (None,) * (npartitions + 1) + df3.divisions = (None,) * (df3.npartitions + 1) # Step 4 - Return final sorted df - # (Can remove after k-way merge added) - df4 = df3.map_partitions(M.sort_values, by) + if sorted_split: + # Data should already be sorted + df4 = df3 + else: + df4 = df3.map_partitions(M.sort_values, by) df4.divisions = tuple(divisions) return df4 diff --git a/thirdparty/jitify b/thirdparty/jitify index e3f867027c1..bcd545071c7 160000 --- a/thirdparty/jitify +++ b/thirdparty/jitify @@ -1 +1 @@ -Subproject commit e3f867027c1d9603b5a677795900465b9fac9cb8 +Subproject commit bcd545071c7a5ddb28cb6576afc6399eb1286c43 From b3edc6dbe21c1bbd853cdb5fa12a4f679216a513 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 13 Feb 2020 06:55:48 -0800 Subject: [PATCH 30/61] partial progress for sorted_split with multiple stages --- python/dask_cudf/dask_cudf/sorting.py | 109 ++++++++++++++++++-------- 1 file changed, 78 insertions(+), 31 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 1a8cf0a2d25..43537b31a8b 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -167,13 +167,30 @@ def set_partitions_pre(s, divisions): return partitions -def sorted_split_divs(df, divisions, col, stage, k, npartitions): - divs = divisions - if k > npartitions: - # Aggregate divisions for "k" partitions - divs = list(divisions)[0::2] - if not len(divisions) % len(divs): - divs += [divisions[-1]] +def sorted_split_divs(df, divisions, col, stage, k, npartitions, inp): + + divs = list(divisions) + if k < npartitions: + # Narrow down which divisions to aggregate + for st in range(stage): + zone = inp[st] + cnt = int(len(divs) / k) + start = cnt * zone + stop = min(cnt * (zone + 1) + 1, len(divs)) + divs = divs[start:stop] + + if len(divs) > (k + 1): + # Aggregate divisions for "k" partitions + start = 0 + stop = len(divs) + stride = int(stop / k) + last = divs[-1] + divs = [divs[i] for i in range(0, stop, stride)] + if len(divs) < (k + 1): + divs += [last] + + while len(divs) < (k + 1): + divs += [divs[-1]] # Get partitions dtype = df[col].dtype @@ -185,8 +202,40 @@ def sorted_split_divs(df, divisions, col, stage, k, npartitions): # Create splits return { - i: df.iloc[partitions[i] : partitions[i + 1]].copy() - for i in range(len(divs) - 1) + i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) + for i in range(k) + } + + +def sorted_split_divs__works__( + df, divisions, col, stage, k, npartitions, sort_by +): + # Get partitions + dtype = df[col].dtype + splits = df[col].searchsorted( + df._constructor_sliced(divisions, dtype=dtype), side="left" + ) + splits[-1] = len(df[col]) + partitions = splits.tolist() + + # Create splits + split_dict = { + i: df.iloc[partitions[i] : partitions[i + 1]] + for i in range(len(divisions) - 1) + } + + # Rearrange the splits (for now -- Need NEW algorithm to avoid this) + # Note that we REALLY don't want to do this if we dont need to!! + agg_dict = {i: [] for i in range(k)} + for c in [int(k) for k in split_dict.keys()]: + c_new = np.mod(np.floor_divide(c, k ** stage), k) + if split_dict[c] is not None: + agg_dict[c_new].append(split_dict[c].copy(deep=False)) + return { + i: gd.merge_sorted(agg_dict[i], keys=sort_by) + if len(agg_dict[i]) + else df.iloc[:0] + for i in range(k) } @@ -205,14 +254,14 @@ def sorted_split_divs_2(df, divisions, col): # Create splits return ( { - i: df.iloc[partitions[i] : partitions[i + 1]].copy() + i: df.iloc[partitions[i] : partitions[i + 1]] for i in range(len(divisions) - 1) }, df.iloc[:0], ) -def shuffle_group_divs(df, divisions, col, stage, k, npartitions): +def shuffle_group_divs(df, divisions, col, stage, k, npartitions, inp): dtype = df[col].dtype c = set_partitions_pre( df[col], divisions=df._constructor_sliced(divisions, dtype=dtype) @@ -240,7 +289,10 @@ def _concat_wrapper(df_list, sort_by): if sort_by: return gd.merge_sorted(df_list, keys=sort_by) else: - return _concat(df_list) + df = _concat(df_list) + if sort_by: + return df.sort_values(sort_by) + return df def rearrange_by_division_list( @@ -252,8 +304,8 @@ def rearrange_by_division_list( max_branch = max_branch or 32 stages = int(math.ceil(math.log(n) / math.log(max_branch))) - if sort_by: - stages = 1 + # if sort_by: + # stages = 1 if stages > 1: k = int(math.ceil(n ** (1 / stages))) @@ -279,12 +331,17 @@ def rearrange_by_division_list( } if sort_by: + # TODO: Figure out "correct" way to use splits _split_func_1 = sorted_split_divs _split_func_2 = sorted_split_divs_2 + # _split_func_1 = shuffle_group_divs + # _split_func_2 = shuffle_group_divs_2 else: _split_func_1 = shuffle_group_divs _split_func_2 = shuffle_group_divs_2 + # import pdb; pdb.set_trace() + for stage in range(1, stages + 1): group = { # Convert partition into dict of dataframe pieces ("shuffle-group-divs-" + token, stage, inp): ( @@ -295,9 +352,11 @@ def rearrange_by_division_list( stage - 1, k, n, + inp, # Need this to know how to split divisions ) for inp in inputs } + # pdb.set_trace() split = { # Get out each individual dataframe piece from the dicts ("shuffle-split-" + token, stage, i, inp): ( @@ -308,6 +367,7 @@ def rearrange_by_division_list( for i in range(k) for inp in inputs } + # pdb.set_trace() join = { # concatenate those pieces together, with their friends ("shuffle-join-" + token, stage, inp): ( @@ -325,10 +385,14 @@ def rearrange_by_division_list( ) for inp in inputs } + # pdb.set_trace() + groups.append(group) splits.append(split) joins.append(join) + if sort_by: + inputs = sorted(inputs) end = { ("shuffle-" + token, i): ("shuffle-join-" + token, stages, inp) for i, inp in enumerate(inputs) @@ -482,23 +546,6 @@ def sort_values_experimental( df2, index, divisions, sort_by, explicit_client ) else: - if sorted_split: - # sorted_split does not support multiple stages (yet) - # Lets aggregate divisions here... - npartitions = len(divisions) - 1 - n = df.npartitions - max_branch = max_branch or 32 - stages = int(math.ceil(math.log(n) / math.log(max_branch))) - k = n - if stages > 1: - k = int(math.ceil(n ** (1 / stages))) - if k < n: - # Aggregate divisions for "k" partitions - divs = list(divisions)[0::2] - if not len(divisions) % len(divs): - divs += [divisions[-1]] - divisions = divs - df3 = rearrange_by_division_list( df2, index, divisions, max_branch=max_branch, sort_by=sort_by ) From 1b25f80d5bb80cc49aa3f0af33eb4e9311a5156a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 13 Feb 2020 11:30:18 -0800 Subject: [PATCH 31/61] sorted_split now working for many cases (non-explicit) --- python/dask_cudf/dask_cudf/sorting.py | 104 +++++++++++--------------- 1 file changed, 42 insertions(+), 62 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 43537b31a8b..b5b40246d25 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -167,76 +167,50 @@ def set_partitions_pre(s, divisions): return partitions -def sorted_split_divs(df, divisions, col, stage, k, npartitions, inp): - +def _get_current_divs(divisions, stage, k, npartitions, inp): divs = list(divisions) if k < npartitions: # Narrow down which divisions to aggregate for st in range(stage): zone = inp[st] - cnt = int(len(divs) / k) - start = cnt * zone - stop = min(cnt * (zone + 1) + 1, len(divs)) + cnt = float(len(divs) / k) + start = math.floor(cnt * zone) + stop = math.floor(cnt * (zone + 1)) + stop = min(stop + 1, len(divs)) divs = divs[start:stop] if len(divs) > (k + 1): # Aggregate divisions for "k" partitions start = 0 - stop = len(divs) - stride = int(stop / k) + stop = float(len(divs)) + stride = float(stop / k) last = divs[-1] - divs = [divs[i] for i in range(0, stop, stride)] + divs = [divs[math.floor(i)] for i in np.arange(0.0, stop, stride)] if len(divs) < (k + 1): divs += [last] while len(divs) < (k + 1): divs += [divs[-1]] - # Get partitions - dtype = df[col].dtype - splits = df[col].searchsorted( - df._constructor_sliced(divs, dtype=dtype), side="left" - ) - splits[-1] = len(df[col]) - partitions = splits.tolist() + # Check new divisions size + assert len(divs) == (k + 1) + return divs - # Create splits - return { - i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) - for i in range(k) - } +def sorted_split_divs(df, divisions, col, stage, k, npartitions, inp): + + divs = _get_current_divs(divisions, stage, k, npartitions, inp) -def sorted_split_divs__works__( - df, divisions, col, stage, k, npartitions, sort_by -): # Get partitions dtype = df[col].dtype splits = df[col].searchsorted( - df._constructor_sliced(divisions, dtype=dtype), side="left" + df._constructor_sliced(divs, dtype=dtype), side="left" ) splits[-1] = len(df[col]) partitions = splits.tolist() # Create splits - split_dict = { - i: df.iloc[partitions[i] : partitions[i + 1]] - for i in range(len(divisions) - 1) - } - - # Rearrange the splits (for now -- Need NEW algorithm to avoid this) - # Note that we REALLY don't want to do this if we dont need to!! - agg_dict = {i: [] for i in range(k)} - for c in [int(k) for k in split_dict.keys()]: - c_new = np.mod(np.floor_divide(c, k ** stage), k) - if split_dict[c] is not None: - agg_dict[c_new].append(split_dict[c].copy(deep=False)) - return { - i: gd.merge_sorted(agg_dict[i], keys=sort_by) - if len(agg_dict[i]) - else df.iloc[:0] - for i in range(k) - } + return {i: df.iloc[partitions[i] : partitions[i + 1]] for i in range(k)} def sorted_split_divs_2(df, divisions, col): @@ -252,13 +226,12 @@ def sorted_split_divs_2(df, divisions, col): partitions = splits.tolist() # Create splits - return ( - { - i: df.iloc[partitions[i] : partitions[i + 1]] - for i in range(len(divisions) - 1) - }, - df.iloc[:0], - ) + result2 = { + i: df.iloc[partitions[i] : partitions[i + 1]] + for i in range(len(divisions) - 1) + if partitions[i] != partitions[i + 1] + } + return result2, df.iloc[:0] def shuffle_group_divs(df, divisions, col, stage, k, npartitions, inp): @@ -298,15 +271,11 @@ def _concat_wrapper(df_list, sort_by): def rearrange_by_division_list( df, column: str, divisions: list, max_branch=None, sort_by=None ): - npartitions = len(divisions) - 1 n = df.npartitions max_branch = max_branch or 32 stages = int(math.ceil(math.log(n) / math.log(max_branch))) - # if sort_by: - # stages = 1 - if stages > 1: k = int(math.ceil(n ** (1 / stages))) else: @@ -331,17 +300,12 @@ def rearrange_by_division_list( } if sort_by: - # TODO: Figure out "correct" way to use splits _split_func_1 = sorted_split_divs _split_func_2 = sorted_split_divs_2 - # _split_func_1 = shuffle_group_divs - # _split_func_2 = shuffle_group_divs_2 else: _split_func_1 = shuffle_group_divs _split_func_2 = shuffle_group_divs_2 - # import pdb; pdb.set_trace() - for stage in range(1, stages + 1): group = { # Convert partition into dict of dataframe pieces ("shuffle-group-divs-" + token, stage, inp): ( @@ -356,7 +320,6 @@ def rearrange_by_division_list( ) for inp in inputs } - # pdb.set_trace() split = { # Get out each individual dataframe piece from the dicts ("shuffle-split-" + token, stage, i, inp): ( @@ -367,7 +330,6 @@ def rearrange_by_division_list( for i in range(k) for inp in inputs } - # pdb.set_trace() join = { # concatenate those pieces together, with their friends ("shuffle-join-" + token, stage, inp): ( @@ -385,8 +347,6 @@ def rearrange_by_division_list( ) for inp in inputs } - # pdb.set_trace() - groups.append(group) splits.append(split) joins.append(join) @@ -406,6 +366,25 @@ def rearrange_by_division_list( if npartitions != df.npartitions: parts = [i % df.npartitions for i in range(npartitions)] + + if sort_by and df.npartitions < npartitions: + # Parts are distribued differently if we used `sorted_split` + divs = [] + for inp in inputs: + divs.append( + _get_current_divs( + divisions, stages - 1, k, npartitions, inp + )[inp[-1]] + ) + divs += [divisions[-1]] + parts = ( + np.searchsorted(divs, list(divisions)[:-1], side="right") - 1 + ).tolist() + + elif sort_by: + # TODO: Address this case. + warnings.warn("WARNING - Haven't considered this case yet.") + token = tokenize(df2, npartitions) dsk = { @@ -512,6 +491,7 @@ def sort_values_experimental( ) # Heuristic: Start with 2x divisions and coarsening divisions = [doubledivs[i] for i in range(0, len(doubledivs), 2)] + divisions[-1] += 1 # Make sure the last division is large enough if repartition: iparts = index2.to_delayed(optimize_graph=False) From bfdf2a39f9e9dcccc2ba47188a83d758c0cf032f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 13 Feb 2020 12:43:12 -0800 Subject: [PATCH 32/61] explicit comms fix for sorted_split --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 10 ++++------ python/dask_cudf/dask_cudf/sorting.py | 7 +++++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index ce6e0430882..3c6079fa604 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -49,9 +49,7 @@ async def exchange_and_concat_parts(rank, eps, parts, sort_by): ret = [parts[rank]] await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) return concat( - [df for df in ret if df is not None and len(df)], - # TODO: USe `sort_by=sort_by` here (not working right now) - sort_by=None, + [df for df in ret if df is not None and len(df)], sort_by=sort_by ) @@ -61,7 +59,8 @@ def concat(df_list, sort_by=None): if isinstance(df_list[0], cudf.DataFrame): if sort_by: new_df = cudf.merge_sorted(df_list, keys=sort_by) - new_df = cudf.concat(df_list) + else: + new_df = cudf.concat(df_list) else: new_df = pd.concat(df_list) _cleanup_parts(df_list) @@ -80,12 +79,11 @@ async def distributed_shuffle( ): if sort_by: parts = [ - table.iloc[partitions[i] : partitions[i + 1]] + table.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) for i in range(0, len(partitions) - 1) ] else: parts = partition_by_column(table, partitions, n_chunks) - del table return await exchange_and_concat_parts(rank, eps, parts, sort_by) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index b5b40246d25..036b4ffc080 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -210,7 +210,10 @@ def sorted_split_divs(df, divisions, col, stage, k, npartitions, inp): partitions = splits.tolist() # Create splits - return {i: df.iloc[partitions[i] : partitions[i + 1]] for i in range(k)} + return { + i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) + for i in range(k) + } def sorted_split_divs_2(df, divisions, col): @@ -227,7 +230,7 @@ def sorted_split_divs_2(df, divisions, col): # Create splits result2 = { - i: df.iloc[partitions[i] : partitions[i + 1]] + i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) for i in range(len(divisions) - 1) if partitions[i] != partitions[i + 1] } From 634af0254dcdd0a00653fbc4bbac93c11e398e72 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 13 Feb 2020 14:00:42 -0800 Subject: [PATCH 33/61] explicit comms cleanup --- .../dask_cudf/dask_cudf/explicit_shuffle.py | 28 +++++++++++-------- 1 file changed, 16 insertions(+), 12 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 3c6079fa604..1f50347e835 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -3,7 +3,6 @@ import pandas as pd -from dask.dataframe.utils import group_split_dispatch from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize @@ -67,23 +66,29 @@ def concat(df_list, sort_by=None): return new_df -def partition_by_column(df, column, n_chunks): +def partition_table(df, partitions, n_chunks, sort_by=None): if df is None: - return [None] * n_chunks + result = [None] * n_chunks + elif sort_by: + result = { + i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) + for i in range(0, len(partitions) - 1) + } else: - return group_split_dispatch(df, column, n_chunks) + result = dict( + zip( + range(n_chunks), + df.scatter_by_map(partitions, map_size=n_chunks), + ) + ) + del df + return result async def distributed_shuffle( n_chunks, rank, eps, table, partitions, index, sort_by ): - if sort_by: - parts = [ - table.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) - for i in range(0, len(partitions) - 1) - ] - else: - parts = partition_by_column(table, partitions, n_chunks) + parts = partition_table(table, partitions, n_chunks, sort_by=sort_by) return await exchange_and_concat_parts(rank, eps, parts, sort_by) @@ -134,7 +139,6 @@ def df_concat(df_parts, sort_by=None): def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): # Explict-comms shuffle - # TODO: Fast repartition back to df.npartitions using views... # client.rebalance(futures=df.to_delayed()) to_cpu = kwargs.get("to_cpu", False) if to_cpu: From 970a1fb1bf97e6de834b7561fd5d7183a68ca2a7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 14 Feb 2020 07:36:14 -0800 Subject: [PATCH 34/61] rolling back sorted_split changes (slow again, but at least generating correct results) --- .../dask_cudf/dask_cudf/explicit_shuffle.py | 6 +- python/dask_cudf/dask_cudf/sorting.py | 80 +++++-------------- 2 files changed, 26 insertions(+), 60 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 1f50347e835..5c6344de2ad 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -1,12 +1,16 @@ import asyncio import warnings +import cupy import pandas as pd from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize import cudf +import rmm + +cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) def _cleanup_parts(df_parts): @@ -139,7 +143,7 @@ def df_concat(df_parts, sort_by=None): def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): # Explict-comms shuffle - # client.rebalance(futures=df.to_delayed()) + client.rebalance(futures=df.to_delayed()) to_cpu = kwargs.get("to_cpu", False) if to_cpu: warnings.warn("Using CPU for shuffling. Performance will suffer!") diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 036b4ffc080..18f8cf0d12e 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -167,54 +167,37 @@ def set_partitions_pre(s, divisions): return partitions -def _get_current_divs(divisions, stage, k, npartitions, inp): - divs = list(divisions) - if k < npartitions: - # Narrow down which divisions to aggregate - for st in range(stage): - zone = inp[st] - cnt = float(len(divs) / k) - start = math.floor(cnt * zone) - stop = math.floor(cnt * (zone + 1)) - stop = min(stop + 1, len(divs)) - divs = divs[start:stop] - - if len(divs) > (k + 1): - # Aggregate divisions for "k" partitions - start = 0 - stop = float(len(divs)) - stride = float(stop / k) - last = divs[-1] - divs = [divs[math.floor(i)] for i in np.arange(0.0, stop, stride)] - if len(divs) < (k + 1): - divs += [last] - - while len(divs) < (k + 1): - divs += [divs[-1]] - - # Check new divisions size - assert len(divs) == (k + 1) - return divs - - -def sorted_split_divs(df, divisions, col, stage, k, npartitions, inp): - - divs = _get_current_divs(divisions, stage, k, npartitions, inp) - +def sorted_split_divs(df, divisions, col, stage, k, npartitions, sort_by): # Get partitions dtype = df[col].dtype splits = df[col].searchsorted( - df._constructor_sliced(divs, dtype=dtype), side="left" + df._constructor_sliced(divisions, dtype=dtype), side="left" ) splits[-1] = len(df[col]) partitions = splits.tolist() # Create splits - return { + split_dict = { i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) - for i in range(k) + for i in range(len(divisions) - 1) } + if k < npartitions: + # Rearrange the splits (for now -- Need NEW algorithm to avoid this) + # Note that we REALLY don't want to do this if we dont need to!! + agg_dict = {i: [] for i in range(k)} + for c in [int(k) for k in split_dict.keys()]: + c_new = np.mod(np.floor_divide(c, k ** stage), k) + if split_dict[c] is not None and len(split_dict[c]): + agg_dict[c_new].append(split_dict[c].copy(deep=False)) + split_dict = {} + for i in range(k): + if len(agg_dict[i]): + split_dict[i] = gd.merge_sorted(agg_dict[i], keys=sort_by) + else: + split_dict[i] = df.iloc[:0] + return split_dict + def sorted_split_divs_2(df, divisions, col): if not len(df): @@ -319,7 +302,7 @@ def rearrange_by_division_list( stage - 1, k, n, - inp, # Need this to know how to split divisions + sort_by, # Need this to rearrange splits (for now) ) for inp in inputs } @@ -354,8 +337,6 @@ def rearrange_by_division_list( splits.append(split) joins.append(join) - if sort_by: - inputs = sorted(inputs) end = { ("shuffle-" + token, i): ("shuffle-join-" + token, stages, inp) for i, inp in enumerate(inputs) @@ -369,25 +350,6 @@ def rearrange_by_division_list( if npartitions != df.npartitions: parts = [i % df.npartitions for i in range(npartitions)] - - if sort_by and df.npartitions < npartitions: - # Parts are distribued differently if we used `sorted_split` - divs = [] - for inp in inputs: - divs.append( - _get_current_divs( - divisions, stages - 1, k, npartitions, inp - )[inp[-1]] - ) - divs += [divisions[-1]] - parts = ( - np.searchsorted(divs, list(divisions)[:-1], side="right") - 1 - ).tolist() - - elif sort_by: - # TODO: Address this case. - warnings.warn("WARNING - Haven't considered this case yet.") - token = tokenize(df2, npartitions) dsk = { From 4e33f714fc1a975a670e661ac6ec487fb54527f5 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 14 Feb 2020 07:41:07 -0800 Subject: [PATCH 35/61] correct rebalance command --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 5c6344de2ad..f5c7b7fab09 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -4,6 +4,7 @@ import cupy import pandas as pd +import distributed from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize @@ -143,7 +144,7 @@ def df_concat(df_parts, sort_by=None): def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): # Explict-comms shuffle - client.rebalance(futures=df.to_delayed()) + client.rebalance(futures=distributed.futures_of(df)) to_cpu = kwargs.get("to_cpu", False) if to_cpu: warnings.warn("Using CPU for shuffling. Performance will suffer!") From 557fc21939ad06ba73f5aee0e8edbfb5f628bbbe Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 14 Feb 2020 10:48:09 -0800 Subject: [PATCH 36/61] some memory cleanup --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index f5c7b7fab09..3b7ab1f2eaa 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -51,10 +51,18 @@ async def recv_parts(eps, parts): async def exchange_and_concat_parts(rank, eps, parts, sort_by): ret = [parts[rank]] + del parts[rank] + parts[rank] = None await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) - return concat( - [df for df in ret if df is not None and len(df)], sort_by=sort_by + for rank in list(parts): + del parts[rank] + new_df = concat( + [df.copy(deep=False) for df in ret if df is not None and len(df)], + sort_by=sort_by, ) + for r in ret: + del r + return new_df def concat(df_list, sort_by=None): From 876cade4380ed2512c6bac16f26a407a9c8b18c7 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 19 Feb 2020 10:19:52 -0800 Subject: [PATCH 37/61] separate aggregation from shuffle --- python/cudf/cudf/_libxx/__init__.py | 1 + .../dask_cudf/dask_cudf/explicit_shuffle.py | 59 +++++++++++++------ 2 files changed, 41 insertions(+), 19 deletions(-) diff --git a/python/cudf/cudf/_libxx/__init__.py b/python/cudf/cudf/_libxx/__init__.py index f4f68ea2925..fc7e3856ea1 100644 --- a/python/cudf/cudf/_libxx/__init__.py +++ b/python/cudf/cudf/_libxx/__init__.py @@ -2,6 +2,7 @@ from . import ( copying, + merge, null_mask, rolling, search, diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 3b7ab1f2eaa..8905d98a9df 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -5,6 +5,7 @@ import pandas as pd import distributed +from dask.distributed import wait from dask_cuda.explicit_comms import comms from distributed.protocol import to_serialize @@ -94,7 +95,6 @@ def partition_table(df, partitions, n_chunks, sort_by=None): df.scatter_by_map(partitions, map_size=n_chunks), ) ) - del df return result @@ -108,21 +108,10 @@ async def distributed_shuffle( async def _explicit_shuffle( s, df_nparts, df_parts, index, sort_by, divisions, to_cpu ): - def df_concat(df_parts, sort_by=None): - """Making sure df_parts is a single dataframe or None""" - if len(df_parts) == 0: - return None - elif len(df_parts) == 1: - return df_parts[0] - else: - return concat(df_parts, sort_by=sort_by) - - # Concatenate all parts owned by this worker into - # a single cudf DataFrame - if to_cpu: - df = df_concat([dfp.to_pandas() for dfp in df_parts[0]]) + if len(df_parts[0]) == 0: + df = None else: - df = df_concat(df_parts[0], sort_by=sort_by) + df = df_parts[0][0].copy(deep=False) # Calculate new partition mapping if df is not None: @@ -131,6 +120,7 @@ def df_concat(df_parts, sort_by=None): splits = df[index].searchsorted(divisions, side="left") splits[-1] = len(df[index]) partitions = splits.tolist() + del splits else: partitions = divisions.searchsorted(df[index], side="right") - 1 partitions[(df[index] >= divisions.iloc[-1]).values] = ( @@ -144,20 +134,51 @@ def df_concat(df_parts, sort_by=None): new_df = await distributed_shuffle( s["nworkers"], s["rank"], s["eps"], df, partitions, index, sort_by ) - del df + if to_cpu: return cudf.from_pandas(new_df) return new_df +async def _explicit_aggregate(s, df_nparts, df_parts, sort_by, to_cpu): + def df_concat(df_parts, sort_by=None): + """Making sure df_parts is a single dataframe or None""" + if len(df_parts) == 0: + return None + elif len(df_parts) == 1: + return df_parts[0] + else: + return concat(df_parts, sort_by=sort_by) + + # Concatenate all parts owned by this worker into + # a single cudf DataFrame + if to_cpu: + return df_concat([dfp.to_pandas() for dfp in df_parts[0]]) + else: + return df_concat(df_parts[0], sort_by=sort_by) + + def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): - # Explict-comms shuffle client.rebalance(futures=distributed.futures_of(df)) to_cpu = kwargs.get("to_cpu", False) if to_cpu: warnings.warn("Using CPU for shuffling. Performance will suffer!") - return comms.default_comms().dataframe_operation( + + # Explict-comms Partition Aggregation + df2 = comms.default_comms().dataframe_operation( + _explicit_aggregate, df_list=(df,), extra_args=(sort_by, to_cpu) + ) + wait(df2.persist()) + wait(client.cancel(df)) + del df + + # Explict-comms shuffle + df3 = comms.default_comms().dataframe_operation( _explicit_shuffle, - df_list=(df,), + df_list=(df2,), extra_args=(index, sort_by, divisions, to_cpu), ) + wait(df3.persist()) + wait(client.cancel(df2)) + del df2 + return df3 From 988f640f3a3d7698463037eaae221c819f037107 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 19 Feb 2020 10:34:19 -0800 Subject: [PATCH 38/61] some memory stability in explicit-comms --- python/dask_cudf/dask_cudf/explicit_shuffle.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py index 8905d98a9df..7620c55c8a4 100644 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ b/python/dask_cudf/dask_cudf/explicit_shuffle.py @@ -61,8 +61,7 @@ async def exchange_and_concat_parts(rank, eps, parts, sort_by): [df.copy(deep=False) for df in ret if df is not None and len(df)], sort_by=sort_by, ) - for r in ret: - del r + del ret return new_df From a586317069071decd6d4e18ce03871dde44e2377 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 19 Feb 2020 20:57:55 -0800 Subject: [PATCH 39/61] start laying groundwork for multi-column sorting --- python/dask_cudf/dask_cudf/core.py | 10 ++- python/dask_cudf/dask_cudf/sorting.py | 95 +++++++++++++++------------ 2 files changed, 62 insertions(+), 43 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 67287186ab7..8c9c5c5ee83 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -183,8 +183,11 @@ def set_index( if experimental and not sorted: # Use sort values for the shuffle divisions = divisions = kwargs.get("divisions", None) + by = other + if not isinstance(other, list): + by = [by] df = self.sort_values( - [other], + by, experimental=experimental, explicit_client=explicit_client, max_branch=max_branch, @@ -192,6 +195,11 @@ def set_index( sorted_split=kwargs.get("sorted_split", False), upsample=kwargs.get("upsample", 1.0), ) + + # Ignore divisions if its a dataframe + if isinstance(divisions, cudf.DataFrame): + divisions = None + # Set index and repartition npartitions = kwargs.get("npartitions", self.npartitions) partition_size = kwargs.get("partition_size", None) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 18f8cf0d12e..adf10b65b73 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -10,13 +10,15 @@ from dask import compute, delayed from dask.base import tokenize from dask.dataframe.core import DataFrame, _concat -from dask.dataframe.shuffle import shuffle_group_get +from dask.dataframe.shuffle import rearrange_by_column, shuffle_group_get from dask.dataframe.utils import group_split_dispatch from dask.highlevelgraph import HighLevelGraph from dask.utils import M, digit, insert import cudf as gd +from .partitionquantiles import partition_quantiles + try: from .explicit_shuffle import explicit_sorted_shuffle @@ -163,7 +165,12 @@ def sort_delayed_frame(parts, by): def set_partitions_pre(s, divisions): partitions = divisions.searchsorted(s, side="right") - 1 - partitions[(s >= divisions.iloc[-1]).values] = len(divisions) - 2 + + # Use searchsorted to avoid string-compare limitations + partitions[divisions.tail(1).searchsorted(s, side="right")] = ( + len(divisions) - 2 + ) + return partitions @@ -407,7 +414,10 @@ def sort_values_experimental( # Make sure first column is numeric # (Cannot handle string column here yet) - if isinstance(df[by[0]]._meta._column, gd.core.column.string.StringColumn): + if divisions is None and isinstance( + df[by[0]]._meta._column, gd.core.column.string.StringColumn + ): + # TODO: Remove when quantile support is added return df.sort_values( by, ignore_index=ignore_index, experimental=False ) @@ -433,67 +443,68 @@ def sort_values_experimental( npartitions = len(explicit_client.cluster.workers) # Step 2 - Calculate new divisions (if necessary) - if not divisions or (use_explicit and len(divisions) != npartitions + 1): + if divisions is None or ( + use_explicit and len(divisions) != npartitions + 1 + ): # TODO: Use input divisions for use_explicit==True - partition_size = None # 10e6 - repartition = False - if partition_size and not use_explicit: - repartition = True index2 = df2[index] - if repartition: - index2, df2 = dask.base.optimize(index2, df2) - parts = df2.to_delayed(optimize_graph=False) - sizes = [delayed(dask.sizeof.sizeof)(part) for part in parts] - else: - (index2,) = dask.base.optimize(index2) - sizes = [] + (index2,) = dask.base.optimize(index2) doubledivs = ( - index2._repartition_quantiles(npartitions * 2, upsample=upsample) + partition_quantiles(index2, npartitions * 2, upsample=upsample) .compute() .to_list() ) # Heuristic: Start with 2x divisions and coarsening divisions = [doubledivs[i] for i in range(0, len(doubledivs), 2)] divisions[-1] += 1 # Make sure the last division is large enough + else: + # For now we can accept multi-column divisions as a dataframe + if isinstance(divisions, gd.DataFrame): + index = by - if repartition: - iparts = index2.to_delayed(optimize_graph=False) - mins = [ipart.min() for ipart in iparts] - maxes = [ipart.max() for ipart in iparts] - sizes, mins, maxes = dask.base.optimize(sizes, mins, maxes) - sizes, mins, maxes = dask.base.compute( - sizes, mins, maxes, optimize_graph=False - ) - - total = sum(sizes) - npartitions = max(math.ceil(total / partition_size), 1) - npartitions = min(npartitions, df2.npartitions) - n = len(divisions) - try: - divisions = np.interp( - x=np.linspace(0, n - 1, npartitions + 1), - xp=np.linspace(0, n - 1, n), - fp=divisions, - ).tolist() - except (TypeError, ValueError): # str type - indexes = np.linspace(0, n - 1, npartitions + 1).astype(int) - divisions = [divisions[i] for i in indexes] + # Make sure index is a list + if not isinstance(index, list): + index = [index] # Step 3 - Perform repartitioning shuffle sort_by = None if sorted_split: sort_by = by - if use_explicit: + if use_explicit and len(index) == 1: + # TODO: Handle len(index) > 1 warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_sorted_shuffle( - df2, index, divisions, sort_by, explicit_client + df2, index[0], divisions, sort_by, explicit_client ) - else: + elif sorted_split and len(index) == 1: + # Need to pass around divisions + # TODO: Handle len(index) > 1 df3 = rearrange_by_division_list( - df2, index, divisions, max_branch=max_branch, sort_by=sort_by + df2, index[0], divisions, max_branch=max_branch, sort_by=sort_by + ) + else: + # Lets assign a new partitions column + # (That is: Use main-line dask shuffle) + # TODO: Handle len(index) > 1 + meta = df2._meta._constructor_sliced([0]) + # meta = df2._meta_nonempty + if not isinstance(divisions, (gd.Series, gd.DataFrame)): + dtype = df2[index[0]].dtype + divisions = df2._meta._constructor_sliced(divisions, dtype=dtype) + + partitions = df2[index].map_partitions( + set_partitions_pre, divisions=divisions, meta=meta ) + df2b = df2.assign(_partitions=partitions) + df3 = rearrange_by_column( + df2b, + "_partitions", + max_branch=max_branch, + npartitions=len(divisions) - 1, + shuffle="tasks", + ).drop(columns=["_partitions"]) df3.divisions = (None,) * (df3.npartitions + 1) # Step 4 - Return final sorted df From f6204d2ba19463f1e77bdae07bd4c54ecf1ca63f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 20 Feb 2020 07:29:18 -0800 Subject: [PATCH 40/61] multi-column sorting works if dataframe-based are provided by the user --- python/dask_cudf/dask_cudf/core.py | 4 ++++ python/dask_cudf/dask_cudf/sorting.py | 14 +++++++++----- 2 files changed, 13 insertions(+), 5 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 8c9c5c5ee83..077a1342d40 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -186,12 +186,15 @@ def set_index( by = other if not isinstance(other, list): by = [by] + if len(by) > 1: + raise ValueError("Dask does not support MultiIndex (yet).") df = self.sort_values( by, experimental=experimental, explicit_client=explicit_client, max_branch=max_branch, divisions=divisions, + set_divisions=True, sorted_split=kwargs.get("sorted_split", False), upsample=kwargs.get("upsample", 1.0), ) @@ -278,6 +281,7 @@ def sort_values( sorted_split=sorted_split, upsample=upsample, divisions=kwargs.get("divisions", None), + set_divisions=kwargs.get("set_divisions", False), ) else: # Legacy sorting algorithm based on "batcher-sortnet" diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index adf10b65b73..30a75f3309c 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -167,9 +167,9 @@ def set_partitions_pre(s, divisions): partitions = divisions.searchsorted(s, side="right") - 1 # Use searchsorted to avoid string-compare limitations - partitions[divisions.tail(1).searchsorted(s, side="right")] = ( - len(divisions) - 2 - ) + partitions[ + divisions.tail(1).searchsorted(s, side="right").astype("bool") + ] = (len(divisions) - 2) return partitions @@ -397,6 +397,7 @@ def sort_values_experimental( divisions=None, sorted_split=False, upsample=1.0, + set_divisions=False, ): """ Experimental sort_values implementation. @@ -489,7 +490,6 @@ def sort_values_experimental( # (That is: Use main-line dask shuffle) # TODO: Handle len(index) > 1 meta = df2._meta._constructor_sliced([0]) - # meta = df2._meta_nonempty if not isinstance(divisions, (gd.Series, gd.DataFrame)): dtype = df2[index[0]].dtype divisions = df2._meta._constructor_sliced(divisions, dtype=dtype) @@ -497,6 +497,7 @@ def sort_values_experimental( partitions = df2[index].map_partitions( set_partitions_pre, divisions=divisions, meta=meta ) + df2b = df2.assign(_partitions=partitions) df3 = rearrange_by_column( df2b, @@ -513,5 +514,8 @@ def sort_values_experimental( df4 = df3 else: df4 = df3.map_partitions(M.sort_values, by) - df4.divisions = tuple(divisions) + + if not isinstance(divisions, gd.DataFrame) and set_divisions: + # Can't have multi-column divisions elsewhere in dask (yet) + df4.divisions = tuple(divisions) return df4 From ce7070725efcbe528fdd877b755e914a6047387b Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 20 Feb 2020 07:37:04 -0800 Subject: [PATCH 41/61] missing file and notes --- .../dask_cudf/dask_cudf/partitionquantiles.py | 448 ++++++++++++++++++ python/dask_cudf/dask_cudf/sorting.py | 5 +- 2 files changed, 451 insertions(+), 2 deletions(-) create mode 100644 python/dask_cudf/dask_cudf/partitionquantiles.py diff --git a/python/dask_cudf/dask_cudf/partitionquantiles.py b/python/dask_cudf/dask_cudf/partitionquantiles.py new file mode 100644 index 00000000000..ec0289b84d7 --- /dev/null +++ b/python/dask_cudf/dask_cudf/partitionquantiles.py @@ -0,0 +1,448 @@ +# Copied from dask.dataframe +# [for temnporary dev purposes only] + +import math +from collections.abc import Iterator +from functools import wraps + +import numpy as np +import pandas as pd +from pandas.api.types import is_datetime64tz_dtype +from toolz import merge, merge_sorted, take + +from dask.base import tokenize +from dask.dataframe.core import Series +from dask.dataframe.utils import is_categorical_dtype +from dask.utils import random_state_data + + +@wraps(np.percentile) +def _percentile(a, q, interpolation="linear"): + n = len(a) + if not len(a): + return None, n + if isinstance(q, Iterator): + q = list(q) + if a.dtype.name == "category": + result = np.percentile(a.codes, q, interpolation=interpolation) + import pandas as pd + + return pd.Categorical.from_codes(result, a.categories, a.ordered), n + if np.issubdtype(a.dtype, np.datetime64): + a2 = a.astype("i8") + result = np.percentile(a2, q, interpolation=interpolation) + return result.astype(a.dtype), n + if not np.issubdtype(a.dtype, np.number): + interpolation = "nearest" + return np.percentile(a, q, interpolation=interpolation), n + + +def sample_percentiles( + num_old, num_new, chunk_length, upsample=1.0, random_state=None +): + """Construct percentiles for a chunk for repartitioning. + + Adapt the number of total percentiles calculated based on the number + of current and new partitions. Returned percentiles include equally + spaced percentiles between [0, 100], and random percentiles. See + detailed discussion below. + + Parameters + ---------- + num_old: int + Number of partitions of the current object + num_new: int + Number of partitions of the new object + chunk_length: int + Number of rows of the partition + upsample : float + Multiplicative factor to increase the number of samples + + Returns + ------- + qs : numpy.ndarray of sorted percentiles between 0, 100 + + Constructing ordered (i.e., not hashed) partitions is hard. Calculating + approximate percentiles for generic objects in an out-of-core fashion is + also hard. Fortunately, partition boundaries don't need to be perfect + in order for partitioning to be effective, so we strive for a "good enough" + method that can scale to many partitions and is reasonably well-behaved for + a wide variety of scenarios. + + Two similar approaches come to mind: (1) take a subsample of every + partition, then find the best new partitions for the combined subsamples; + and (2) calculate equally-spaced percentiles on every partition (a + relatively cheap operation), then merge the results. We do both, but + instead of random samples, we use random percentiles. + + If the number of partitions isn't changing, then the ratio of fixed + percentiles to random percentiles is 2 to 1. If repartitioning goes from + a very high number of partitions to a very low number of partitions, then + we use more random percentiles, because a stochastic approach will be more + stable to potential correlations in the data that may cause a few equally- + spaced partitions to under-sample the data. + + The more partitions there are, then the more total percentiles will get + calculated across all partitions. Squaring the number of partitions + approximately doubles the number of total percentiles calculated, so + num_total_percentiles ~ sqrt(num_partitions). We assume each partition + is approximately the same length. This should provide adequate resolution + and allow the number of partitions to scale. + + For numeric data, one could instead use T-Digest for floats and Q-Digest + for ints to calculate approximate percentiles. Our current method works + for any dtype. + """ + # *waves hands* + random_percentage = 1 / (1 + (4 * num_new / num_old) ** 0.5) + num_percentiles = upsample * num_new * (num_old + 22) ** 0.55 / num_old + num_fixed = int(num_percentiles * (1 - random_percentage)) + 2 + num_random = int(num_percentiles * random_percentage) + 2 + + if num_fixed + num_random + 5 >= chunk_length: + return np.linspace(0, 100, chunk_length + 1) + + if not isinstance(random_state, np.random.RandomState): + random_state = np.random.RandomState(random_state) + + q_fixed = np.linspace(0, 100, num_fixed) + q_random = random_state.rand(num_random) * 100 + q_edges = [60 / (num_fixed - 1), 100 - 60 / (num_fixed - 1)] + qs = np.concatenate([q_fixed, q_random, q_edges, [0, 100]]) + qs.sort() + # Make the divisions between percentiles a little more even + qs = 0.5 * (qs[:-1] + qs[1:]) + return qs + + +def tree_width(N, to_binary=False): + """Generate tree width suitable for ``merge_sorted`` given N inputs + + The larger N is, the more tasks are reduced in a single task. + + In theory, this is designed so all tasks are of comparable effort. + """ + if N < 32: + group_size = 2 + else: + group_size = int(math.log(N)) + num_groups = N // group_size + if to_binary or num_groups < 16: + return 2 ** int(math.log(N / group_size, 2)) + else: + return num_groups + + +def tree_groups(N, num_groups): + """Split an integer N into evenly sized and spaced groups. + + >>> tree_groups(16, 6) + [3, 2, 3, 3, 2, 3] + """ + # Bresenham, you so smooth! + group_size = N // num_groups + dx = num_groups + dy = N - group_size * num_groups + D = 2 * dy - dx + rv = [] + for _ in range(num_groups): + if D < 0: + rv.append(group_size) + else: + rv.append(group_size + 1) + D -= 2 * dx + D += 2 * dy + return rv + + +def create_merge_tree(func, keys, token): + """Create a task tree that merges all the keys with a reduction function. + + Parameters + ---------- + func: callable + Reduction function that accepts a single list of values to reduce. + keys: iterable + Keys to reduce from the source dask graph. + token: object + Included in each key of the returned dict. + + This creates a k-ary tree where k depends on the current level and is + greater the further away a node is from the root node. This reduces the + total number of nodes (thereby reducing scheduler overhead), but still + has beneficial properties of trees. + + For reasonable numbers of keys, N < 1e5, the total number of nodes in the + tree is roughly ``N**0.78``. For 1e5 < N < 2e5, is it roughly ``N**0.8``. + """ + level = 0 + prev_width = len(keys) + prev_keys = iter(keys) + rv = {} + while prev_width > 1: + width = tree_width(prev_width) + groups = tree_groups(prev_width, width) + keys = [(token, level, i) for i in range(width)] + rv.update( + (key, (func, list(take(num, prev_keys)))) + for num, key in zip(groups, keys) + ) + prev_width = width + prev_keys = iter(keys) + level += 1 + return rv + + +def percentiles_to_weights(qs, vals, length): + """Weigh percentile values by length and the difference between percentiles + + >>> percentiles = np.array([0, 25, 50, 90, 100]) + >>> values = np.array([2, 3, 5, 8, 13]) + >>> length = 10 + >>> percentiles_to_weights(percentiles, values, length) + ([2, 3, 5, 8, 13], [125.0, 250.0, 325.0, 250.0, 50.0]) + + The weight of the first element, ``2``, is determined by the difference + between the first and second percentiles, and then scaled by length: + + >>> 0.5 * length * (percentiles[1] - percentiles[0]) + 125.0 + + The second weight uses the difference of percentiles on both sides, so + it will be twice the first weight if the percentiles are equally spaced: + + >>> 0.5 * length * (percentiles[2] - percentiles[0]) + 250.0 + """ + if length == 0: + return () + diff = np.ediff1d(qs, 0.0, 0.0) + weights = 0.5 * length * (diff[1:] + diff[:-1]) + return vals.tolist(), weights.tolist() + + +def merge_and_compress_summaries(vals_and_weights): + """Merge and sort percentile summaries that are already sorted. + + Each item is a tuple like ``(vals, weights)`` where vals and weights + are lists. We sort both by vals. + + Equal values will be combined, their weights summed together. + """ + vals_and_weights = [x for x in vals_and_weights if x] + if not vals_and_weights: + return () + it = merge_sorted(*[zip(x, y) for x, y in vals_and_weights]) + vals = [] + weights = [] + vals_append = vals.append + weights_append = weights.append + val, weight = prev_val, prev_weight = next(it) + for val, weight in it: + if val == prev_val: + prev_weight += weight + else: + vals_append(prev_val) + weights_append(prev_weight) + prev_val, prev_weight = val, weight + if val == prev_val: + vals_append(prev_val) + weights_append(prev_weight) + return vals, weights + + +def process_val_weights(vals_and_weights, npartitions, dtype_info): + """Calculate final approximate percentiles given weighted vals + + ``vals_and_weights`` is assumed to be sorted. We take a cumulative + sum of the weights, which makes them percentile-like (their scale is + [0, N] instead of [0, 100]). Next we find the divisions to create + partitions of approximately equal size. + + It is possible for adjacent values of the result to be the same. Since + these determine the divisions of the new partitions, some partitions + may be empty. This can happen if we under-sample the data, or if there + aren't enough unique values in the column. Increasing ``upsample`` + keyword argument in ``df.set_index`` may help. + """ + dtype, info = dtype_info + + if not vals_and_weights: + try: + return np.array(None, dtype=dtype) + except Exception: + # dtype does not support None value so allow it to change + return np.array(None, dtype=np.float_) + + vals, weights = vals_and_weights + vals = np.array(vals) + weights = np.array(weights) + + # We want to create exactly `npartition` number of groups of `vals` that + # are approximately the same weight and non-empty if possible. We use a + # simple approach (more accurate algorithms exist): + # 1. Remove all the values with weights larger than the relative + # percentile width from consideration (these are `jumbo`s) + # 2. Calculate percentiles with "interpolation=left" of percentile-like + # weights of the remaining values. These are guaranteed to be unique. + # 3. Concatenate the values from (1) and (2), sort, and return. + # + # We assume that all values are unique, which happens in the previous + # step `merge_and_compress_summaries`. + + if len(vals) == npartitions + 1: + rv = vals + elif len(vals) < npartitions + 1: + # The data is under-sampled + if np.issubdtype(vals.dtype, np.number) and not is_categorical_dtype( + dtype + ): + # Interpolate extra divisions + q_weights = np.cumsum(weights) + q_target = np.linspace( + q_weights[0], q_weights[-1], npartitions + 1 + ) + rv = np.interp(q_target, q_weights, vals) + else: + # Distribute the empty partitions + duplicated_index = np.linspace( + 0, len(vals) - 1, npartitions - len(vals) + 1, dtype=int + ) + duplicated_vals = vals[duplicated_index] + rv = np.concatenate([vals, duplicated_vals]) + rv.sort() + else: + target_weight = weights.sum() / npartitions + jumbo_mask = weights >= target_weight + jumbo_vals = vals[jumbo_mask] + + trimmed_vals = vals[~jumbo_mask] + trimmed_weights = weights[~jumbo_mask] + trimmed_npartitions = npartitions - len(jumbo_vals) + + # percentile-like, but scaled by weights + q_weights = np.cumsum(trimmed_weights) + q_target = np.linspace(0, q_weights[-1], trimmed_npartitions + 1) + + left = np.searchsorted(q_weights, q_target, side="left") + right = np.searchsorted(q_weights, q_target, side="right") - 1 + # stay inbounds + np.maximum(right, 0, right) + lower = np.minimum(left, right) + trimmed = trimmed_vals[lower] + + rv = np.concatenate([trimmed, jumbo_vals]) + rv.sort() + + if is_categorical_dtype(dtype): + rv = pd.Categorical.from_codes(rv, info[0], info[1]) + elif is_datetime64tz_dtype(dtype): + rv = pd.DatetimeIndex(rv).tz_localize(dtype.tz) + elif "datetime64" in str(dtype): + rv = pd.DatetimeIndex(rv, dtype=dtype) + elif rv.dtype != dtype: + rv = rv.astype(dtype) + return rv + + +def percentiles_summary(df, num_old, num_new, upsample, state): + """Summarize data using percentiles and derived weights. + + These summaries can be merged, compressed, and converted back into + approximate percentiles. + + Parameters + ---------- + df: pandas.Series + Data to summarize + num_old: int + Number of partitions of the current object + num_new: int + Number of partitions of the new object + upsample: float + Scale factor to increase the number of percentiles calculated in + each partition. Use to improve accuracy. + """ + length = len(df) + if length == 0: + return () + random_state = np.random.RandomState(state) + qs = sample_percentiles(num_old, num_new, length, upsample, random_state) + data = df.values + interpolation = "linear" + if is_categorical_dtype(data): + data = data.codes + interpolation = "nearest" + vals, n = _percentile(data, qs, interpolation=interpolation) + if interpolation == "linear" and np.issubdtype(data.dtype, np.integer): + vals = np.round(vals).astype(data.dtype) + vals_and_weights = percentiles_to_weights(qs, vals, length) + return vals_and_weights + + +def dtype_info(df): + info = None + if is_categorical_dtype(df): + data = df.values + info = (data.categories, data.ordered) + return df.dtype, info + + +def partition_quantiles(df, npartitions, upsample=1.0, random_state=None): + """ Approximate quantiles of Series used for repartitioning + """ + assert isinstance(df, Series) + # currently, only Series has quantile method + # Index.quantile(list-like) must be pd.Series, not pd.Index + return_type = Series + + qs = np.linspace(0, 1, npartitions + 1) + token = tokenize(df, qs, upsample) + if random_state is None: + random_state = int(token, 16) % np.iinfo(np.int32).max + state_data = random_state_data(df.npartitions, random_state) + + df_keys = df.__dask_keys__() + + name0 = "re-quantiles-0-" + token + dtype_dsk = {(name0, 0): (dtype_info, df_keys[0])} + + name1 = "re-quantiles-1-" + token + val_dsk = { + (name1, i): ( + percentiles_summary, + key, + df.npartitions, + npartitions, + upsample, + state, + ) + for i, (state, key) in enumerate(zip(state_data, df_keys)) + } + + name2 = "re-quantiles-2-" + token + merge_dsk = create_merge_tree( + merge_and_compress_summaries, sorted(val_dsk), name2 + ) + if not merge_dsk: + # Compress the data even if we only have one partition + merge_dsk = { + (name2, 0, 0): (merge_and_compress_summaries, [list(val_dsk)[0]]) + } + + merged_key = max(merge_dsk) + + name3 = "re-quantiles-3-" + token + last_dsk = { + (name3, 0): ( + pd.Series, # TODO: Use `type(df._meta)` when cudf adds `tolist()` + (process_val_weights, merged_key, npartitions, (name0, 0)), + qs, + None, + df.name, + ) + } + + dsk = merge(df.dask, dtype_dsk, val_dsk, merge_dsk, last_dsk) + new_divisions = [0.0, 1.0] + return return_type(dsk, name3, df._meta, new_divisions) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 30a75f3309c..0ae47455b81 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -429,8 +429,9 @@ def sort_values_experimental( else: df2 = df - # Only handle single-column partitioning (for now) - # TODO: Handle partitioning on multiple columns? + # Only handle single-column partitioning (for now), + # UNLESS multi-column divisions are provided by user + # TODO: Add multicolumn quantiles logic... if len(by) > 1: warnings.warn( "Using experimental version of sort_values." From 9e3f398298c9603cafc7c967409d36278ce1689f Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 20 Feb 2020 07:58:55 -0800 Subject: [PATCH 42/61] merge cleanup --- python/cudf/cudf/_libxx/lib.pxd | 8 -------- 1 file changed, 8 deletions(-) diff --git a/python/cudf/cudf/_libxx/lib.pxd b/python/cudf/cudf/_libxx/lib.pxd index abe7bf07faf..68bba368615 100644 --- a/python/cudf/cudf/_libxx/lib.pxd +++ b/python/cudf/cudf/_libxx/lib.pxd @@ -17,14 +17,6 @@ cdef extern from "cudf/types.hpp" namespace "cudf" nogil: ctypedef int32_t size_type ctypedef uint32_t bitmask_type - ctypedef enum order: - ASCENDING "cudf::order::ASCENDING" - DESCENDING "cudf::order::DESCENDING" - - ctypedef enum null_order: - AFTER "cudf::null_order::AFTER" - BEFORE "cudf::null_order::BEFORE" - cdef enum: UNKNOWN_NULL_COUNT = -1 From 7aeee20d9568c9d8a3689ffd06adf72474fa5732 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 27 Feb 2020 22:03:42 -0800 Subject: [PATCH 43/61] balanced partitions for single column --- .../dask_cudf/dask_cudf/partitionquantiles.py | 448 ------------------ python/dask_cudf/dask_cudf/sorting.py | 107 ++++- 2 files changed, 91 insertions(+), 464 deletions(-) delete mode 100644 python/dask_cudf/dask_cudf/partitionquantiles.py diff --git a/python/dask_cudf/dask_cudf/partitionquantiles.py b/python/dask_cudf/dask_cudf/partitionquantiles.py deleted file mode 100644 index ec0289b84d7..00000000000 --- a/python/dask_cudf/dask_cudf/partitionquantiles.py +++ /dev/null @@ -1,448 +0,0 @@ -# Copied from dask.dataframe -# [for temnporary dev purposes only] - -import math -from collections.abc import Iterator -from functools import wraps - -import numpy as np -import pandas as pd -from pandas.api.types import is_datetime64tz_dtype -from toolz import merge, merge_sorted, take - -from dask.base import tokenize -from dask.dataframe.core import Series -from dask.dataframe.utils import is_categorical_dtype -from dask.utils import random_state_data - - -@wraps(np.percentile) -def _percentile(a, q, interpolation="linear"): - n = len(a) - if not len(a): - return None, n - if isinstance(q, Iterator): - q = list(q) - if a.dtype.name == "category": - result = np.percentile(a.codes, q, interpolation=interpolation) - import pandas as pd - - return pd.Categorical.from_codes(result, a.categories, a.ordered), n - if np.issubdtype(a.dtype, np.datetime64): - a2 = a.astype("i8") - result = np.percentile(a2, q, interpolation=interpolation) - return result.astype(a.dtype), n - if not np.issubdtype(a.dtype, np.number): - interpolation = "nearest" - return np.percentile(a, q, interpolation=interpolation), n - - -def sample_percentiles( - num_old, num_new, chunk_length, upsample=1.0, random_state=None -): - """Construct percentiles for a chunk for repartitioning. - - Adapt the number of total percentiles calculated based on the number - of current and new partitions. Returned percentiles include equally - spaced percentiles between [0, 100], and random percentiles. See - detailed discussion below. - - Parameters - ---------- - num_old: int - Number of partitions of the current object - num_new: int - Number of partitions of the new object - chunk_length: int - Number of rows of the partition - upsample : float - Multiplicative factor to increase the number of samples - - Returns - ------- - qs : numpy.ndarray of sorted percentiles between 0, 100 - - Constructing ordered (i.e., not hashed) partitions is hard. Calculating - approximate percentiles for generic objects in an out-of-core fashion is - also hard. Fortunately, partition boundaries don't need to be perfect - in order for partitioning to be effective, so we strive for a "good enough" - method that can scale to many partitions and is reasonably well-behaved for - a wide variety of scenarios. - - Two similar approaches come to mind: (1) take a subsample of every - partition, then find the best new partitions for the combined subsamples; - and (2) calculate equally-spaced percentiles on every partition (a - relatively cheap operation), then merge the results. We do both, but - instead of random samples, we use random percentiles. - - If the number of partitions isn't changing, then the ratio of fixed - percentiles to random percentiles is 2 to 1. If repartitioning goes from - a very high number of partitions to a very low number of partitions, then - we use more random percentiles, because a stochastic approach will be more - stable to potential correlations in the data that may cause a few equally- - spaced partitions to under-sample the data. - - The more partitions there are, then the more total percentiles will get - calculated across all partitions. Squaring the number of partitions - approximately doubles the number of total percentiles calculated, so - num_total_percentiles ~ sqrt(num_partitions). We assume each partition - is approximately the same length. This should provide adequate resolution - and allow the number of partitions to scale. - - For numeric data, one could instead use T-Digest for floats and Q-Digest - for ints to calculate approximate percentiles. Our current method works - for any dtype. - """ - # *waves hands* - random_percentage = 1 / (1 + (4 * num_new / num_old) ** 0.5) - num_percentiles = upsample * num_new * (num_old + 22) ** 0.55 / num_old - num_fixed = int(num_percentiles * (1 - random_percentage)) + 2 - num_random = int(num_percentiles * random_percentage) + 2 - - if num_fixed + num_random + 5 >= chunk_length: - return np.linspace(0, 100, chunk_length + 1) - - if not isinstance(random_state, np.random.RandomState): - random_state = np.random.RandomState(random_state) - - q_fixed = np.linspace(0, 100, num_fixed) - q_random = random_state.rand(num_random) * 100 - q_edges = [60 / (num_fixed - 1), 100 - 60 / (num_fixed - 1)] - qs = np.concatenate([q_fixed, q_random, q_edges, [0, 100]]) - qs.sort() - # Make the divisions between percentiles a little more even - qs = 0.5 * (qs[:-1] + qs[1:]) - return qs - - -def tree_width(N, to_binary=False): - """Generate tree width suitable for ``merge_sorted`` given N inputs - - The larger N is, the more tasks are reduced in a single task. - - In theory, this is designed so all tasks are of comparable effort. - """ - if N < 32: - group_size = 2 - else: - group_size = int(math.log(N)) - num_groups = N // group_size - if to_binary or num_groups < 16: - return 2 ** int(math.log(N / group_size, 2)) - else: - return num_groups - - -def tree_groups(N, num_groups): - """Split an integer N into evenly sized and spaced groups. - - >>> tree_groups(16, 6) - [3, 2, 3, 3, 2, 3] - """ - # Bresenham, you so smooth! - group_size = N // num_groups - dx = num_groups - dy = N - group_size * num_groups - D = 2 * dy - dx - rv = [] - for _ in range(num_groups): - if D < 0: - rv.append(group_size) - else: - rv.append(group_size + 1) - D -= 2 * dx - D += 2 * dy - return rv - - -def create_merge_tree(func, keys, token): - """Create a task tree that merges all the keys with a reduction function. - - Parameters - ---------- - func: callable - Reduction function that accepts a single list of values to reduce. - keys: iterable - Keys to reduce from the source dask graph. - token: object - Included in each key of the returned dict. - - This creates a k-ary tree where k depends on the current level and is - greater the further away a node is from the root node. This reduces the - total number of nodes (thereby reducing scheduler overhead), but still - has beneficial properties of trees. - - For reasonable numbers of keys, N < 1e5, the total number of nodes in the - tree is roughly ``N**0.78``. For 1e5 < N < 2e5, is it roughly ``N**0.8``. - """ - level = 0 - prev_width = len(keys) - prev_keys = iter(keys) - rv = {} - while prev_width > 1: - width = tree_width(prev_width) - groups = tree_groups(prev_width, width) - keys = [(token, level, i) for i in range(width)] - rv.update( - (key, (func, list(take(num, prev_keys)))) - for num, key in zip(groups, keys) - ) - prev_width = width - prev_keys = iter(keys) - level += 1 - return rv - - -def percentiles_to_weights(qs, vals, length): - """Weigh percentile values by length and the difference between percentiles - - >>> percentiles = np.array([0, 25, 50, 90, 100]) - >>> values = np.array([2, 3, 5, 8, 13]) - >>> length = 10 - >>> percentiles_to_weights(percentiles, values, length) - ([2, 3, 5, 8, 13], [125.0, 250.0, 325.0, 250.0, 50.0]) - - The weight of the first element, ``2``, is determined by the difference - between the first and second percentiles, and then scaled by length: - - >>> 0.5 * length * (percentiles[1] - percentiles[0]) - 125.0 - - The second weight uses the difference of percentiles on both sides, so - it will be twice the first weight if the percentiles are equally spaced: - - >>> 0.5 * length * (percentiles[2] - percentiles[0]) - 250.0 - """ - if length == 0: - return () - diff = np.ediff1d(qs, 0.0, 0.0) - weights = 0.5 * length * (diff[1:] + diff[:-1]) - return vals.tolist(), weights.tolist() - - -def merge_and_compress_summaries(vals_and_weights): - """Merge and sort percentile summaries that are already sorted. - - Each item is a tuple like ``(vals, weights)`` where vals and weights - are lists. We sort both by vals. - - Equal values will be combined, their weights summed together. - """ - vals_and_weights = [x for x in vals_and_weights if x] - if not vals_and_weights: - return () - it = merge_sorted(*[zip(x, y) for x, y in vals_and_weights]) - vals = [] - weights = [] - vals_append = vals.append - weights_append = weights.append - val, weight = prev_val, prev_weight = next(it) - for val, weight in it: - if val == prev_val: - prev_weight += weight - else: - vals_append(prev_val) - weights_append(prev_weight) - prev_val, prev_weight = val, weight - if val == prev_val: - vals_append(prev_val) - weights_append(prev_weight) - return vals, weights - - -def process_val_weights(vals_and_weights, npartitions, dtype_info): - """Calculate final approximate percentiles given weighted vals - - ``vals_and_weights`` is assumed to be sorted. We take a cumulative - sum of the weights, which makes them percentile-like (their scale is - [0, N] instead of [0, 100]). Next we find the divisions to create - partitions of approximately equal size. - - It is possible for adjacent values of the result to be the same. Since - these determine the divisions of the new partitions, some partitions - may be empty. This can happen if we under-sample the data, or if there - aren't enough unique values in the column. Increasing ``upsample`` - keyword argument in ``df.set_index`` may help. - """ - dtype, info = dtype_info - - if not vals_and_weights: - try: - return np.array(None, dtype=dtype) - except Exception: - # dtype does not support None value so allow it to change - return np.array(None, dtype=np.float_) - - vals, weights = vals_and_weights - vals = np.array(vals) - weights = np.array(weights) - - # We want to create exactly `npartition` number of groups of `vals` that - # are approximately the same weight and non-empty if possible. We use a - # simple approach (more accurate algorithms exist): - # 1. Remove all the values with weights larger than the relative - # percentile width from consideration (these are `jumbo`s) - # 2. Calculate percentiles with "interpolation=left" of percentile-like - # weights of the remaining values. These are guaranteed to be unique. - # 3. Concatenate the values from (1) and (2), sort, and return. - # - # We assume that all values are unique, which happens in the previous - # step `merge_and_compress_summaries`. - - if len(vals) == npartitions + 1: - rv = vals - elif len(vals) < npartitions + 1: - # The data is under-sampled - if np.issubdtype(vals.dtype, np.number) and not is_categorical_dtype( - dtype - ): - # Interpolate extra divisions - q_weights = np.cumsum(weights) - q_target = np.linspace( - q_weights[0], q_weights[-1], npartitions + 1 - ) - rv = np.interp(q_target, q_weights, vals) - else: - # Distribute the empty partitions - duplicated_index = np.linspace( - 0, len(vals) - 1, npartitions - len(vals) + 1, dtype=int - ) - duplicated_vals = vals[duplicated_index] - rv = np.concatenate([vals, duplicated_vals]) - rv.sort() - else: - target_weight = weights.sum() / npartitions - jumbo_mask = weights >= target_weight - jumbo_vals = vals[jumbo_mask] - - trimmed_vals = vals[~jumbo_mask] - trimmed_weights = weights[~jumbo_mask] - trimmed_npartitions = npartitions - len(jumbo_vals) - - # percentile-like, but scaled by weights - q_weights = np.cumsum(trimmed_weights) - q_target = np.linspace(0, q_weights[-1], trimmed_npartitions + 1) - - left = np.searchsorted(q_weights, q_target, side="left") - right = np.searchsorted(q_weights, q_target, side="right") - 1 - # stay inbounds - np.maximum(right, 0, right) - lower = np.minimum(left, right) - trimmed = trimmed_vals[lower] - - rv = np.concatenate([trimmed, jumbo_vals]) - rv.sort() - - if is_categorical_dtype(dtype): - rv = pd.Categorical.from_codes(rv, info[0], info[1]) - elif is_datetime64tz_dtype(dtype): - rv = pd.DatetimeIndex(rv).tz_localize(dtype.tz) - elif "datetime64" in str(dtype): - rv = pd.DatetimeIndex(rv, dtype=dtype) - elif rv.dtype != dtype: - rv = rv.astype(dtype) - return rv - - -def percentiles_summary(df, num_old, num_new, upsample, state): - """Summarize data using percentiles and derived weights. - - These summaries can be merged, compressed, and converted back into - approximate percentiles. - - Parameters - ---------- - df: pandas.Series - Data to summarize - num_old: int - Number of partitions of the current object - num_new: int - Number of partitions of the new object - upsample: float - Scale factor to increase the number of percentiles calculated in - each partition. Use to improve accuracy. - """ - length = len(df) - if length == 0: - return () - random_state = np.random.RandomState(state) - qs = sample_percentiles(num_old, num_new, length, upsample, random_state) - data = df.values - interpolation = "linear" - if is_categorical_dtype(data): - data = data.codes - interpolation = "nearest" - vals, n = _percentile(data, qs, interpolation=interpolation) - if interpolation == "linear" and np.issubdtype(data.dtype, np.integer): - vals = np.round(vals).astype(data.dtype) - vals_and_weights = percentiles_to_weights(qs, vals, length) - return vals_and_weights - - -def dtype_info(df): - info = None - if is_categorical_dtype(df): - data = df.values - info = (data.categories, data.ordered) - return df.dtype, info - - -def partition_quantiles(df, npartitions, upsample=1.0, random_state=None): - """ Approximate quantiles of Series used for repartitioning - """ - assert isinstance(df, Series) - # currently, only Series has quantile method - # Index.quantile(list-like) must be pd.Series, not pd.Index - return_type = Series - - qs = np.linspace(0, 1, npartitions + 1) - token = tokenize(df, qs, upsample) - if random_state is None: - random_state = int(token, 16) % np.iinfo(np.int32).max - state_data = random_state_data(df.npartitions, random_state) - - df_keys = df.__dask_keys__() - - name0 = "re-quantiles-0-" + token - dtype_dsk = {(name0, 0): (dtype_info, df_keys[0])} - - name1 = "re-quantiles-1-" + token - val_dsk = { - (name1, i): ( - percentiles_summary, - key, - df.npartitions, - npartitions, - upsample, - state, - ) - for i, (state, key) in enumerate(zip(state_data, df_keys)) - } - - name2 = "re-quantiles-2-" + token - merge_dsk = create_merge_tree( - merge_and_compress_summaries, sorted(val_dsk), name2 - ) - if not merge_dsk: - # Compress the data even if we only have one partition - merge_dsk = { - (name2, 0, 0): (merge_and_compress_summaries, [list(val_dsk)[0]]) - } - - merged_key = max(merge_dsk) - - name3 = "re-quantiles-3-" + token - last_dsk = { - (name3, 0): ( - pd.Series, # TODO: Use `type(df._meta)` when cudf adds `tolist()` - (process_val_weights, merged_key, npartitions, (name0, 0)), - qs, - None, - df.name, - ) - } - - dsk = merge(df.dask, dtype_dsk, val_dsk, merge_dsk, last_dsk) - new_divisions = [0.0, 1.0] - return return_type(dsk, name3, df._meta, new_divisions) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 0ae47455b81..b535c516f24 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -3,13 +3,13 @@ import warnings from operator import getitem +import cupy import numpy as np import toolz -import dask from dask import compute, delayed from dask.base import tokenize -from dask.dataframe.core import DataFrame, _concat +from dask.dataframe.core import DataFrame, Index, Series, _concat from dask.dataframe.shuffle import rearrange_by_column, shuffle_group_get from dask.dataframe.utils import group_split_dispatch from dask.highlevelgraph import HighLevelGraph @@ -17,8 +17,6 @@ import cudf as gd -from .partitionquantiles import partition_quantiles - try: from .explicit_shuffle import explicit_sorted_shuffle @@ -388,6 +386,90 @@ def rearrange_by_division_list( return df3 +def _percentile(a, q, interpolation="linear"): + n = len(a) + if not len(a): + return None, n + return ( + cupy.asnumpy( + gd.Series(a).quantile(q, interpolation=interpolation).values + ), + n, + ) + + +def quantile(df, q): + """Approximate quantiles of Series. + Parameters + ---------- + q : list/array of floats + Iterable of numbers ranging from 0 to 100 for the desired quantiles + """ + # current implementation needs q to be sorted so + # sort if array-like, otherwise leave it alone + q_ndarray = np.array(q) + if q_ndarray.ndim > 0: + q_ndarray.sort(kind="mergesort") + q = q_ndarray + + assert isinstance(df, Series) + + # currently, only Series has quantile method + if isinstance(df, Index): + meta = gd.Series(df._meta_nonempty).quantile(q=q) + else: + meta = df._meta_nonempty.quantile(q=q) + + # Index.quantile(list-like) must be pd.Series, not pd.Index + df_name = df.name + + def finalize_tsk(tsk): + return (gd.Series, tsk, q, None, df_name) + + return_type = Series + + # pandas uses quantile in [0, 1] + # numpy / everyone else uses [0, 100] + qs = np.asarray(q) # * 100 + token = tokenize(df, qs) + + if len(qs) == 0: + name = "quantiles-" + token + empty_index = gd.Index([], dtype=float) + return Series( + { + (name, 0): gd.Series( + [], name=df.name, index=empty_index, dtype="float" + ) + }, + name, + df._meta, + [None, None], + ) + else: + new_divisions = [np.min(q), np.max(q)] + + df = df.dropna() + + from dask.array.percentile import merge_percentiles + + name = "quantiles-1-" + token + val_dsk = { + (name, i): (_percentile, (getattr, key, "values"), qs) + for i, key in enumerate(df.__dask_keys__()) + } + + name2 = "quantiles-2-" + token + merge_dsk = { + (name2, 0): finalize_tsk( + (merge_percentiles, qs, [qs] * df.npartitions, sorted(val_dsk)) + ) + } + dsk = toolz.merge(val_dsk, merge_dsk) + graph = HighLevelGraph.from_collections(name2, dsk, dependencies=[df]) + return return_type(graph, name2, meta, new_divisions) + + def sort_values_experimental( df, by, @@ -449,18 +531,11 @@ def sort_values_experimental( use_explicit and len(divisions) != npartitions + 1 ): # TODO: Use input divisions for use_explicit==True - - index2 = df2[index] - (index2,) = dask.base.optimize(index2) - - doubledivs = ( - partition_quantiles(index2, npartitions * 2, upsample=upsample) - .compute() - .to_list() - ) - # Heuristic: Start with 2x divisions and coarsening - divisions = [doubledivs[i] for i in range(0, len(doubledivs), 2)] - divisions[-1] += 1 # Make sure the last division is large enough + qn = np.linspace(0.0, 1.0, npartitions + 1).tolist() + divisions = ( + quantile(df2[index], qn).astype("int").compute().values + 1 + ).tolist() + divisions[0] = 0 else: # For now we can accept multi-column divisions as a dataframe if isinstance(divisions, gd.DataFrame): From 82180cf25495f9559c49bf991042d65ba9de7e09 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 06:49:47 -0800 Subject: [PATCH 44/61] use faster version of quantiles (numpy percentile with casting) for now --- python/dask_cudf/dask_cudf/sorting.py | 19 +++++++++++++------ 1 file changed, 13 insertions(+), 6 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index b535c516f24..a62d5973104 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -390,9 +390,15 @@ def _percentile(a, q, interpolation="linear"): n = len(a) if not len(a): return None, n + # return ( + # cupy.asnumpy( + # gd.Series(a).quantile(q, interpolation=interpolation).values + # ), + # n, + # ) return ( cupy.asnumpy( - gd.Series(a).quantile(q, interpolation=interpolation).values + np.percentile(a.astype("float64"), q, interpolation=interpolation) ), n, ) @@ -430,7 +436,7 @@ def finalize_tsk(tsk): # pandas uses quantile in [0, 1] # numpy / everyone else uses [0, 100] - qs = np.asarray(q) # * 100 + qs = np.asarray(q) * 100 token = tokenize(df, qs) if len(qs) == 0: @@ -449,8 +455,6 @@ def finalize_tsk(tsk): else: new_divisions = [np.min(q), np.max(q)] - df = df.dropna() - from dask.array.percentile import merge_percentiles name = "quantiles-1-" + token @@ -532,9 +536,12 @@ def sort_values_experimental( ): # TODO: Use input divisions for use_explicit==True qn = np.linspace(0.0, 1.0, npartitions + 1).tolist() + dtype = df2[index].dtype divisions = ( - quantile(df2[index], qn).astype("int").compute().values + 1 - ).tolist() + (quantile(df2[index], qn).compute().values + 1) + .astype(dtype) + .tolist() + ) divisions[0] = 0 else: # For now we can accept multi-column divisions as a dataframe From 2b23d04fc5b0ef9de15f7a069e572fcbd513a59e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 07:13:44 -0800 Subject: [PATCH 45/61] use cudf quantiles --- python/dask_cudf/dask_cudf/sorting.py | 138 ++++++++++++++++++++++++-- 1 file changed, 127 insertions(+), 11 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index a62d5973104..ef896cd7600 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -1,6 +1,7 @@ # Copyright (c) 2020, NVIDIA CORPORATION. import math import warnings +from collections.abc import Iterator from operator import getitem import cupy @@ -390,20 +391,135 @@ def _percentile(a, q, interpolation="linear"): n = len(a) if not len(a): return None, n - # return ( - # cupy.asnumpy( - # gd.Series(a).quantile(q, interpolation=interpolation).values - # ), - # n, - # ) return ( - cupy.asnumpy( - np.percentile(a.astype("float64"), q, interpolation=interpolation) - ), + cupy.asnumpy(gd.DataFrame({"a": a}).quantiles(q.tolist()).values), n, ) +def merge_percentiles(finalq, qs, vals, interpolation="lower", Ns=None): + """ Combine several percentile calculations of different data. + [NOTE: Mostly copied from dask.array] + + Parameters + ---------- + + finalq : numpy.array + Percentiles to compute (must use same scale as ``qs``). + qs : sequence of :class:`numpy.array`s + Percentiles calculated on different sets of data. + vals : sequence of :class:`numpy.array`s + Resulting values associated with percentiles ``qs``. + Ns : sequence of integers + The number of data elements associated with each data set. + interpolation : {'linear', 'lower', 'higher', 'midpoint', 'nearest'} + Specify the type of interpolation to use to calculate final + percentiles. For more information, see :func:`numpy.percentile`. + + Examples + -------- + + >>> finalq = [10, 20, 30, 40, 50, 60, 70, 80] + >>> qs = [[20, 40, 60, 80], [20, 40, 60, 80]] + >>> vals = [np.array([1, 2, 3, 4]), np.array([10, 11, 12, 13])] + >>> Ns = [100, 100] # Both original arrays had 100 elements + + >>> merge_percentiles(finalq, qs, vals, Ns=Ns) + array([ 1, 2, 3, 4, 10, 11, 12, 13]) + """ + if isinstance(finalq, Iterator): + finalq = list(finalq) + finalq = np.array(finalq) + qs = list(map(list, qs)) + vals = list(vals) + if Ns is None: + vals, Ns = zip(*vals) + Ns = list(Ns) + + L = list(zip(*[(q, val, N) for q, val, N in zip(qs, vals, Ns) if N])) + if not L: + raise ValueError("No non-trivial arrays found") + qs, vals, Ns = L + + # TODO: Perform this check above in percentile once dtype checking is easy + # Here we silently change meaning + if vals[0].dtype.name == "category": + result = merge_percentiles( + finalq, qs, [v.codes for v in vals], interpolation, Ns + ) + import pandas as pd + + return pd.Categorical.from_codes( + result, vals[0].categories, vals[0].ordered + ) + if not np.issubdtype(vals[0].dtype, np.number): + interpolation = "nearest" + + if len(vals) != len(qs) or len(Ns) != len(qs): + raise ValueError("qs, vals, and Ns parameters must be the same length") + + # transform qs and Ns into number of observations between percentiles + counts = [] + for q, N in zip(qs, Ns): + count = np.empty(len(q)) + count[1:] = np.diff(q) + count[0] = q[0] + count *= N + counts.append(count) + + # Sort by calculated percentile values, then number of observations. + # >95% of the time in this function is spent in `merge_sorted` below. + # An alternative that uses numpy sort is shown. It is sometimes + # comparable to, but typically slower than, `merge_sorted`. + # + # >>> A = np.concatenate(map(np.array, map(zip, vals, counts))) + # >>> A.sort(0, kind='mergesort') + + combined_vals_counts = toolz.merge_sorted(*map(zip, vals, counts)) + combined_vals, combined_counts = zip(*combined_vals_counts) + + combined_vals = np.array(combined_vals) + combined_counts = np.array(combined_counts) + + # percentile-like, but scaled by total number of observations + combined_q = np.cumsum(combined_counts) + + # rescale finalq percentiles to match combined_q + desired_q = finalq * sum(Ns) + + # the behavior of different interpolation methods should be + # investigated further. + if interpolation == "linear": + rv = np.interp(desired_q, combined_q, combined_vals) + else: + left = np.searchsorted(combined_q, desired_q, side="left") + right = np.searchsorted(combined_q, desired_q, side="right") - 1 + np.minimum( + left, len(combined_vals) - 1, left + ) # don't exceed max index + lower = np.minimum(left, right) + upper = np.maximum(left, right) + if interpolation == "lower": + rv = combined_vals[lower] + elif interpolation == "higher": + rv = combined_vals[upper] + elif interpolation == "midpoint": + rv = 0.5 * (combined_vals[lower] + combined_vals[upper]) + elif interpolation == "nearest": + lower_residual = np.abs(combined_q[lower] - desired_q) + upper_residual = np.abs(combined_q[upper] - desired_q) + mask = lower_residual > upper_residual + index = lower # alias; we no longer need lower + index[mask] = upper[mask] + rv = combined_vals[index] + else: + raise ValueError( + "interpolation can only be 'linear', 'lower', " + "'higher', 'midpoint', or 'nearest'" + ) + return rv + + def quantile(df, q): """Approximate quantiles of Series. Parameters @@ -436,7 +552,7 @@ def finalize_tsk(tsk): # pandas uses quantile in [0, 1] # numpy / everyone else uses [0, 100] - qs = np.asarray(q) * 100 + qs = np.asarray(q) # * 100 token = tokenize(df, qs) if len(qs) == 0: @@ -455,7 +571,7 @@ def finalize_tsk(tsk): else: new_divisions = [np.min(q), np.max(q)] - from dask.array.percentile import merge_percentiles + # from dask.array.percentile import merge_percentiles name = "quantiles-1-" + token val_dsk = { From 9910eafd4eb1ba0b530f508ec112220f1465a340 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 10:44:52 -0800 Subject: [PATCH 46/61] handle multi-column quantile calculations --- python/dask_cudf/dask_cudf/sorting.py | 243 +++++++++----------------- 1 file changed, 83 insertions(+), 160 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index ef896cd7600..690d141723b 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -387,53 +387,23 @@ def rearrange_by_division_list( return df3 -def _percentile(a, q, interpolation="linear"): +def _quantile(a, q, interpolation="linear"): n = len(a) if not len(a): return None, n - return ( - cupy.asnumpy(gd.DataFrame({"a": a}).quantiles(q.tolist()).values), - n, - ) - + return (a.quantiles(q.tolist()), n) -def merge_percentiles(finalq, qs, vals, interpolation="lower", Ns=None): - """ Combine several percentile calculations of different data. - [NOTE: Mostly copied from dask.array] - Parameters - ---------- - - finalq : numpy.array - Percentiles to compute (must use same scale as ``qs``). - qs : sequence of :class:`numpy.array`s - Percentiles calculated on different sets of data. - vals : sequence of :class:`numpy.array`s - Resulting values associated with percentiles ``qs``. - Ns : sequence of integers - The number of data elements associated with each data set. - interpolation : {'linear', 'lower', 'higher', 'midpoint', 'nearest'} - Specify the type of interpolation to use to calculate final - percentiles. For more information, see :func:`numpy.percentile`. - - Examples - -------- - - >>> finalq = [10, 20, 30, 40, 50, 60, 70, 80] - >>> qs = [[20, 40, 60, 80], [20, 40, 60, 80]] - >>> vals = [np.array([1, 2, 3, 4]), np.array([10, 11, 12, 13])] - >>> Ns = [100, 100] # Both original arrays had 100 elements - - >>> merge_percentiles(finalq, qs, vals, Ns=Ns) - array([ 1, 2, 3, 4, 10, 11, 12, 13]) +def merge_quantiles(finalq, qs, vals): + """ Combine several quantile calculations of different data. + [NOTE: Logic copied from dask.array merge_percentiles] """ if isinstance(finalq, Iterator): finalq = list(finalq) finalq = np.array(finalq) qs = list(map(list, qs)) vals = list(vals) - if Ns is None: - vals, Ns = zip(*vals) + vals, Ns = zip(*vals) Ns = list(Ns) L = list(zip(*[(q, val, N) for q, val, N in zip(qs, vals, Ns) if N])) @@ -441,24 +411,10 @@ def merge_percentiles(finalq, qs, vals, interpolation="lower", Ns=None): raise ValueError("No non-trivial arrays found") qs, vals, Ns = L - # TODO: Perform this check above in percentile once dtype checking is easy - # Here we silently change meaning - if vals[0].dtype.name == "category": - result = merge_percentiles( - finalq, qs, [v.codes for v in vals], interpolation, Ns - ) - import pandas as pd - - return pd.Categorical.from_codes( - result, vals[0].categories, vals[0].ordered - ) - if not np.issubdtype(vals[0].dtype, np.number): - interpolation = "nearest" - if len(vals) != len(qs) or len(Ns) != len(qs): raise ValueError("qs, vals, and Ns parameters must be the same length") - # transform qs and Ns into number of observations between percentiles + # transform qs and Ns into number of observations between quantiles counts = [] for q, N in zip(qs, Ns): count = np.empty(len(q)) @@ -467,57 +423,37 @@ def merge_percentiles(finalq, qs, vals, interpolation="lower", Ns=None): count *= N counts.append(count) - # Sort by calculated percentile values, then number of observations. - # >95% of the time in this function is spent in `merge_sorted` below. - # An alternative that uses numpy sort is shown. It is sometimes - # comparable to, but typically slower than, `merge_sorted`. - # - # >>> A = np.concatenate(map(np.array, map(zip, vals, counts))) - # >>> A.sort(0, kind='mergesort') + def _append_counts(val, count): + val["_counts"] = count + return val - combined_vals_counts = toolz.merge_sorted(*map(zip, vals, counts)) - combined_vals, combined_counts = zip(*combined_vals_counts) - - combined_vals = np.array(combined_vals) - combined_counts = np.array(combined_counts) + # Sort by calculated quantile values, then number of observations. + combined_vals_counts = gd.merge_sorted( + [*map(_append_counts, vals, counts)] + ) + combined_counts = cupy.asnumpy(combined_vals_counts["_counts"].values) + combined_vals = combined_vals_counts.drop(columns=["_counts"]) - # percentile-like, but scaled by total number of observations + # quantile-like, but scaled by total number of observations combined_q = np.cumsum(combined_counts) - # rescale finalq percentiles to match combined_q + # rescale finalq quantiles to match combined_q desired_q = finalq * sum(Ns) - # the behavior of different interpolation methods should be - # investigated further. - if interpolation == "linear": - rv = np.interp(desired_q, combined_q, combined_vals) - else: - left = np.searchsorted(combined_q, desired_q, side="left") - right = np.searchsorted(combined_q, desired_q, side="right") - 1 - np.minimum( - left, len(combined_vals) - 1, left - ) # don't exceed max index - lower = np.minimum(left, right) - upper = np.maximum(left, right) - if interpolation == "lower": - rv = combined_vals[lower] - elif interpolation == "higher": - rv = combined_vals[upper] - elif interpolation == "midpoint": - rv = 0.5 * (combined_vals[lower] + combined_vals[upper]) - elif interpolation == "nearest": - lower_residual = np.abs(combined_q[lower] - desired_q) - upper_residual = np.abs(combined_q[upper] - desired_q) - mask = lower_residual > upper_residual - index = lower # alias; we no longer need lower - index[mask] = upper[mask] - rv = combined_vals[index] - else: - raise ValueError( - "interpolation can only be 'linear', 'lower', " - "'higher', 'midpoint', or 'nearest'" - ) - return rv + # TODO: Support other interpolation methods + # For now - Always use "nearest" for interpolation + left = np.searchsorted(combined_q, desired_q, side="left") + right = np.searchsorted(combined_q, desired_q, side="right") - 1 + np.minimum(left, len(combined_vals) - 1, left) # don't exceed max index + lower = np.minimum(left, right) + upper = np.maximum(left, right) + lower_residual = np.abs(combined_q[lower] - desired_q) + upper_residual = np.abs(combined_q[upper] - desired_q) + mask = lower_residual > upper_residual + index = lower # alias; we no longer need lower + index[mask] = upper[mask] + rv = combined_vals.iloc[index] + return rv.reset_index(drop=True) def quantile(df, q): @@ -534,25 +470,24 @@ def quantile(df, q): q_ndarray.sort(kind="mergesort") q = q_ndarray - assert isinstance(df, Series) + # Lets assume we are dealing with a DataFrame throughout + if isinstance(df, (Series, Index)): + df = df.to_frame() + assert isinstance(df, DataFrame) + final_type = df._meta._constructor - # currently, only Series has quantile method - if isinstance(df, Index): - meta = gd.Series(df._meta_nonempty).quantile(q=q) - else: - meta = df._meta_nonempty.quantile(q=q) - - # Index.quantile(list-like) must be pd.Series, not pd.Index - df_name = df.name + # Create metadata + meta = df._meta_nonempty.quantile(q=q) + # Define final action (create df with quantiles as index) def finalize_tsk(tsk): - return (gd.Series, tsk, q, None, df_name) + return (final_type, tsk, q) - return_type = Series + return_type = DataFrame - # pandas uses quantile in [0, 1] - # numpy / everyone else uses [0, 100] - qs = np.asarray(q) # * 100 + # pandas/cudf uses quantile in [0, 1] + # numpy / cupy uses [0, 100] + qs = np.asarray(q) token = tokenize(df, qs) if len(qs) == 0: @@ -560,8 +495,10 @@ def finalize_tsk(tsk): empty_index = gd.Index([], dtype=float) return Series( { - (name, 0): gd.Series( - [], name=df.name, index=empty_index, dtype="float" + (name, 0): final_type( + {col: [] for col in df.columns}, + name=df.name, + index=empty_index, ) }, name, @@ -571,18 +508,16 @@ def finalize_tsk(tsk): else: new_divisions = [np.min(q), np.max(q)] - # from dask.array.percentile import merge_percentiles - name = "quantiles-1-" + token val_dsk = { - (name, i): (_percentile, (getattr, key, "values"), qs) + (name, i): (_quantile, key, qs) for i, key in enumerate(df.__dask_keys__()) } name2 = "quantiles-2-" + token merge_dsk = { (name2, 0): finalize_tsk( - (merge_percentiles, qs, [qs] * df.npartitions, sorted(val_dsk)) + (merge_quantiles, qs, [qs] * df.npartitions, sorted(val_dsk)) ) } dsk = toolz.merge(val_dsk, merge_dsk) @@ -615,32 +550,12 @@ def sort_values_experimental( elif isinstance(by, tuple): by = list(by) - # Make sure first column is numeric - # (Cannot handle string column here yet) - if divisions is None and isinstance( - df[by[0]]._meta._column, gd.core.column.string.StringColumn - ): - # TODO: Remove when quantile support is added - return df.sort_values( - by, ignore_index=ignore_index, experimental=False - ) - # Step 1 - Pre-sort each partition if sorted_split: df2 = df.map_partitions(M.sort_values, by) else: df2 = df - # Only handle single-column partitioning (for now), - # UNLESS multi-column divisions are provided by user - # TODO: Add multicolumn quantiles logic... - if len(by) > 1: - warnings.warn( - "Using experimental version of sort_values." - " Only `by[0]` will be used for partitioning." - ) - index = by[0] - # Check if we are using explicit comms use_explicit = explicit_comms and explicit_client if use_explicit: @@ -652,48 +567,56 @@ def sort_values_experimental( ): # TODO: Use input divisions for use_explicit==True qn = np.linspace(0.0, 1.0, npartitions + 1).tolist() - dtype = df2[index].dtype - divisions = ( - (quantile(df2[index], qn).compute().values + 1) - .astype(dtype) - .tolist() - ) - divisions[0] = 0 - else: - # For now we can accept multi-column divisions as a dataframe - if isinstance(divisions, gd.DataFrame): - index = by - - # Make sure index is a list - if not isinstance(index, list): - index = [index] + divisions = quantile(df2[by], qn).compute().drop_duplicates() + columns = divisions.columns + # TODO: Make sure divisions are correctly handled for + # non-numerical datatypes.. + if len(columns) == 1 and df2[columns[0]].dtype != "object": + dtype = df2[columns[0]].dtype + divisions = divisions[columns[0]].astype(dtype).values + if dtype in ("int", "float"): + divisions = divisions + 1 + divisions[0] = 0 + divisions = sorted(divisions.tolist()) + else: + for col in columns: + dtype = df2[col].dtype + divisions[col] = divisions[col].astype(dtype) + if dtype in ("int", "float"): + divisions[col] += 1 + divisions[col].iloc[0] = 0 + elif dtype == "object": + divisions[col].iloc[-1] = chr( + ord(divisions[col].iloc[-1][0]) + 1 + ) + divisions[col].iloc[0] = chr(0) # Step 3 - Perform repartitioning shuffle sort_by = None if sorted_split: sort_by = by - if use_explicit and len(index) == 1: - # TODO: Handle len(index) > 1 + if use_explicit and len(by) == 1: + # TODO: Handle len(by) > 1 warnings.warn("Using explicit comms - This is an advanced feature.") df3 = explicit_sorted_shuffle( - df2, index[0], divisions, sort_by, explicit_client + df2, by[0], divisions, sort_by, explicit_client ) - elif sorted_split and len(index) == 1: + elif sorted_split and len(by) == 1: # Need to pass around divisions - # TODO: Handle len(index) > 1 + # TODO: Handle len(by) > 1 df3 = rearrange_by_division_list( - df2, index[0], divisions, max_branch=max_branch, sort_by=sort_by + df2, by[0], divisions, max_branch=max_branch, sort_by=sort_by ) else: # Lets assign a new partitions column # (That is: Use main-line dask shuffle) - # TODO: Handle len(index) > 1 + # TODO: Handle len(by) > 1 meta = df2._meta._constructor_sliced([0]) if not isinstance(divisions, (gd.Series, gd.DataFrame)): - dtype = df2[index[0]].dtype + dtype = df2[by[0]].dtype divisions = df2._meta._constructor_sliced(divisions, dtype=dtype) - partitions = df2[index].map_partitions( + partitions = df2[by].map_partitions( set_partitions_pre, divisions=divisions, meta=meta ) From e12208c962ede78d252437337581b7e3f5a63d9e Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 11:19:44 -0800 Subject: [PATCH 47/61] multi-column division cleanup --- python/dask_cudf/dask_cudf/sorting.py | 28 +++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 690d141723b..99ffc01dc6f 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -567,29 +567,29 @@ def sort_values_experimental( ): # TODO: Use input divisions for use_explicit==True qn = np.linspace(0.0, 1.0, npartitions + 1).tolist() - divisions = quantile(df2[by], qn).compute().drop_duplicates() + divisions = quantile(df2[by], qn).compute() columns = divisions.columns - # TODO: Make sure divisions are correctly handled for - # non-numerical datatypes.. + + # TODO: Make sure divisions are correct for all dtypes.. if len(columns) == 1 and df2[columns[0]].dtype != "object": dtype = df2[columns[0]].dtype - divisions = divisions[columns[0]].astype(dtype).values - if dtype in ("int", "float"): - divisions = divisions + 1 - divisions[0] = 0 - divisions = sorted(divisions.tolist()) + divisions = divisions[columns[0]].astype("int64") + divisions.iloc[-1] += 1 + divisions = sorted( + divisions.drop_duplicates().astype(dtype).values.tolist() + ) else: for col in columns: dtype = df2[col].dtype - divisions[col] = divisions[col].astype(dtype) - if dtype in ("int", "float"): - divisions[col] += 1 - divisions[col].iloc[0] = 0 - elif dtype == "object": + if dtype != "object": + divisions[col] = divisions[col].astype("int64") + 1 + divisions[col].iloc[-1] += 1 + divisions[col] = divisions[col].astype(dtype) + else: divisions[col].iloc[-1] = chr( ord(divisions[col].iloc[-1][0]) + 1 ) - divisions[col].iloc[0] = chr(0) + divisions = divisions.drop_duplicates() # Step 3 - Perform repartitioning shuffle sort_by = None From 2ce04d9f3a1db44ba806b12a5ff91f2b83a74ab8 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 11:37:07 -0800 Subject: [PATCH 48/61] correct multi-col divisions typo --- python/dask_cudf/dask_cudf/sorting.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 99ffc01dc6f..1d0400a37a2 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -582,7 +582,7 @@ def sort_values_experimental( for col in columns: dtype = df2[col].dtype if dtype != "object": - divisions[col] = divisions[col].astype("int64") + 1 + divisions[col] = divisions[col].astype("int64") divisions[col].iloc[-1] += 1 divisions[col] = divisions[col].astype(dtype) else: From 8a7a80883a3e89dc46c811d7fba9128a10616b7d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 15:28:00 -0800 Subject: [PATCH 49/61] merge mutli-column and quantile supprot into dask_cudf --- python/dask_cudf/dask_cudf/core.py | 154 ++--- .../dask_cudf/dask_cudf/explicit_shuffle.py | 183 ------ python/dask_cudf/dask_cudf/sorting.py | 540 ++---------------- python/dask_cudf/dask_cudf/tests/test_core.py | 6 +- python/dask_cudf/dask_cudf/tests/test_sort.py | 35 +- 5 files changed, 109 insertions(+), 809 deletions(-) delete mode 100644 python/dask_cudf/dask_cudf/explicit_shuffle.py diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 2557ce82c4b..265c1161814 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -9,15 +9,12 @@ import dask import dask.dataframe as dd -from dask import compute from dask.base import normalize_token, tokenize from dask.compatibility import apply from dask.context import _globals from dask.core import flatten -from dask.dataframe import from_delayed from dask.dataframe.core import Scalar, handle_out, map_partitions from dask.dataframe.utils import raise_on_meta_error -from dask.delayed import delayed from dask.highlevelgraph import HighLevelGraph from dask.optimization import cull, fuse from dask.utils import M, OperatorMethodMixin, derived_from, funcname @@ -175,32 +172,40 @@ def join(self, other, **kwargs): on = list(on) return super().join(other, how=how, on=on, shuffle="tasks", **kwargs) - def set_index( - self, - other, - experimental=False, - sorted=False, - explicit_client=None, - max_branch=None, - **kwargs, - ): - if experimental and not sorted: - # Use sort values for the shuffle - divisions = divisions = kwargs.get("divisions", None) + def set_index(self, other, sorted=False, divisions=None, **kwargs): + if kwargs.pop("shuffle", "tasks") != "tasks": + raise ValueError( + "Dask-cudf only supports task based shuffling, got %s" + % kwargs["shuffle"] + ) + pre_sorted = sorted + del sorted + + if divisions == "quantile" or isinstance( + divisions, (cudf.DataFrame, cudf.Series) + ): + + # Let upstream-dask handle "pre-sorted" case + if pre_sorted: + return dd.shuffle.set_sorted_index( + self, other, divisions=divisions, **kwargs + ) + by = other if not isinstance(other, list): by = [by] if len(by) > 1: raise ValueError("Dask does not support MultiIndex (yet).") + if divisions == "quantile": + divisions = None + + # Use dask_cudf's sort_values + # TODO: Handle `sorted=True` df = self.sort_values( by, - experimental=experimental, - explicit_client=explicit_client, - max_branch=max_branch, + max_branch=kwargs.get("max_branch", None), divisions=divisions, set_divisions=True, - sorted_split=kwargs.get("sorted_split", False), - upsample=kwargs.get("upsample", 1.0), ) # Ignore divisions if its a dataframe @@ -218,116 +223,39 @@ def set_index( if divisions and df2.npartitions != len(divisions) - 1: return df2.repartition(divisions=divisions) return df2 - if kwargs.pop("shuffle", "tasks") != "tasks": - raise ValueError( - "Dask-cudf only supports task based shuffling, got %s" - % kwargs["shuffle"] - ) + return super().set_index( - other, sorted=sorted, shuffle="tasks", **kwargs + other, + sorted=pre_sorted, + shuffle="tasks", + divisions=divisions, + **kwargs, ) def sort_values( self, by, ignore_index=False, - experimental=False, - explicit_client=None, max_branch=None, - sorted_split=False, - upsample=1.0, + divisions=None, + set_divisions=False, **kwargs, ): - """Sort by the given column - - Parameter - --------- - by : str - """ if self.npartitions == 1: df = self.map_partitions(M.sort_values, by) else: - if experimental: - # Experimental aglorithm (mostly) matches - # the set_index sorting procedure used in Dask. - # Note that, if len(by)>1, only the first column - # is used for repartitioning. All columns are - # used for intra-partition sorting. - df = sorting.sort_values_experimental( - self, - by, - ignore_index=ignore_index, - explicit_client=explicit_client, - max_branch=max_branch, - sorted_split=sorted_split, - upsample=upsample, - divisions=kwargs.get("divisions", None), - set_divisions=kwargs.get("set_divisions", False), - ) - else: - # Legacy sorting algorithm based on "batcher-sortnet" - parts = self.to_delayed() - sorted_parts = sorting.sort_delayed_frame(parts, by) - df = from_delayed(sorted_parts, meta=self._meta) + df = sorting.sort_values( + self, + by, + max_branch=max_branch, + divisions=divisions, + set_divisions=set_divisions, + ) + if ignore_index: return df.reset_index(drop=True) return df - def sort_values_binned(self, by): - """Sorty by the given column and ensure that the same key - doesn't spread across multiple partitions. - """ - # Get sorted partitions - parts = self.sort_values(by=by).to_delayed() - - # Get unique keys in each partition - @delayed - def get_unique(p): - return set(p[by].unique()) - - uniques = list(compute(*map(get_unique, parts))) - - joiner = {} - for i in range(len(uniques)): - joiner[i] = to_join = {} - for j in range(i + 1, len(uniques)): - intersect = uniques[i] & uniques[j] - # If the keys intersect - if intersect: - # Remove keys - uniques[j] -= intersect - to_join[j] = frozenset(intersect) - else: - break - - @delayed - def join(df, other, keys): - others = [ - other.query("{by}==@k".format(by=by)) for k in sorted(keys) - ] - return cudf.concat([df] + others) - - @delayed - def drop(df, keep_keys): - locvars = locals() - for i, k in enumerate(keep_keys): - locvars["k{}".format(i)] = k - - conds = [ - "{by}==@k{i}".format(by=by, i=i) for i in range(len(keep_keys)) - ] - expr = " or ".join(conds) - return df.query(expr) - - for i in range(len(parts)): - if uniques[i]: - parts[i] = drop(parts[i], uniques[i]) - for joinee, intersect in joiner[i].items(): - parts[i] = join(parts[i], parts[joinee], intersect) - - results = [p for i, p in enumerate(parts) if uniques[i]] - return from_delayed(results, meta=self._meta).reset_index() - def to_parquet(self, path, *args, **kwargs): """ Calls dask.dataframe.io.to_parquet with CudfEngine backend """ from dask_cudf.io import to_parquet diff --git a/python/dask_cudf/dask_cudf/explicit_shuffle.py b/python/dask_cudf/dask_cudf/explicit_shuffle.py deleted file mode 100644 index 7620c55c8a4..00000000000 --- a/python/dask_cudf/dask_cudf/explicit_shuffle.py +++ /dev/null @@ -1,183 +0,0 @@ -import asyncio -import warnings - -import cupy -import pandas as pd - -import distributed -from dask.distributed import wait -from dask_cuda.explicit_comms import comms -from distributed.protocol import to_serialize - -import cudf -import rmm - -cupy.cuda.set_allocator(rmm.rmm_cupy_allocator) - - -def _cleanup_parts(df_parts): - for part in df_parts: - if part: - del part - - -async def send_df(ep, df): - if df is None: - return await ep.write("empty") - else: - return await ep.write([to_serialize(df)]) - - -async def recv_df(ep): - ret = await ep.read() - if ret == "empty": - return None - else: - return ret[0] - - -async def send_parts(eps, parts): - futures = [] - for rank, ep in eps.items(): - futures.append(send_df(ep, parts[rank])) - await asyncio.gather(*futures) - - -async def recv_parts(eps, parts): - futures = [] - for ep in eps.values(): - futures.append(recv_df(ep)) - parts.extend(await asyncio.gather(*futures)) - - -async def exchange_and_concat_parts(rank, eps, parts, sort_by): - ret = [parts[rank]] - del parts[rank] - parts[rank] = None - await asyncio.gather(recv_parts(eps, ret), send_parts(eps, parts)) - for rank in list(parts): - del parts[rank] - new_df = concat( - [df.copy(deep=False) for df in ret if df is not None and len(df)], - sort_by=sort_by, - ) - del ret - return new_df - - -def concat(df_list, sort_by=None): - if len(df_list) == 0: - return None - if isinstance(df_list[0], cudf.DataFrame): - if sort_by: - new_df = cudf.merge_sorted(df_list, keys=sort_by) - else: - new_df = cudf.concat(df_list) - else: - new_df = pd.concat(df_list) - _cleanup_parts(df_list) - return new_df - - -def partition_table(df, partitions, n_chunks, sort_by=None): - if df is None: - result = [None] * n_chunks - elif sort_by: - result = { - i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) - for i in range(0, len(partitions) - 1) - } - else: - result = dict( - zip( - range(n_chunks), - df.scatter_by_map(partitions, map_size=n_chunks), - ) - ) - return result - - -async def distributed_shuffle( - n_chunks, rank, eps, table, partitions, index, sort_by -): - parts = partition_table(table, partitions, n_chunks, sort_by=sort_by) - return await exchange_and_concat_parts(rank, eps, parts, sort_by) - - -async def _explicit_shuffle( - s, df_nparts, df_parts, index, sort_by, divisions, to_cpu -): - if len(df_parts[0]) == 0: - df = None - else: - df = df_parts[0][0].copy(deep=False) - - # Calculate new partition mapping - if df is not None: - divisions = df._constructor_sliced(divisions, dtype=df[index].dtype) - if sort_by: - splits = df[index].searchsorted(divisions, side="left") - splits[-1] = len(df[index]) - partitions = splits.tolist() - del splits - else: - partitions = divisions.searchsorted(df[index], side="right") - 1 - partitions[(df[index] >= divisions.iloc[-1]).values] = ( - len(divisions) - 2 - ) - del divisions - else: - partitions = None - - # Run distributed shuffle and set_index algorithm - new_df = await distributed_shuffle( - s["nworkers"], s["rank"], s["eps"], df, partitions, index, sort_by - ) - - if to_cpu: - return cudf.from_pandas(new_df) - return new_df - - -async def _explicit_aggregate(s, df_nparts, df_parts, sort_by, to_cpu): - def df_concat(df_parts, sort_by=None): - """Making sure df_parts is a single dataframe or None""" - if len(df_parts) == 0: - return None - elif len(df_parts) == 1: - return df_parts[0] - else: - return concat(df_parts, sort_by=sort_by) - - # Concatenate all parts owned by this worker into - # a single cudf DataFrame - if to_cpu: - return df_concat([dfp.to_pandas() for dfp in df_parts[0]]) - else: - return df_concat(df_parts[0], sort_by=sort_by) - - -def explicit_sorted_shuffle(df, index, divisions, sort_by, client, **kwargs): - client.rebalance(futures=distributed.futures_of(df)) - to_cpu = kwargs.get("to_cpu", False) - if to_cpu: - warnings.warn("Using CPU for shuffling. Performance will suffer!") - - # Explict-comms Partition Aggregation - df2 = comms.default_comms().dataframe_operation( - _explicit_aggregate, df_list=(df,), extra_args=(sort_by, to_cpu) - ) - wait(df2.persist()) - wait(client.cancel(df)) - del df - - # Explict-comms shuffle - df3 = comms.default_comms().dataframe_operation( - _explicit_shuffle, - df_list=(df2,), - extra_args=(index, sort_by, divisions, to_cpu), - ) - wait(df3.persist()) - wait(client.cancel(df2)) - del df2 - return df3 diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 1d0400a37a2..beab2c1a35c 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -1,168 +1,20 @@ # Copyright (c) 2020, NVIDIA CORPORATION. -import math -import warnings from collections.abc import Iterator -from operator import getitem import cupy import numpy as np import toolz -from dask import compute, delayed from dask.base import tokenize -from dask.dataframe.core import DataFrame, Index, Series, _concat -from dask.dataframe.shuffle import rearrange_by_column, shuffle_group_get -from dask.dataframe.utils import group_split_dispatch +from dask.dataframe.core import DataFrame, Index, Series +from dask.dataframe.shuffle import rearrange_by_column from dask.highlevelgraph import HighLevelGraph -from dask.utils import M, digit, insert +from dask.utils import M import cudf as gd -try: - from .explicit_shuffle import explicit_sorted_shuffle - explicit_comms = True -except ImportError: - explicit_comms = False - - -""" -Batcher's Odd-even sorting network -Adapted from https://en.wikipedia.org/wiki/Batcher_odd%E2%80%93even_mergesort -""" - - -def get_oversized(length): - """ - The oddeven network requires a power-of-2 length. - This method computes the next power-of-2 from the *length* if - *length* is not a power-of-2 value. - """ - return 2 ** math.ceil(math.log2(length)) - - -def is_power_of_2(length): - return math.log2(length).is_integer() - - -def oddeven_merge(lo, hi, r): - step = r * 2 - if step < hi - lo: - for each in oddeven_merge(lo, hi, step): - yield each - for each in oddeven_merge(lo + r, hi, step): - yield each - for i in range(lo + r, hi - r, step): - yield (i, i + r) - else: - yield (lo, lo + r) - - -def oddeven_merge_sort_range(lo, hi): - """ sort the part of x with indices between lo and hi. - - Note: endpoints (lo and hi) are included. - """ - if (hi - lo) >= 1: - # if there is more than one element, split the input - # down the middle and first sort the first and second - # half, followed by merging them. - mid = lo + ((hi - lo) // 2) - for each in oddeven_merge_sort_range(lo, mid): - yield each - for each in oddeven_merge_sort_range(mid + 1, hi): - yield each - for each in oddeven_merge(lo, hi, 1): - yield each - - -def oddeven_merge_sort(length): - """ "length" is the length of the list to be sorted. - Returns a list of pairs of indices starting with 0 """ - assert is_power_of_2(length) - for each in oddeven_merge_sort_range(0, length - 1): - yield each - - -def _pad_data_to_length(parts): - parts = list(parts) - needed = get_oversized(len(parts)) - padn = needed - len(parts) - return parts + [None] * padn, len(parts) - - -def _compare_frame(a, b, max_part_size, by): - if a is not None and b is not None: - joint = gd.concat([a, b]) - sorten = joint.sort_values(by=by) - # Split the sorted frame using the *max_part_size* - lhs, rhs = sorten[:max_part_size], sorten[max_part_size:] - # Replace empty frame with None - return lhs or None, rhs or None - elif a is None and b is None: - return None, None - elif a is None: - return b.sort_values(by=by), None - else: - return a.sort_values(by=by), None - - -def _compare_and_swap_frame(parts, a, b, max_part_size, by): - compared = delayed(_compare_frame)( - parts[a], parts[b], max_part_size, by=by - ) - parts[a] = compared[0] - parts[b] = compared[1] - - -def _cleanup(df): - if "__dask_cudf__valid" in df.columns: - out = df.query("__dask_cudf__valid") - del out["__dask_cudf__valid"] - else: - out = df - return out - - -def sort_delayed_frame(parts, by): - """ - Parameters - ---------- - parts : - Delayed partitions of cudf.DataFrame - by : str - Column name by which to sort - - The sort will also rebalance the partition sizes so that all output - partitions has partition size of atmost `max(original_partition_sizes)`. - Therefore, they may be fewer partitions in the output. - """ - # Empty frame? - if len(parts) == 0: - return parts - # Compute maximum paritition size, which is needed - # for non-uniform partition size - max_part_size = delayed(max)(*map(delayed(len), parts)) - # Add empty partitions to match power-of-2 requirement. - parts, valid = _pad_data_to_length(parts) - # More than 1 input? - if len(parts) > 1: - # Build batcher's odd-even sorting network - for a, b in oddeven_merge_sort(len(parts)): - _compare_and_swap_frame(parts, a, b, max_part_size, by=by) - # Single input? - else: - parts = [delayed(lambda x: x.sort_values(by=by))(parts[0])] - # Count number of non-empty partitions - valid_ct = delayed(sum)( - list(map(delayed(lambda x: int(x is not None)), parts[:valid])) - ) - valid = compute(valid_ct)[0] - validparts = parts[:valid] - return validparts - - -def set_partitions_pre(s, divisions): +def _set_partitions_pre(s, divisions): partitions = divisions.searchsorted(s, side="right") - 1 # Use searchsorted to avoid string-compare limitations @@ -173,230 +25,16 @@ def set_partitions_pre(s, divisions): return partitions -def sorted_split_divs(df, divisions, col, stage, k, npartitions, sort_by): - # Get partitions - dtype = df[col].dtype - splits = df[col].searchsorted( - df._constructor_sliced(divisions, dtype=dtype), side="left" - ) - splits[-1] = len(df[col]) - partitions = splits.tolist() - - # Create splits - split_dict = { - i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) - for i in range(len(divisions) - 1) - } - - if k < npartitions: - # Rearrange the splits (for now -- Need NEW algorithm to avoid this) - # Note that we REALLY don't want to do this if we dont need to!! - agg_dict = {i: [] for i in range(k)} - for c in [int(k) for k in split_dict.keys()]: - c_new = np.mod(np.floor_divide(c, k ** stage), k) - if split_dict[c] is not None and len(split_dict[c]): - agg_dict[c_new].append(split_dict[c].copy(deep=False)) - split_dict = {} - for i in range(k): - if len(agg_dict[i]): - split_dict[i] = gd.merge_sorted(agg_dict[i], keys=sort_by) - else: - split_dict[i] = df.iloc[:0] - return split_dict - - -def sorted_split_divs_2(df, divisions, col): - if not len(df): - return {}, df - - # Get partitions - dtype = df[col].dtype - splits = df[col].searchsorted( - df._constructor_sliced(divisions, dtype=dtype), side="left" - ) - splits[-1] = len(df[col]) - partitions = splits.tolist() - - # Create splits - result2 = { - i: df.iloc[partitions[i] : partitions[i + 1]].copy(deep=False) - for i in range(len(divisions) - 1) - if partitions[i] != partitions[i + 1] - } - return result2, df.iloc[:0] - - -def shuffle_group_divs(df, divisions, col, stage, k, npartitions, inp): - dtype = df[col].dtype - c = set_partitions_pre( - df[col], divisions=df._constructor_sliced(divisions, dtype=dtype) - ) - typ = np.min_scalar_type(npartitions * 2) - c = np.mod(c, npartitions).astype(typ, copy=False) - np.floor_divide(c, k ** stage, out=c) - np.mod(c, k, out=c) - return dict( - zip(range(k), df.scatter_by_map(c.astype(np.int32), map_size=k)) - ) - - -def shuffle_group_divs_2(df, divisions, col): - if not len(df): - return {}, df - ind = set_partitions_pre( - df[col], divisions=df._constructor_sliced(divisions) - ).astype(np.int32) - result2 = group_split_dispatch(df, ind.view(np.int32), len(divisions) - 1) - return result2, df.iloc[:0] - - -def _concat_wrapper(df_list, sort_by): - if sort_by: - return gd.merge_sorted(df_list, keys=sort_by) - else: - df = _concat(df_list) - if sort_by: - return df.sort_values(sort_by) - return df - - -def rearrange_by_division_list( - df, column: str, divisions: list, max_branch=None, sort_by=None -): - npartitions = len(divisions) - 1 - n = df.npartitions - max_branch = max_branch or 32 - stages = int(math.ceil(math.log(n) / math.log(max_branch))) - - if stages > 1: - k = int(math.ceil(n ** (1 / stages))) - else: - k = n - - groups = [] - splits = [] - joins = [] - - inputs = [ - tuple(digit(i, j, k) for j in range(stages)) - for i in range(k ** stages) - ] - - token = tokenize(df, column, max_branch) - - start = { - ("shuffle-join-" + token, 0, inp): (df._name, i) - if i < df.npartitions - else df._meta - for i, inp in enumerate(inputs) - } - - if sort_by: - _split_func_1 = sorted_split_divs - _split_func_2 = sorted_split_divs_2 - else: - _split_func_1 = shuffle_group_divs - _split_func_2 = shuffle_group_divs_2 - - for stage in range(1, stages + 1): - group = { # Convert partition into dict of dataframe pieces - ("shuffle-group-divs-" + token, stage, inp): ( - _split_func_1, - ("shuffle-join-" + token, stage - 1, inp), - divisions, - column, - stage - 1, - k, - n, - sort_by, # Need this to rearrange splits (for now) - ) - for inp in inputs - } - - split = { # Get out each individual dataframe piece from the dicts - ("shuffle-split-" + token, stage, i, inp): ( - getitem, - ("shuffle-group-divs-" + token, stage, inp), - i, - ) - for i in range(k) - for inp in inputs - } - - join = { # concatenate those pieces together, with their friends - ("shuffle-join-" + token, stage, inp): ( - _concat_wrapper, - [ - ( - "shuffle-split-" + token, - stage, - inp[stage - 1], - insert(inp, stage - 1, j), - ) - for j in range(k) - ], - sort_by, - ) - for inp in inputs - } - groups.append(group) - splits.append(split) - joins.append(join) - - end = { - ("shuffle-" + token, i): ("shuffle-join-" + token, stages, inp) - for i, inp in enumerate(inputs) - } - - dsk = toolz.merge(start, end, *(groups + splits + joins)) - graph = HighLevelGraph.from_collections( - "shuffle-" + token, dsk, dependencies=[df] - ) - df2 = DataFrame(graph, "shuffle-" + token, df, df.divisions) - - if npartitions != df.npartitions: - parts = [i % df.npartitions for i in range(npartitions)] - token = tokenize(df2, npartitions) - - dsk = { - ("repartition-group-" + token, i): ( - _split_func_2, - k, - divisions, - column, - ) - for i, k in enumerate(df2.__dask_keys__()) - } - for p in range(npartitions): - dsk[("repartition-get-" + token, p)] = ( - shuffle_group_get, - ("repartition-group-" + token, parts[p]), - p, - ) - - graph2 = HighLevelGraph.from_collections( - "repartition-get-" + token, dsk, dependencies=[df2] - ) - df3 = DataFrame( - graph2, "repartition-get-" + token, df2, [None] * (npartitions + 1) - ) - else: - df3 = df2 - df3.divisions = (None,) * (df.npartitions + 1) - - return df3 - - -def _quantile(a, q, interpolation="linear"): +def _quantile(a, q): n = len(a) if not len(a): return None, n - return (a.quantiles(q.tolist()), n) + return (a.quantiles(q.tolist(), interpolation="nearest"), n) def merge_quantiles(finalq, qs, vals): """ Combine several quantile calculations of different data. - [NOTE: Logic copied from dask.array merge_percentiles] + [NOTE: Same logic as dask.array merge_percentiles] """ if isinstance(finalq, Iterator): finalq = list(finalq) @@ -456,12 +94,9 @@ def _append_counts(val, count): return rv.reset_index(drop=True) -def quantile(df, q): - """Approximate quantiles of Series. - Parameters - ---------- - q : list/array of floats - Iterable of numbers ranging from 0 to 100 for the desired quantiles +def _approximate_quantile(df, q): + """Approximate quantiles of DataFrame or Series. + [NOTE: Same logic as dask.dataframe Series quantile] """ # current implementation needs q to be sorted so # sort if array-like, otherwise leave it alone @@ -525,24 +160,36 @@ def finalize_tsk(tsk): return return_type(graph, name2, meta, new_divisions) -def sort_values_experimental( - df, - by, - ignore_index=False, - explicit_client=None, - max_branch=None, - divisions=None, - sorted_split=False, - upsample=1.0, - set_divisions=False, -): - """ Experimental sort_values implementation. - - Sort by the given column name or list/tuple of column names. - - Parameter - --------- - by : list, tuple, str +def quantile_divisions(df, by, npartitions): + qn = np.linspace(0.0, 1.0, npartitions + 1).tolist() + divisions = _approximate_quantile(df[by], qn).compute() + columns = divisions.columns + + # TODO: Make sure divisions are correct for all dtypes.. + if len(columns) == 1 and df[columns[0]].dtype != "object": + dtype = df[columns[0]].dtype + divisions = divisions[columns[0]].astype("int64") + divisions.iloc[-1] += 1 + divisions = sorted( + divisions.drop_duplicates().astype(dtype).values.tolist() + ) + else: + for col in columns: + dtype = df[col].dtype + if dtype != "object": + divisions[col] = divisions[col].astype("int64") + divisions[col].iloc[-1] += 1 + divisions[col] = divisions[col].astype(dtype) + else: + divisions[col].iloc[-1] = chr( + ord(divisions[col].iloc[-1][0]) + 1 + ) + divisions = divisions.drop_duplicates() + return divisions + + +def sort_values(df, by, max_branch=None, divisions=None, set_divisions=False): + """ Sort by the given list/tuple of column names. """ npartitions = df.npartitions if isinstance(by, str): @@ -550,93 +197,32 @@ def sort_values_experimental( elif isinstance(by, tuple): by = list(by) - # Step 1 - Pre-sort each partition - if sorted_split: - df2 = df.map_partitions(M.sort_values, by) - else: - df2 = df - - # Check if we are using explicit comms - use_explicit = explicit_comms and explicit_client - if use_explicit: - npartitions = len(explicit_client.cluster.workers) - - # Step 2 - Calculate new divisions (if necessary) - if divisions is None or ( - use_explicit and len(divisions) != npartitions + 1 - ): - # TODO: Use input divisions for use_explicit==True - qn = np.linspace(0.0, 1.0, npartitions + 1).tolist() - divisions = quantile(df2[by], qn).compute() - columns = divisions.columns - - # TODO: Make sure divisions are correct for all dtypes.. - if len(columns) == 1 and df2[columns[0]].dtype != "object": - dtype = df2[columns[0]].dtype - divisions = divisions[columns[0]].astype("int64") - divisions.iloc[-1] += 1 - divisions = sorted( - divisions.drop_duplicates().astype(dtype).values.tolist() - ) - else: - for col in columns: - dtype = df2[col].dtype - if dtype != "object": - divisions[col] = divisions[col].astype("int64") - divisions[col].iloc[-1] += 1 - divisions[col] = divisions[col].astype(dtype) - else: - divisions[col].iloc[-1] = chr( - ord(divisions[col].iloc[-1][0]) + 1 - ) - divisions = divisions.drop_duplicates() - - # Step 3 - Perform repartitioning shuffle - sort_by = None - if sorted_split: - sort_by = by - if use_explicit and len(by) == 1: - # TODO: Handle len(by) > 1 - warnings.warn("Using explicit comms - This is an advanced feature.") - df3 = explicit_sorted_shuffle( - df2, by[0], divisions, sort_by, explicit_client - ) - elif sorted_split and len(by) == 1: - # Need to pass around divisions - # TODO: Handle len(by) > 1 - df3 = rearrange_by_division_list( - df2, by[0], divisions, max_branch=max_branch, sort_by=sort_by - ) - else: - # Lets assign a new partitions column - # (That is: Use main-line dask shuffle) - # TODO: Handle len(by) > 1 - meta = df2._meta._constructor_sliced([0]) - if not isinstance(divisions, (gd.Series, gd.DataFrame)): - dtype = df2[by[0]].dtype - divisions = df2._meta._constructor_sliced(divisions, dtype=dtype) - - partitions = df2[by].map_partitions( - set_partitions_pre, divisions=divisions, meta=meta - ) + # Step 1 - Calculate new divisions (if necessary) + if divisions is None: + divisions = quantile_divisions(df, by, npartitions) - df2b = df2.assign(_partitions=partitions) - df3 = rearrange_by_column( - df2b, - "_partitions", - max_branch=max_branch, - npartitions=len(divisions) - 1, - shuffle="tasks", - ).drop(columns=["_partitions"]) - df3.divisions = (None,) * (df3.npartitions + 1) + # Step 2 - Perform repartitioning shuffle + meta = df._meta._constructor_sliced([0]) + if not isinstance(divisions, (gd.Series, gd.DataFrame)): + dtype = df[by[0]].dtype + divisions = df._meta._constructor_sliced(divisions, dtype=dtype) - # Step 4 - Return final sorted df - if sorted_split: - # Data should already be sorted - df4 = df3 - else: - df4 = df3.map_partitions(M.sort_values, by) + partitions = df[by].map_partitions( + _set_partitions_pre, divisions=divisions, meta=meta + ) + + df2 = df.assign(_partitions=partitions) + df3 = rearrange_by_column( + df2, + "_partitions", + max_branch=max_branch, + npartitions=len(divisions) - 1, + shuffle="tasks", + ).drop(columns=["_partitions"]) + df3.divisions = (None,) * (df3.npartitions + 1) + # Step 3 - Return final sorted df + df4 = df3.map_partitions(M.sort_values, by) if not isinstance(divisions, gd.DataFrame) and set_divisions: # Can't have multi-column divisions elsewhere in dask (yet) df4.divisions = tuple(divisions) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index 91a9a11b49e..16937a0282b 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -122,8 +122,8 @@ def test_from_dask_dataframe(): @pytest.mark.parametrize("nelem", [10, 200, 1333]) -@pytest.mark.parametrize("experimental", [True, False]) -def test_set_index(nelem, experimental): +@pytest.mark.parametrize("divisions", [None, "quantile"]) +def test_set_index(nelem, divisions): with dask.config.set(scheduler="single-threaded"): np.random.seed(0) # Use unique index range as the sort may not be stable-ordering @@ -136,7 +136,7 @@ def test_set_index(nelem, experimental): dgdf = ddf.map_partitions(cudf.from_pandas) expect = ddf.set_index("x") - got = dgdf.set_index("x", experimental=experimental) + got = dgdf.set_index("x", divisions=divisions) dd.assert_eq(expect, got, check_index=False, check_divisions=False) diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index 8235bb905e9..a4600999086 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -10,49 +10,18 @@ @pytest.mark.parametrize("by", ["a", "b", ["a", "b"]]) @pytest.mark.parametrize("nelem", [10, 100, 1000]) @pytest.mark.parametrize("nparts", [1, 2, 5, 10]) -@pytest.mark.parametrize("experimental", [True, False]) -def test_sort_values(nelem, nparts, by, experimental): +def test_sort_values(nelem, nparts, by): df = cudf.DataFrame() df["a"] = np.ascontiguousarray(np.arange(nelem)[::-1]) df["b"] = np.arange(100, nelem + 100) ddf = dd.from_pandas(df, npartitions=nparts) with dask.config.set(scheduler="single-threaded"): - got = ddf.sort_values(by=by, experimental=experimental) + got = ddf.sort_values(by=by) expect = df.sort_values(by=by) dd.assert_eq(got, expect, check_index=False) -def test_sort_values_binned(): - np.random.seed(43) - nelem = 100 - nparts = 5 - by = "a" - df = cudf.DataFrame() - df["a"] = np.random.randint(1, 5, nelem) - ddf = dd.from_pandas(df, npartitions=nparts) - - parts = ddf.sort_values_binned(by=by).to_delayed() - part_uniques = [] - for i, p in enumerate(parts): - part = dask.compute(p)[0] - part_uniques.append(set(part.a.unique())) - - # Partitions do not have intersecting keys - for i in range(len(part_uniques)): - for j in range(i + 1, len(part_uniques)): - assert not ( - part_uniques[i] & part_uniques[j] - ), "should have empty intersection" - - -def test_sort_binned_meta(): - df = cudf.DataFrame({"a": [0, 1, 2, 3, 4], "b": [5, 6, 7, 7, 8]}) - ddf = dd.from_pandas(df, npartitions=2).persist() - - ddf.sort_values_binned(by="b") - - @pytest.mark.parametrize("by", ["a", "b", ["a", "b"]]) def test_sort_values_single_partition(by): df = cudf.DataFrame() From 0486be2988a62b4bd3812d341e0fbbe23032b1b3 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 2 Mar 2020 15:33:22 -0800 Subject: [PATCH 50/61] merge mutli-column and quantile supprot into dask_cudf --- python/dask_cudf/dask_cudf/__init__.py | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/python/dask_cudf/dask_cudf/__init__.py b/python/dask_cudf/dask_cudf/__init__.py index 346dfdae0bb..372665e3f91 100644 --- a/python/dask_cudf/dask_cudf/__init__.py +++ b/python/dask_cudf/dask_cudf/__init__.py @@ -1,15 +1,10 @@ +from dask.dataframe import from_delayed + import cudf from cudf._version import get_versions from . import backends -from .core import ( - DataFrame, - Series, - concat, - from_cudf, - from_dask_dataframe, - from_delayed, -) +from .core import DataFrame, Series, concat, from_cudf, from_dask_dataframe from .io import read_csv, read_json, read_orc, to_orc try: From cda222fdd7c4541cfdde584e58e825482b51eaee Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 4 Mar 2020 15:26:12 -0800 Subject: [PATCH 51/61] remove test_batcher_sortnet.py --- .../dask_cudf/tests/test_batcher_sortnet.py | 61 ------------------- 1 file changed, 61 deletions(-) delete mode 100644 python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py diff --git a/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py b/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py deleted file mode 100644 index 7cbd06084ea..00000000000 --- a/python/dask_cudf/dask_cudf/tests/test_batcher_sortnet.py +++ /dev/null @@ -1,61 +0,0 @@ -import numpy as np -import pytest - -import cudf - -from dask_cudf import sorting - - -@pytest.mark.parametrize("n", list(range(1, 40))) -def test_padding(n): - data = list(range(n)) - padded, valid = sorting._pad_data_to_length(data) - assert len(data) == valid - assert sorting.is_power_of_2(len(padded)) - assert valid > len(padded) / 2 - assert all(x is not None for x in padded[:valid]) - assert all(x is None for x in padded[valid:]) - - -@pytest.mark.parametrize("seed", [43, 120]) -@pytest.mark.parametrize("nelem", [2, 10, 100]) -def test_compare_frame(seed, nelem): - np.random.seed(seed) - max_part_size = nelem - # Make LHS - lhs = cudf.DataFrame() - lhs["a"] = lhs_a = np.random.random(nelem) - lhs["b"] = lhs_b = np.random.random(nelem) - - # Make RHS - rhs = cudf.DataFrame() - rhs["a"] = rhs_a = np.random.random(nelem) - rhs["b"] = rhs_b = np.random.random(nelem) - - # Sort by column "a" - got_a = sorting._compare_frame(lhs, rhs, max_part_size, by="a") - # Check - expect_a = np.hstack([lhs_a, rhs_a]) - expect_a.sort() - np.testing.assert_array_equal(got_a[0].a.to_array(), expect_a[:nelem]) - np.testing.assert_array_equal(got_a[1].a.to_array(), expect_a[nelem:]) - - # Sort by column "b" - got_b = sorting._compare_frame(lhs, rhs, max_part_size, by="b") - # Check - expect_b = np.hstack([lhs_b, rhs_b]) - expect_b.sort() - np.testing.assert_array_equal(got_b[0].b.to_array(), expect_b[:nelem]) - np.testing.assert_array_equal(got_b[1].b.to_array(), expect_b[nelem:]) - - -def test_compare_frame_with_none(): - df = cudf.DataFrame() - max_part_size = 1 - df["a"] = [0] - res = sorting._compare_frame(df, None, max_part_size, by="a") - assert res[0] is not None, res[1] is None - res = sorting._compare_frame(None, df, max_part_size, by="a") - assert res[0] is not None, res[1] is None - res = sorting._compare_frame(None, None, max_part_size, by="a") - assert res == (None, None) From 849304faa90648d804f0427cbb13c741c3dfde62 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 5 Mar 2020 06:29:06 -0800 Subject: [PATCH 52/61] use int32 instead of int64 when using group_split_dispatch --- python/dask_cudf/dask_cudf/sorting.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index b556dd1cb5c..2805a15e7fb 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -28,7 +28,7 @@ def _shuffle_group_2(df, columns, ignore_index): ind = hash_object_dispatch(df[columns], index=False) n = ind.max() + 1 result2 = group_split_dispatch( - df, ind.values.view(np.int64), n, ignore_index=ignore_index + df, ind.values.view(), n, ignore_index=ignore_index ) return result2, df.iloc[:0] @@ -48,7 +48,7 @@ def _shuffle_group(df, columns, stage, k, npartitions, ignore_index): np.floor_divide(c, k ** stage, out=c) np.mod(c, k, out=c) return group_split_dispatch( - df, c.astype(np.int64), k, ignore_index=ignore_index + df, c.astype(np.int32), k, ignore_index=ignore_index ) From 90c316cb69ffa05fcaaf9d3821432675e9a07115 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 5 Mar 2020 07:18:49 -0800 Subject: [PATCH 53/61] bugfixes related to drop= and ignore_index= settings in set_index and sort_values, respectively --- python/dask_cudf/dask_cudf/core.py | 9 ++++++++- python/dask_cudf/dask_cudf/sorting.py | 17 +++++++++++++++-- 2 files changed, 23 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index 0d32da47a7b..bae95cca8ee 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -206,6 +206,7 @@ def set_index(self, other, sorted=False, divisions=None, **kwargs): max_branch=kwargs.get("max_branch", None), divisions=divisions, set_divisions=True, + ignore_index=True, ) # Ignore divisions if its a dataframe @@ -213,9 +214,14 @@ def set_index(self, other, sorted=False, divisions=None, **kwargs): divisions = None # Set index and repartition + df2 = df.map_partitions( + sorting.set_index_post, + index_name=other, + drop=kwargs.get("drop", True), + column_dtype=df.columns.dtype, + ) npartitions = kwargs.get("npartitions", self.npartitions) partition_size = kwargs.get("partition_size", None) - df2 = df.map_partitions(M.set_index, other) if partition_size: return df2.repartition(partition_size=partition_size) if not divisions and df2.npartitions != npartitions: @@ -250,6 +256,7 @@ def sort_values( max_branch=max_branch, divisions=divisions, set_divisions=set_divisions, + ignore_index=ignore_index, ) if ignore_index: diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 2805a15e7fb..5cc72aa8636 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -178,6 +178,12 @@ def rearrange_by_hash( return df3 +def set_index_post(df, index_name, drop, column_dtype): + df2 = df.set_index(index_name, drop=drop) + df2.columns = df2.columns.astype(column_dtype) + return df2 + + def _set_partitions_pre(s, divisions): partitions = divisions.searchsorted(s, side="right") - 1 @@ -352,7 +358,14 @@ def quantile_divisions(df, by, npartitions): return divisions -def sort_values(df, by, max_branch=None, divisions=None, set_divisions=False): +def sort_values( + df, + by, + max_branch=None, + divisions=None, + set_divisions=False, + ignore_index=False, +): """ Sort by the given list/tuple of column names. """ npartitions = df.npartitions @@ -382,7 +395,7 @@ def sort_values(df, by, max_branch=None, divisions=None, set_divisions=False): max_branch=max_branch, npartitions=len(divisions) - 1, shuffle="tasks", - ignore_index=True, + ignore_index=ignore_index, ).drop(columns=["_partitions"]) df3.divisions = (None,) * (df3.npartitions + 1) From e53857719db033d0f6fd1ed48f3151ab2fc6ef69 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 5 Mar 2020 08:02:22 -0800 Subject: [PATCH 54/61] add test coverage --- python/dask_cudf/dask_cudf/sorting.py | 7 ++++++- python/dask_cudf/dask_cudf/tests/test_sort.py | 8 +++++--- 2 files changed, 11 insertions(+), 4 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index 5cc72aa8636..6208f8a7029 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -15,6 +15,7 @@ from dask.utils import M, digit, insert import cudf as gd +from cudf.utils.dtypes import is_categorical_dtype def set_partitions_hash(df, columns, npartitions): @@ -336,7 +337,11 @@ def quantile_divisions(df, by, npartitions): columns = divisions.columns # TODO: Make sure divisions are correct for all dtypes.. - if len(columns) == 1 and df[columns[0]].dtype != "object": + if ( + len(columns) == 1 + and df[columns[0]].dtype != "object" + and not is_categorical_dtype(df[columns[0]].dtype) + ): dtype = df[columns[0]].dtype divisions = divisions[columns[0]].astype("int64") divisions.iloc[-1] += 1 diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py index a4600999086..d69ed7d9b42 100644 --- a/python/dask_cudf/dask_cudf/tests/test_sort.py +++ b/python/dask_cudf/dask_cudf/tests/test_sort.py @@ -7,13 +7,15 @@ import cudf -@pytest.mark.parametrize("by", ["a", "b", ["a", "b"]]) -@pytest.mark.parametrize("nelem", [10, 100, 1000]) -@pytest.mark.parametrize("nparts", [1, 2, 5, 10]) +@pytest.mark.parametrize("by", ["a", "b", "c", "d", ["a", "b"], ["c", "d"]]) +@pytest.mark.parametrize("nelem", [10, 500]) +@pytest.mark.parametrize("nparts", [1, 10]) def test_sort_values(nelem, nparts, by): df = cudf.DataFrame() df["a"] = np.ascontiguousarray(np.arange(nelem)[::-1]) df["b"] = np.arange(100, nelem + 100) + df["c"] = np.random.choice(cudf.datasets.names, size=nelem) + df["d"] = df["c"].astype("category") ddf = dd.from_pandas(df, npartitions=nparts) with dask.config.set(scheduler="single-threaded"): From 803aecae4cbe809c21921f37907d8d5882d4e7d9 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Thu, 5 Mar 2020 08:47:26 -0800 Subject: [PATCH 55/61] add set_index coverage for string column --- python/dask_cudf/dask_cudf/tests/test_core.py | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index bf756d6bb3d..9db9ce7f899 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -141,6 +141,21 @@ def test_set_index(nelem, divisions): dd.assert_eq(expect, got, check_index=False, check_divisions=False) +@pytest.mark.parametrize("by", ["a", "b"]) +@pytest.mark.parametrize("nelem", [10, 500]) +@pytest.mark.parametrize("nparts", [1, 10]) +def test_set_index_quantile(nelem, nparts, by): + df = cudf.DataFrame() + df["a"] = np.ascontiguousarray(np.arange(nelem)[::-1]) + df["b"] = np.random.choice(cudf.datasets.names, size=nelem) + ddf = dd.from_pandas(df, npartitions=nparts) + + with dask.config.set(scheduler="single-threaded"): + got = ddf.set_index(by, divisions="quantile") + expect = df.sort_values(by=by).set_index(by) + dd.assert_eq(got, expect) + + def assert_frame_equal_by_index_group(expect, got): assert sorted(expect.columns) == sorted(got.columns) assert sorted(set(got.index)) == sorted(set(expect.index)) From aa4262ea40928dd0eb8a7bd3a3fda3b1083a5421 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Fri, 6 Mar 2020 19:03:21 -0800 Subject: [PATCH 56/61] syncing repartition changes --- python/dask_cudf/dask_cudf/core.py | 23 +++++++++++++---------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py index bae95cca8ee..9a2e9d74de0 100644 --- a/python/dask_cudf/dask_cudf/core.py +++ b/python/dask_cudf/dask_cudf/core.py @@ -318,14 +318,11 @@ def repartition_by_hash( npartitions=None, max_branch=None, disk_path=None, + ignore_index=True, **kwargs, ): """Repartition a dask_cudf DataFrame by hashing. - - Warning: Index will be ignored/dropped during this operation. - Be sure to call reset_index beforehand to preserve the - index as a column. - + Warning: By default, index will be ignored/dropped. Parameter --------- columns : list, default None @@ -340,6 +337,9 @@ def repartition_by_hash( disk_path : str, default None If set to a string value, the repartitioning will be performed "on disk," using a partitioned parquet dataset. + ignore_index : bool, default True + Ignore the index values while shuffling data into new + partitions. This can boost performance significantly. kwargs : dict Other `repartition` arguments. Ignored. """ @@ -359,9 +359,12 @@ def repartition_by_hash( sorting.set_partitions_hash, columns, npartitions, meta=meta ) df2 = self.assign(_partitions=partitions) + index = False + if not ignore_index: + index = df2.index.name or "index" df2.to_parquet( disk_path, - write_index=False, + write_index=not ignore_index, partition_on=["_partitions"], append=False, compression="snappy", @@ -370,7 +373,7 @@ def repartition_by_hash( ) from dask_cudf import read_parquet - return read_parquet(disk_path, index=False).drop( + return read_parquet(disk_path, index=index).drop( columns=["_partitions"] ) @@ -380,7 +383,7 @@ def repartition_by_hash( columns, npartitions, max_branch=max_branch, - ignore_index=True, + ignore_index=ignore_index, ) def repartition(self, *args, **kwargs): @@ -390,8 +393,8 @@ def repartition(self, *args, **kwargs): columns = kwargs.pop("columns", None) if columns: warnings.warn( - "Repartitioning by column hash. Index will be ignored," - "and divisions will lost." + "Repartitioning by column hash. Divisions will lost. " + "Set ignore_index=False to preserve Index values." ) return self.repartition_by_hash(columns=columns, **kwargs) return super().repartition(*args, **kwargs) From 07d8118aea5c19ad69d2e10d5dc27df186fd2b2d Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 10 Mar 2020 15:25:20 -0700 Subject: [PATCH 57/61] changelog --- CHANGELOG.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 69d7ac9f372..5c0d65bd61d 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -58,7 +58,6 @@ - PR #3911 Adding null boolean handling for copy_if_else - PR #4003 Drop old `to_device` utility wrapper function - PR #4002 Adding to_frame and fix for categorical column issue -- PR #4016 Adding experimental code path for dask_cudf sorting - PR #4035 Port NVText tokenize function to libcudf++ - PR #4009 build script update to enable cudf build without installing - PR #3897 Port cuIO JSON reader to cudf::column types @@ -147,6 +146,7 @@ - PR #4339 Port libcudf strings `wrap` api to cython/python - PR #4311 Port nvstrings String Manipulations functions to cuDF Python/Cython - PR #4373 Port nvstrings Regular Expressions functions to cuDF Python/Cython +- PR #4308 Replace dask_cudf sort_values and improve set_index ## Bug Fixes From f39ede75da9fdbda6ced365909e06a5cf627c80b Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 11 Mar 2020 09:37:27 -0700 Subject: [PATCH 58/61] code review --- python/dask_cudf/dask_cudf/sorting.py | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index f92ccc9f7e9..e7f5700acad 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -158,6 +158,10 @@ def _set_partitions_pre(s, divisions): partitions = divisions.searchsorted(s, side="right") - 1 # Use searchsorted to avoid string-compare limitations + # TODO: Simplify after github issue #4432 is resolved... + # partitions[ + # (s >= divisions.iloc[-1]) + # ] = len(divisions) - 2 partitions[ divisions.tail(1).searchsorted(s, side="right").astype("bool") ] = (len(divisions) - 2) @@ -343,10 +347,10 @@ def sort_values( """ Sort by the given list/tuple of column names. """ npartitions = df.npartitions - if isinstance(by, str): - by = [by] - elif isinstance(by, tuple): + if isinstance(by, tuple): by = list(by) + elif not isinstance(by, list): + by = [by] # Step 1 - Calculate new divisions (if necessary) if divisions is None: From 9ad0d6eacb69662c65ead2566782505153f7b450 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Wed, 11 Mar 2020 11:25:19 -0700 Subject: [PATCH 59/61] add multi-stage repartitioning test --- python/dask_cudf/dask_cudf/sorting.py | 6 ++-- python/dask_cudf/dask_cudf/tests/test_core.py | 29 +++++++++++++++++++ 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/sorting.py b/python/dask_cudf/dask_cudf/sorting.py index e7f5700acad..c8b56bf9be7 100644 --- a/python/dask_cudf/dask_cudf/sorting.py +++ b/python/dask_cudf/dask_cudf/sorting.py @@ -27,8 +27,10 @@ def _shuffle_group(df, columns, stage, k, npartitions, ignore_index): c = hash_object_dispatch(df[columns], index=False) typ = np.min_scalar_type(npartitions * 2) c = np.mod(c, npartitions).astype(typ, copy=False) - np.floor_divide(c, k ** stage, out=c) - np.mod(c, k, out=c) + if stage > 0: + np.floor_divide(c, k ** stage, out=c) + elif k < npartitions: + np.mod(c, k, out=c) return group_split_dispatch( df, c.astype(np.int32), k, ignore_index=ignore_index ) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index 9b4746ed08a..9ac788fe7f4 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -382,6 +382,35 @@ def test_repartition_simple_divisions(start, stop): dd.utils.assert_eq(a, b) +def test_repartition_hash_staged(): + by = ["b"] + datarange = 350 + size = 1_000 + gdf = cudf.DataFrame( + { + "a": np.arange(size, dtype="int64"), + "b": np.random.randint(datarange, size=size), + } + ) + ddf = dgd.from_cudf(gdf, npartitions=35) + ddf_new = ddf.repartition(columns=by, max_branch=32) + + # Check that the length was preserved + assert len(ddf_new) == len(ddf) + + # Check that the partitions have unique keys, + # and that the key values are preserved + expect_unique = gdf[by].drop_duplicates().sort_values(by) + got_unique = cudf.concat( + [ + part[by].compute().drop_duplicates() + for part in ddf_new[by].partitions + ], + ignore_index=True, + ).sort_values(by) + dd.assert_eq(got_unique, expect_unique, check_index=False) + + @pytest.mark.parametrize("by", [["b"], ["c"], ["d"], ["b", "c"]]) @pytest.mark.parametrize("npartitions", [4, 5]) def test_repartition_hash(by, npartitions): From ae64f0e23936275e363a57d5bbcead6b1437ce9a Mon Sep 17 00:00:00 2001 From: rjzamora Date: Mon, 16 Mar 2020 18:52:36 -0700 Subject: [PATCH 60/61] remove new use of single-threaded scheduler --- python/dask_cudf/dask_cudf/tests/test_core.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/dask_cudf/dask_cudf/tests/test_core.py b/python/dask_cudf/dask_cudf/tests/test_core.py index 9ac788fe7f4..d4349fa783c 100644 --- a/python/dask_cudf/dask_cudf/tests/test_core.py +++ b/python/dask_cudf/dask_cudf/tests/test_core.py @@ -150,8 +150,7 @@ def test_set_index_quantile(nelem, nparts, by): df["b"] = np.random.choice(cudf.datasets.names, size=nelem) ddf = dd.from_pandas(df, npartitions=nparts) - with dask.config.set(scheduler="single-threaded"): - got = ddf.set_index(by, divisions="quantile") + got = ddf.set_index(by, divisions="quantile") expect = df.sort_values(by=by).set_index(by) dd.assert_eq(got, expect) From 58bb900098bcedd8ad49ef4241466f94a5464459 Mon Sep 17 00:00:00 2001 From: rjzamora Date: Tue, 17 Mar 2020 08:40:26 -0700 Subject: [PATCH 61/61] upstate jitify --- thirdparty/jitify | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/thirdparty/jitify b/thirdparty/jitify index bcd545071c7..e3f867027c1 160000 --- a/thirdparty/jitify +++ b/thirdparty/jitify @@ -1 +1 @@ -Subproject commit bcd545071c7a5ddb28cb6576afc6399eb1286c43 +Subproject commit e3f867027c1d9603b5a677795900465b9fac9cb8