You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by li...@apache.org on 2018/09/28 22:08:18 UTC

spark git commit: [SPARK-25429][SQL] Use Set instead of Array to improve lookup performance

Repository: spark
Updated Branches:
  refs/heads/master 7deef7a49 -> a28146568


[SPARK-25429][SQL] Use Set instead of Array to improve lookup performance

## What changes were proposed in this pull request?

Use `Set` instead of `Array` to improve `accumulatorIds.contains(acc.id)` performance.

This PR close https://github.com/apache/spark/pull/22420

## How was this patch tested?

manual tests.
Benchmark code:
```scala
def benchmark(func: () => Unit): Long = {
  val start = System.currentTimeMillis()
  func()
  val end = System.currentTimeMillis()
  end - start
}

val range = Range(1, 1000000)
val set = range.toSet
val array = range.toArray

for (i <- 0 until 5) {
  val setExecutionTime =
    benchmark(() => for (i <- 0 until 500) { set.contains(scala.util.Random.nextInt()) })
  val arrayExecutionTime =
    benchmark(() => for (i <- 0 until 500) { array.contains(scala.util.Random.nextInt()) })
  println(s"set execution time: $setExecutionTime, array execution time: $arrayExecutionTime")
}
```

Benchmark result:
```
set execution time: 4, array execution time: 2760
set execution time: 1, array execution time: 1911
set execution time: 3, array execution time: 2043
set execution time: 12, array execution time: 2214
set execution time: 6, array execution time: 1770
```

Closes #22579 from wangyum/SPARK-25429.

Authored-by: Yuming Wang <yu...@ebay.com>
Signed-off-by: gatorsmile <ga...@gmail.com>


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a2814656
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a2814656
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a2814656

Branch: refs/heads/master
Commit: a281465686e8099bb2c0fa4f2ef4822b6e634269
Parents: 7deef7a
Author: Yuming Wang <yu...@ebay.com>
Authored: Fri Sep 28 15:08:15 2018 -0700
Committer: gatorsmile <ga...@gmail.com>
Committed: Fri Sep 28 15:08:15 2018 -0700

----------------------------------------------------------------------
 .../apache/spark/sql/execution/ui/SQLAppStatusListener.scala   | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/a2814656/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
index d254af4..1199eec 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala
@@ -81,9 +81,9 @@ class SQLAppStatusListener(
 
     // Record the accumulator IDs for the stages of this job, so that the code that keeps
     // track of the metrics knows which accumulators to look at.
-    val accumIds = exec.metrics.map(_.accumulatorId).sorted.toList
+    val accumIds = exec.metrics.map(_.accumulatorId).toSet
     event.stageIds.foreach { id =>
-      stageMetrics.put(id, new LiveStageMetrics(id, 0, accumIds.toArray, new ConcurrentHashMap()))
+      stageMetrics.put(id, new LiveStageMetrics(id, 0, accumIds, new ConcurrentHashMap()))
     }
 
     exec.jobs = exec.jobs + (jobId -> JobExecutionStatus.RUNNING)
@@ -382,7 +382,7 @@ private class LiveExecutionData(val executionId: Long) extends LiveEntity {
 private class LiveStageMetrics(
     val stageId: Int,
     var attemptId: Int,
-    val accumulatorIds: Array[Long],
+    val accumulatorIds: Set[Long],
     val taskMetrics: ConcurrentHashMap[Long, LiveTaskMetrics])
 
 private class LiveTaskMetrics(


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org