|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.spark.sql.execution.ui |
| 19 | + |
| 20 | +import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} |
| 21 | +import org.apache.spark.LocalSparkContext.withSpark |
| 22 | +import org.apache.spark.internal.config |
| 23 | +import org.apache.spark.sql.{Column, SparkSession} |
| 24 | +import org.apache.spark.sql.catalyst.util.quietly |
| 25 | +import org.apache.spark.sql.functions._ |
| 26 | + |
| 27 | +class SQLListenerMemorySuite extends SparkFunSuite { |
| 28 | + |
| 29 | + test("SPARK-22471 - _stageIdToStageMetrics grows too large on long executions") { |
| 30 | + quietly { |
| 31 | + val conf = new SparkConf() |
| 32 | + .setMaster("local[*]") |
| 33 | + .setAppName("MemoryLeakTest") |
| 34 | + /* Don't retry the tasks to run this test quickly */ |
| 35 | + .set(config.MAX_TASK_FAILURES, 1) |
| 36 | + .set("spark.ui.retainedStages", "50") |
| 37 | + withSpark(new SparkContext(conf)) { sc => |
| 38 | + SparkSession.sqlListener.set(null) |
| 39 | + val spark = new SparkSession(sc) |
| 40 | + import spark.implicits._ |
| 41 | + |
| 42 | + val sample = List( |
| 43 | + (1, 10), |
| 44 | + (2, 20), |
| 45 | + (3, 30) |
| 46 | + ).toDF("id", "value") |
| 47 | + |
| 48 | + /* Some complex computation with many stages. */ |
| 49 | + val joins = 1 to 100 |
| 50 | + val summedCol: Column = joins |
| 51 | + .map(j => col(s"value$j")) |
| 52 | + .reduce(_ + _) |
| 53 | + val res = joins |
| 54 | + .map { j => |
| 55 | + sample.select($"id", $"value" * j as s"value$j") |
| 56 | + } |
| 57 | + .reduce(_.join(_, "id")) |
| 58 | + .select($"id", summedCol as "value") |
| 59 | + .groupBy("id") |
| 60 | + .agg(sum($"value") as "value") |
| 61 | + .orderBy("id") |
| 62 | + res.collect() |
| 63 | + |
| 64 | + sc.listenerBus.waitUntilEmpty(10000) |
| 65 | + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 50) |
| 66 | + } |
| 67 | + } |
| 68 | + } |
| 69 | + |
| 70 | + test("no memory leak") { |
| 71 | + quietly { |
| 72 | + val conf = new SparkConf() |
| 73 | + .setMaster("local") |
| 74 | + .setAppName("test") |
| 75 | + .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly |
| 76 | + .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly |
| 77 | + withSpark(new SparkContext(conf)) { sc => |
| 78 | + SparkSession.sqlListener.set(null) |
| 79 | + val spark = new SparkSession(sc) |
| 80 | + import spark.implicits._ |
| 81 | + // Run 100 successful executions and 100 failed executions. |
| 82 | + // Each execution only has one job and one stage. |
| 83 | + for (i <- 0 until 100) { |
| 84 | + val df = Seq( |
| 85 | + (1, 1), |
| 86 | + (2, 2) |
| 87 | + ).toDF() |
| 88 | + df.collect() |
| 89 | + try { |
| 90 | + df.foreach(_ => throw new RuntimeException("Oops")) |
| 91 | + } catch { |
| 92 | + case e: SparkException => // This is expected for a failed job |
| 93 | + } |
| 94 | + } |
| 95 | + sc.listenerBus.waitUntilEmpty(10000) |
| 96 | + assert(spark.sharedState.listener.getCompletedExecutions.size <= 50) |
| 97 | + assert(spark.sharedState.listener.getFailedExecutions.size <= 50) |
| 98 | + // 50 for successful executions and 50 for failed executions |
| 99 | + assert(spark.sharedState.listener.executionIdToData.size <= 100) |
| 100 | + assert(spark.sharedState.listener.jobIdToExecutionId.size <= 100) |
| 101 | + assert(spark.sharedState.listener.stageIdToStageMetrics.size <= 100) |
| 102 | + } |
| 103 | + } |
| 104 | + } |
| 105 | + |
| 106 | +} |
0 commit comments