From 5b825814ebe786f2406a3f6b77caf12f86781eb7 Mon Sep 17 00:00:00 2001
From: rjzamora <rzamora217@gmail.com>
Date: Wed, 29 Jan 2020 13:03:37 -0800
Subject: [PATCH 1/3] add short-circuit for single partition

---
 python/dask_cudf/dask_cudf/core.py            | 14 +++++++++-----
 python/dask_cudf/dask_cudf/tests/test_sort.py |  4 +++-
 2 files changed, 12 insertions(+), 6 deletions(-)

diff --git a/python/dask_cudf/dask_cudf/core.py b/python/dask_cudf/dask_cudf/core.py
index 2c950967a59..c96ead7ebcf 100644
--- a/python/dask_cudf/dask_cudf/core.py
+++ b/python/dask_cudf/dask_cudf/core.py
@@ -209,11 +209,15 @@ def sort_values(self, by, ignore_index=False):
         ---------
         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 self.npartitions == 1:
+            df = self.map_partitions(M.sort_values, by)
+        else:
+            parts = self.to_delayed()
+            sorted_parts = batcher_sortnet.sort_delayed_frame(parts, by)
+            df = from_delayed(sorted_parts, meta=self._meta)
+        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
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 41259d0f3b3bfd3d6d20ddff8c3468f45008d5ed Mon Sep 17 00:00:00 2001
From: rjzamora <rzamora217@gmail.com>
Date: Wed, 29 Jan 2020 13:53:35 -0800
Subject: [PATCH 2/3] update tests

---
 python/dask_cudf/dask_cudf/tests/test_sort.py | 16 +++++++++++++++-
 1 file changed, 15 insertions(+), 1 deletion(-)

diff --git a/python/dask_cudf/dask_cudf/tests/test_sort.py b/python/dask_cudf/dask_cudf/tests/test_sort.py
index fe1ef83e5d1..7e63f7a7e87 100644
--- a/python/dask_cudf/dask_cudf/tests/test_sort.py
+++ b/python/dask_cudf/dask_cudf/tests/test_sort.py
@@ -8,7 +8,7 @@
 import cudf
 
 
-@pytest.mark.parametrize("by", ["a", "b"])
+@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):
@@ -53,3 +53,17 @@ def test_sort_binned_meta():
     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()
+    nelem = 1000
+    df["a"] = np.ascontiguousarray(np.arange(nelem)[::-1])
+    df["b"] = np.arange(100, nelem + 100)
+    ddf = dd.from_pandas(df, npartitions=1)
+
+    with dask.config.set(scheduler="single-threaded"):
+        got = ddf.sort_values(by=by)
+    expect = df.sort_values(by=by)
+    dd.assert_eq(got, expect)

From cf9decc911cbf0bb9fd70033838169d8ed0f0a29 Mon Sep 17 00:00:00 2001
From: rjzamora <rzamora217@gmail.com>
Date: Wed, 29 Jan 2020 14:03:00 -0800
Subject: [PATCH 3/3] changelog

---
 CHANGELOG.md | 1 +
 1 file changed, 1 insertion(+)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 4a5a6acd132..e927bab123c 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -21,6 +21,7 @@
 - PR #3970 Fix for Series Pickle
 - PR #3964 Restore legacy NVStrings and NVCategory dependencies in Java jar
 - PR #3982 Fix java unary op enum and add missing ops
+- PR #4000 Fix dask_cudf sort_values performance for single partitions
 
 
 # cuDF 0.12.0 (Date TBD)