diff --git a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js
index 4337c42087e79986377d189e73afdc4ada2029f9..1b0d4692d9cd03e8084faf6f4e970b1fc4bdb419 100644
--- a/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js
+++ b/core/src/main/resources/org/apache/spark/ui/static/spark-dag-viz.js
@@ -222,10 +222,11 @@ function renderDagVizForJob(svgContainer) {
       var attemptId = 0
       var stageLink = d3.select("#stage-" + stageId + "-" + attemptId)
         .select("a.name-link")
-        .attr("href") + "&expandDagViz=true";
+        .attr("href");
       container = svgContainer
         .append("a")
         .attr("xlink:href", stageLink)
+        .attr("onclick", "window.localStorage.setItem(expandDagVizArrowKey(false), true)")
         .append("g")
         .attr("id", containerId);
     }
diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
index ddd7f713fe417950dae138879a0d067623115316..0493513d667c252ef969b3c17c3339e6d98e6f51 100644
--- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
+++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala
@@ -408,13 +408,6 @@ private[spark] object UIUtils extends Logging {
     </sup>
   }
 
-  /** Return a script element that automatically expands the DAG visualization on page load. */
-  def expandDagVizOnLoad(forJob: Boolean): Seq[Node] = {
-    <script type="text/javascript">
-      {Unparsed("$(document).ready(function() { toggleDagViz(" + forJob + ") });")}
-    </script>
-  }
-
   /**
    * Returns HTML rendering of a job or stage description. It will try to parse the string as HTML
    * and make sure that it only contains anchors with root-relative links. Otherwise,
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
index 0b68b88566b703709d7bed6813e2155252630252..689ab7dd5ed62d4a02cefa999edb69581cca375f 100644
--- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
+++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala
@@ -107,10 +107,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
       val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100)
       val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize)
 
-      // If this is set, expand the dag visualization by default
-      val expandDagVizParam = request.getParameter("expandDagViz")
-      val expandDagViz = expandDagVizParam != null && expandDagVizParam.toBoolean
-
       val stageId = parameterId.toInt
       val stageAttemptId = parameterAttempt.toInt
       val stageDataOption = progressListener.stageIdToData.get((stageId, stageAttemptId))
@@ -263,13 +259,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
       val dagViz = UIUtils.showDagVizForStage(
         stageId, operationGraphListener.getOperationGraphForStage(stageId))
 
-      val maybeExpandDagViz: Seq[Node] =
-        if (expandDagViz) {
-          UIUtils.expandDagVizOnLoad(forJob = false)
-        } else {
-          Seq.empty
-        }
-
       val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value")
       def accumulableRow(acc: AccumulableInfo): Seq[Node] = {
         (acc.name, acc.value) match {
@@ -578,7 +567,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") {
       val content =
         summary ++
         dagViz ++
-        maybeExpandDagViz ++
         showAdditionalMetrics ++
         makeTimeline(
           // Only show the tasks in the table