From cb80edc26349e2e358d27fe2ae8e5d6959b77fab Mon Sep 17 00:00:00 2001
From: wangzhenhua <wangzhenhua@huawei.com>
Date: Tue, 1 Nov 2016 13:11:24 +0000
Subject: [PATCH] [SPARK-18111][SQL] Wrong ApproximatePercentile answer when
 multiple records have the minimum value

## What changes were proposed in this pull request?

When multiple records have the minimum value, the answer of ApproximatePercentile is wrong.
## How was this patch tested?

add a test case

Author: wangzhenhua <wangzhenhua@huawei.com>

Closes #15641 from wzhfy/percentile.
---
 .../spark/sql/catalyst/util/QuantileSummaries.scala   |  4 +++-
 .../spark/sql/ApproximatePercentileQuerySuite.scala   | 11 +++++++++++
 2 files changed, 14 insertions(+), 1 deletion(-)

diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala
index 27928c493d..04f4ff2a92 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala
@@ -264,7 +264,9 @@ object QuantileSummaries {
     res.prepend(head)
     // If necessary, add the minimum element:
     val currHead = currentSamples.head
-    if (currHead.value < head.value) {
+    // don't add the minimum element if `currentSamples` has only one element (both `currHead` and
+    // `head` point to the same element)
+    if (currHead.value <= head.value && currentSamples.length > 1) {
       res.prepend(currentSamples.head)
     }
     res.toArray
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
index 37d7c442bb..e98092df49 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala
@@ -64,6 +64,17 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext {
     }
   }
 
+  test("percentile_approx, multiple records with the minimum value in a partition") {
+    withTempView(table) {
+      spark.sparkContext.makeRDD(Seq(1, 1, 2, 1, 1, 3, 1, 1, 4, 1, 1, 5), 4).toDF("col")
+        .createOrReplaceTempView(table)
+      checkAnswer(
+        spark.sql(s"SELECT percentile_approx(col, array(0.5)) FROM $table"),
+        Row(Seq(1.0D))
+      )
+    }
+  }
+
   test("percentile_approx, with different accuracies") {
 
     withTempView(table) {
-- 
GitLab