From 08b7c82be3effe094e40618fe992d3c50c3e2d98 Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Wed, 8 Nov 2017 18:41:36 +0300 Subject: [PATCH 1/7] Add reproducer for the issue SPARK-22471 --- .../execution/ui/SQLListenerMemorySuite.scala | 81 +++++++++++++++++++ 1 file changed, 81 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala new file mode 100644 index 0000000000000..e0dd48e13e71c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.ui + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.LocalSparkContext.withSpark +import org.apache.spark.internal.config +import org.apache.spark.sql.{Column, DataFrame, SparkSession} +import org.apache.spark.sql.functions._ + +class SQLListenerMemorySuite extends SparkFunSuite { + + test("_stageIdToStageMetrics grows too large on long executions") { + val conf = new SparkConf() + .setMaster("local[*]") + .setAppName("MemoryLeakTest") + /* Don't retry the tasks to run this test quickly */ + .set(config.MAX_TASK_FAILURES, 1) +// .set("spark.sql.ui.retainedExecutions", "50") + .set("spark.ui.retainedStages", "50") + withSpark(new SparkContext(conf)) { sc => + SparkSession.sqlListener.set(null) + val spark = new SparkSession(sc) + import spark.implicits._ + + Console.err.print("--- executionIdToData: " + + spark.sharedState.listener.executionIdToData.size + "\n") + Console.err.print("--- stageIdToStageMetrics: " + + spark.sharedState.listener.stageIdToStageMetrics.size + "\n") + + val sample = List( + (1, 10), + (2, 20), + (3, 30) + ).toDF("id", "value") + + /* Some complex computation with many stages. */ + val joins = 1 to 100 + val summedCol: Column = joins + .map(j => col(s"value$j")) + .reduce(_ + _) + val res = joins + .map { j => + sample.select($"id", $"value" * j as s"value$j") + } + .reduce(_.join(_, "id")) + .select($"id", summedCol as "value") + .groupBy("id") + .agg(sum($"value") as "value") + .orderBy("id") + res.collect() + + sc.listenerBus.waitUntilEmpty(10000) + assert(spark.sharedState.listener.getCompletedExecutions.size == 1) + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 50) + + res.show() + + Console.err.print("--- executionIdToData: " + + spark.sharedState.listener.executionIdToData.size + "\n") + Console.err.print("--- stageIdToStageMetrics: " + + spark.sharedState.listener.stageIdToStageMetrics.size + "\n") + Thread.sleep(10000L) + } + } +} From 2502a7e9846e359d793c485db1d3abef8a2c1e12 Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Wed, 8 Nov 2017 18:41:54 +0300 Subject: [PATCH 2/7] Add fix for the issue SPARK-22471 --- .../apache/spark/sql/execution/ui/SQLListener.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index b4a91230a0012..e8b90f631ad31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -101,6 +101,8 @@ class SQLListener(conf: SparkConf) extends SparkListener with Logging { private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) + private val retainedStages = conf.getInt("spark.ui.retainedStages", 1000) + private val activeExecutions = mutable.HashMap[Long, SQLExecutionUIData]() // Old data in the following fields must be removed in "trimExecutionsIfNecessary". @@ -113,7 +115,7 @@ class SQLListener(conf: SparkConf) extends SparkListener with Logging { */ private val _jobIdToExecutionId = mutable.HashMap[Long, Long]() - private val _stageIdToStageMetrics = mutable.HashMap[Long, SQLStageMetrics]() + private val _stageIdToStageMetrics = mutable.LinkedHashMap[Long, SQLStageMetrics]() private val failedExecutions = mutable.ListBuffer[SQLExecutionUIData]() @@ -207,6 +209,14 @@ class SQLListener(conf: SparkConf) extends SparkListener with Logging { } } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + val extraStages = _stageIdToStageMetrics.size - retainedStages + if (extraStages > 0) { + val toRemove = _stageIdToStageMetrics.take(extraStages).keys + _stageIdToStageMetrics --= toRemove + } + } + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { if (taskEnd.taskMetrics != null) { updateTaskAccumulatorValues( From 2a13530db9ec611b6ee55fc9d79bd8aac5c01862 Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Wed, 8 Nov 2017 23:39:02 +0300 Subject: [PATCH 3/7] Remove debug print and irrelevant checks. Add a reference to the issue. --- .../execution/ui/SQLListenerMemorySuite.scala | 17 ++--------------- 1 file changed, 2 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala index e0dd48e13e71c..0dd0392da32a1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala @@ -20,29 +20,23 @@ package org.apache.spark.sql.execution.ui import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.LocalSparkContext.withSpark import org.apache.spark.internal.config -import org.apache.spark.sql.{Column, DataFrame, SparkSession} +import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.functions._ class SQLListenerMemorySuite extends SparkFunSuite { - test("_stageIdToStageMetrics grows too large on long executions") { + test("SPARK-22471 - _stageIdToStageMetrics grows too large on long executions") { val conf = new SparkConf() .setMaster("local[*]") .setAppName("MemoryLeakTest") /* Don't retry the tasks to run this test quickly */ .set(config.MAX_TASK_FAILURES, 1) -// .set("spark.sql.ui.retainedExecutions", "50") .set("spark.ui.retainedStages", "50") withSpark(new SparkContext(conf)) { sc => SparkSession.sqlListener.set(null) val spark = new SparkSession(sc) import spark.implicits._ - Console.err.print("--- executionIdToData: " - + spark.sharedState.listener.executionIdToData.size + "\n") - Console.err.print("--- stageIdToStageMetrics: " - + spark.sharedState.listener.stageIdToStageMetrics.size + "\n") - val sample = List( (1, 10), (2, 20), @@ -69,13 +63,6 @@ class SQLListenerMemorySuite extends SparkFunSuite { assert(spark.sharedState.listener.getCompletedExecutions.size == 1) assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 50) - res.show() - - Console.err.print("--- executionIdToData: " - + spark.sharedState.listener.executionIdToData.size + "\n") - Console.err.print("--- stageIdToStageMetrics: " - + spark.sharedState.listener.stageIdToStageMetrics.size + "\n") - Thread.sleep(10000L) } } } From 98f7b23fb52ffd11ae92716c871e5aa06ea61428 Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Wed, 8 Nov 2017 23:47:44 +0300 Subject: [PATCH 4/7] Remove debug print and irrelevant checks. Add a reference to the issue. --- .../apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala | 2 -- 1 file changed, 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala index 0dd0392da32a1..29766b5868425 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala @@ -60,9 +60,7 @@ class SQLListenerMemorySuite extends SparkFunSuite { res.collect() sc.listenerBus.waitUntilEmpty(10000) - assert(spark.sharedState.listener.getCompletedExecutions.size == 1) assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 50) - } } } From 80755ece91703b3b6436f88e14eb11251ae6678f Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Thu, 9 Nov 2017 00:21:42 +0300 Subject: [PATCH 5/7] Collect memory-related tests on SQLListener in the same suite --- .../execution/ui/SQLListenerMemorySuite.scala | 106 ++++++++++++------ .../sql/execution/ui/SQLListenerSuite.scala | 47 +------- 2 files changed, 74 insertions(+), 79 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala index 29766b5868425..24a09f37c645c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerMemorySuite.scala @@ -17,50 +17,90 @@ package org.apache.spark.sql.execution.ui -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} import org.apache.spark.LocalSparkContext.withSpark import org.apache.spark.internal.config import org.apache.spark.sql.{Column, SparkSession} +import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.functions._ class SQLListenerMemorySuite extends SparkFunSuite { test("SPARK-22471 - _stageIdToStageMetrics grows too large on long executions") { - val conf = new SparkConf() - .setMaster("local[*]") - .setAppName("MemoryLeakTest") - /* Don't retry the tasks to run this test quickly */ - .set(config.MAX_TASK_FAILURES, 1) - .set("spark.ui.retainedStages", "50") - withSpark(new SparkContext(conf)) { sc => - SparkSession.sqlListener.set(null) - val spark = new SparkSession(sc) - import spark.implicits._ + quietly { + val conf = new SparkConf() + .setMaster("local[*]") + .setAppName("MemoryLeakTest") + /* Don't retry the tasks to run this test quickly */ + .set(config.MAX_TASK_FAILURES, 1) + .set("spark.ui.retainedStages", "50") + withSpark(new SparkContext(conf)) { sc => + SparkSession.sqlListener.set(null) + val spark = new SparkSession(sc) + import spark.implicits._ - val sample = List( - (1, 10), - (2, 20), - (3, 30) - ).toDF("id", "value") + val sample = List( + (1, 10), + (2, 20), + (3, 30) + ).toDF("id", "value") - /* Some complex computation with many stages. */ - val joins = 1 to 100 - val summedCol: Column = joins - .map(j => col(s"value$j")) - .reduce(_ + _) - val res = joins - .map { j => - sample.select($"id", $"value" * j as s"value$j") - } - .reduce(_.join(_, "id")) - .select($"id", summedCol as "value") - .groupBy("id") - .agg(sum($"value") as "value") - .orderBy("id") - res.collect() + /* Some complex computation with many stages. */ + val joins = 1 to 100 + val summedCol: Column = joins + .map(j => col(s"value$j")) + .reduce(_ + _) + val res = joins + .map { j => + sample.select($"id", $"value" * j as s"value$j") + } + .reduce(_.join(_, "id")) + .select($"id", summedCol as "value") + .groupBy("id") + .agg(sum($"value") as "value") + .orderBy("id") + res.collect() + + sc.listenerBus.waitUntilEmpty(10000) + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 50) + } + } + } - sc.listenerBus.waitUntilEmpty(10000) - assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 50) + test("no memory leak") { + quietly { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly + .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly + withSpark(new SparkContext(conf)) { sc => + SparkSession.sqlListener.set(null) + val spark = new SparkSession(sc) + import spark.implicits._ + // Run 100 successful executions and 100 failed executions. + // Each execution only has one job and one stage. + for (i <- 0 until 100) { + val df = Seq( + (1, 1), + (2, 2) + ).toDF() + df.collect() + try { + df.foreach(_ => throw new RuntimeException("Oops")) + } catch { + case e: SparkException => // This is expected for a failed job + } + } + sc.listenerBus.waitUntilEmpty(10000) + assert(spark.sharedState.listener.getCompletedExecutions.size <= 50) + assert(spark.sharedState.listener.getFailedExecutions.size <= 50) + // 50 for successful executions and 50 for failed executions + assert(spark.sharedState.listener.executionIdToData.size <= 100) + assert(spark.sharedState.listener.jobIdToExecutionId.size <= 100) + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 100) + } } } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala index e6cd41e4facf1..23420a5af59a2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLListenerSuite.scala @@ -24,14 +24,12 @@ import org.mockito.Mockito.mock import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.internal.config import org.apache.spark.rdd.RDD import org.apache.spark.scheduler._ -import org.apache.spark.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlanInfo, SQLExecution} import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.test.SharedSQLContext @@ -485,46 +483,3 @@ private case class MyPlan(sc: SparkContext, expectedValue: Long) extends LeafExe sc.emptyRDD } } - - -class SQLListenerMemoryLeakSuite extends SparkFunSuite { - - test("no memory leak") { - quietly { - val conf = new SparkConf() - .setMaster("local") - .setAppName("test") - .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly - .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly - val sc = new SparkContext(conf) - try { - SparkSession.sqlListener.set(null) - val spark = new SparkSession(sc) - import spark.implicits._ - // Run 100 successful executions and 100 failed executions. - // Each execution only has one job and one stage. - for (i <- 0 until 100) { - val df = Seq( - (1, 1), - (2, 2) - ).toDF() - df.collect() - try { - df.foreach(_ => throw new RuntimeException("Oops")) - } catch { - case e: SparkException => // This is expected for a failed job - } - } - sc.listenerBus.waitUntilEmpty(10000) - assert(spark.sharedState.listener.getCompletedExecutions.size <= 50) - assert(spark.sharedState.listener.getFailedExecutions.size <= 50) - // 50 for successful executions and 50 for failed executions - assert(spark.sharedState.listener.executionIdToData.size <= 100) - assert(spark.sharedState.listener.jobIdToExecutionId.size <= 100) - assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 100) - } finally { - sc.stop() - } - } - } -} From 4a0582de45a1db6f6a48c85eb2a3e7383cc56c04 Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Fri, 10 Nov 2017 00:59:19 +0300 Subject: [PATCH 6/7] Don't hardcode magic numbers --- .../scala/org/apache/spark/sql/execution/ui/SQLListener.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index e8b90f631ad31..54e475ffe3d97 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -101,7 +101,8 @@ class SQLListener(conf: SparkConf) extends SparkListener with Logging { private val retainedExecutions = conf.getInt("spark.sql.ui.retainedExecutions", 1000) - private val retainedStages = conf.getInt("spark.ui.retainedStages", 1000) + private val retainedStages = conf.getInt("spark.ui.retainedStages", + SparkUI.DEFAULT_RETAINED_STAGES) private val activeExecutions = mutable.HashMap[Long, SQLExecutionUIData]() From 3ecce56f084e7e08f0c98d82192a1cd313fef65f Mon Sep 17 00:00:00 2001 From: Arseniy Tashoyan Date: Fri, 10 Nov 2017 11:19:38 +0300 Subject: [PATCH 7/7] Synchronize access to _stageIdToStageMetrics --- .../scala/org/apache/spark/sql/execution/ui/SQLListener.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index 54e475ffe3d97..e0c8cb3487c92 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -210,7 +210,7 @@ class SQLListener(conf: SparkConf) extends SparkListener with Logging { } } - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = synchronized { val extraStages = _stageIdToStageMetrics.size - retainedStages if (extraStages > 0) { val toRemove = _stageIdToStageMetrics.take(extraStages).keys