diff --git a/dask/dataframe/core.py b/dask/dataframe/core.py index 92305916b0d..a3dd506962e 100644 --- a/dask/dataframe/core.py +++ b/dask/dataframe/core.py @@ -1008,7 +1008,9 @@ def repartition_divisions(a, b, name, out1, out2): new divisions name: str name of old dataframe - out: str + out1: str + name of temporary splits + out2: str name of new dataframe >>> repartition_divisions([1, 3, 7], [1, 4, 6, 7], 'a', 'b', 'c') # doctest: +SKIP @@ -1048,7 +1050,6 @@ def repartition_divisions(a, b, name, out1, out2): d[(out1, k - 1)] = tup[:-1] + (True,) c.append(a[-1]) - i, j = 0, 1 while j < len(b): tmp = [] @@ -1068,6 +1069,13 @@ def repartition_divisions(a, b, name, out1, out2): def repartition(df, divisions): """ Repartition dataframe along new divisions + Dask.DataFrame objects are partitioned along their index. Often when + multiple dataframes interact we need to align these partitionings. The + ``repartition`` function constructs a new DataFrame object holding the same + data but partitioned on different values. It does this by performing a + sequence of ``loc`` and ``concat`` calls to split and merge the previous + generation of partitions. + >>> df = df.repartition([0, 5, 10, 20]) # doctest: +SKIP Also works on Pandas objects