Repository: spark
Updated Branches:
  refs/heads/master c0d1bf032 -> 78fa1be29


[SPARK-25926][CORE] Move config entries in core module to internal.config.

## What changes were proposed in this pull request?

Currently definitions of config entries in `core` module are in several files 
separately. We should move them into `internal/config` to be easy to manage.

## How was this patch tested?

Existing tests.

Closes #22928 from ueshin/issues/SPARK-25926/single_config_file.

Authored-by: Takuya UESHIN <ues...@databricks.com>
Signed-off-by: Wenchen Fan <wenc...@databricks.com>


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

Branch: refs/heads/master
Commit: 78fa1be29bc9fbe98dd0226418aafc221c5e5309
Parents: c0d1bf0
Author: Takuya UESHIN <ues...@databricks.com>
Authored: Tue Nov 6 09:18:17 2018 +0800
Committer: Wenchen Fan <wenc...@databricks.com>
Committed: Tue Nov 6 09:18:17 2018 +0800

----------------------------------------------------------------------
 .../main/scala/org/apache/spark/SparkConf.scala |  2 +-
 .../deploy/history/FsHistoryProvider.scala      |  4 +-
 .../spark/deploy/history/HistoryServer.scala    |  2 +-
 .../history/HistoryServerDiskManager.scala      |  3 +-
 .../apache/spark/deploy/history/config.scala    | 67 --------------------
 .../apache/spark/internal/config/History.scala  | 65 +++++++++++++++++++
 .../apache/spark/internal/config/Status.scala   | 58 +++++++++++++++++
 .../apache/spark/status/AppStatusListener.scala |  3 +-
 .../apache/spark/status/AppStatusSource.scala   | 11 +---
 .../spark/status/ElementTrackingStore.scala     |  3 +-
 .../scala/org/apache/spark/status/config.scala  | 54 ----------------
 .../scala/org/apache/spark/SparkConfSuite.scala |  2 +-
 .../deploy/history/FsHistoryProviderSuite.scala |  2 +-
 .../history/HistoryServerDiskManagerSuite.scala |  3 +-
 .../deploy/history/HistoryServerSuite.scala     |  2 +-
 .../spark/status/AppStatusListenerSuite.scala   |  3 +-
 .../status/ElementTrackingStoreSuite.scala      |  3 +-
 .../org/apache/spark/ui/StagePageSuite.scala    |  2 +-
 .../org/apache/spark/ui/UISeleniumSuite.scala   |  2 +-
 .../cluster/mesos/MesosSchedulerUtils.scala     |  2 +-
 .../sql/execution/ui/SQLAppStatusListener.scala |  2 +-
 .../ui/SQLAppStatusListenerSuite.scala          |  2 +-
 22 files changed, 143 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/SparkConf.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala 
b/core/src/main/scala/org/apache/spark/SparkConf.scala
index 8537c53..21c5cbc 100644
--- a/core/src/main/scala/org/apache/spark/SparkConf.scala
+++ b/core/src/main/scala/org/apache/spark/SparkConf.scala
@@ -25,9 +25,9 @@ import scala.collection.mutable.LinkedHashSet
 
 import org.apache.avro.{Schema, SchemaNormalization}
 
-import org.apache.spark.deploy.history.config._
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
+import org.apache.spark.internal.config.History._
 import org.apache.spark.serializer.KryoSerializer
 import org.apache.spark.util.Utils
 

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
index c4517d3..2230bc8 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala
@@ -42,13 +42,14 @@ import org.fusesource.leveldbjni.internal.NativeDB
 import org.apache.spark.{SecurityManager, SparkConf, SparkException}
 import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.History._
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.io.CompressionCodec
 import org.apache.spark.scheduler._
 import org.apache.spark.scheduler.ReplayListenerBus._
 import org.apache.spark.status._
 import org.apache.spark.status.KVUtils._
 import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo}
-import org.apache.spark.status.config._
 import org.apache.spark.ui.SparkUI
 import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils}
 import org.apache.spark.util.kvstore._
@@ -86,7 +87,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, 
clock: Clock)
     this(conf, new SystemClock())
   }
 
-  import config._
   import FsHistoryProvider._
 
   // Interval between safemode checks.

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
index 56f3f59..5856c70 100644
--- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
+++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala
@@ -28,9 +28,9 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, 
ServletHolder}
 
 import org.apache.spark.{SecurityManager, SparkConf}
 import org.apache.spark.deploy.SparkHadoopUtil
-import org.apache.spark.deploy.history.config.HISTORY_SERVER_UI_PORT
 import org.apache.spark.internal.Logging
 import org.apache.spark.internal.config._
+import org.apache.spark.internal.config.History.HISTORY_SERVER_UI_PORT
 import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, 
UIRoot}
 import org.apache.spark.ui.{SparkUI, UIUtils, WebUI}
 import org.apache.spark.ui.JettyUtils._

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala
 
b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala
index ad0dd23..0a1f333 100644
--- 
a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala
+++ 
b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala
@@ -27,6 +27,7 @@ import org.apache.commons.io.FileUtils
 
 import org.apache.spark.SparkConf
 import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.History._
 import org.apache.spark.status.KVUtils._
 import org.apache.spark.util.{Clock, Utils}
 import org.apache.spark.util.kvstore.KVStore
@@ -50,8 +51,6 @@ private class HistoryServerDiskManager(
     listing: KVStore,
     clock: Clock) extends Logging {
 
-  import config._
-
   private val appStoreDir = new File(path, "apps")
   if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) {
     throw new IllegalArgumentException(s"Failed to create app directory 
($appStoreDir).")

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/deploy/history/config.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/deploy/history/config.scala 
b/core/src/main/scala/org/apache/spark/deploy/history/config.scala
deleted file mode 100644
index 25ba9ed..0000000
--- a/core/src/main/scala/org/apache/spark/deploy/history/config.scala
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * 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.deploy.history
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.spark.internal.config.ConfigBuilder
-import org.apache.spark.network.util.ByteUnit
-
-private[spark] object config {
-
-  val DEFAULT_LOG_DIR = "file:/tmp/spark-events"
-
-  val EVENT_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory")
-    .stringConf
-    .createWithDefault(DEFAULT_LOG_DIR)
-
-  val MAX_LOG_AGE_S = ConfigBuilder("spark.history.fs.cleaner.maxAge")
-    .timeConf(TimeUnit.SECONDS)
-    .createWithDefaultString("7d")
-
-  val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path")
-    .doc("Local directory where to cache application history information. By 
default this is " +
-      "not set, meaning all history information will be kept in memory.")
-    .stringConf
-    .createOptional
-
-  val MAX_LOCAL_DISK_USAGE = ConfigBuilder("spark.history.store.maxDiskUsage")
-    .bytesConf(ByteUnit.BYTE)
-    .createWithDefaultString("10g")
-
-  val HISTORY_SERVER_UI_PORT = ConfigBuilder("spark.history.ui.port")
-    .doc("Web UI port to bind Spark History Server")
-    .intConf
-    .createWithDefault(18080)
-
-  val FAST_IN_PROGRESS_PARSING =
-    ConfigBuilder("spark.history.fs.inProgressOptimization.enabled")
-      .doc("Enable optimized handling of in-progress logs. This option may 
leave finished " +
-        "applications that fail to rename their event logs listed as 
in-progress.")
-      .booleanConf
-      .createWithDefault(true)
-
-  val END_EVENT_REPARSE_CHUNK_SIZE =
-    ConfigBuilder("spark.history.fs.endEventReparseChunkSize")
-      .doc("How many bytes to parse at the end of log files looking for the 
end event. " +
-        "This is used to speed up generation of application listings by 
skipping unnecessary " +
-        "parts of event log files. It can be disabled by setting this config 
to 0.")
-      .bytesConf(ByteUnit.BYTE)
-      .createWithDefaultString("1m")
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/internal/config/History.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala 
b/core/src/main/scala/org/apache/spark/internal/config/History.scala
new file mode 100644
index 0000000..3f74eb3
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala
@@ -0,0 +1,65 @@
+/*
+ * 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.internal.config
+
+import java.util.concurrent.TimeUnit
+
+import org.apache.spark.network.util.ByteUnit
+
+private[spark] object History {
+
+  val DEFAULT_LOG_DIR = "file:/tmp/spark-events"
+
+  val EVENT_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory")
+    .stringConf
+    .createWithDefault(DEFAULT_LOG_DIR)
+
+  val MAX_LOG_AGE_S = ConfigBuilder("spark.history.fs.cleaner.maxAge")
+    .timeConf(TimeUnit.SECONDS)
+    .createWithDefaultString("7d")
+
+  val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path")
+    .doc("Local directory where to cache application history information. By 
default this is " +
+      "not set, meaning all history information will be kept in memory.")
+    .stringConf
+    .createOptional
+
+  val MAX_LOCAL_DISK_USAGE = ConfigBuilder("spark.history.store.maxDiskUsage")
+    .bytesConf(ByteUnit.BYTE)
+    .createWithDefaultString("10g")
+
+  val HISTORY_SERVER_UI_PORT = ConfigBuilder("spark.history.ui.port")
+    .doc("Web UI port to bind Spark History Server")
+    .intConf
+    .createWithDefault(18080)
+
+  val FAST_IN_PROGRESS_PARSING =
+    ConfigBuilder("spark.history.fs.inProgressOptimization.enabled")
+      .doc("Enable optimized handling of in-progress logs. This option may 
leave finished " +
+        "applications that fail to rename their event logs listed as 
in-progress.")
+      .booleanConf
+      .createWithDefault(true)
+
+  val END_EVENT_REPARSE_CHUNK_SIZE =
+    ConfigBuilder("spark.history.fs.endEventReparseChunkSize")
+      .doc("How many bytes to parse at the end of log files looking for the 
end event. " +
+        "This is used to speed up generation of application listings by 
skipping unnecessary " +
+        "parts of event log files. It can be disabled by setting this config 
to 0.")
+      .bytesConf(ByteUnit.BYTE)
+      .createWithDefaultString("1m")
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/internal/config/Status.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/internal/config/Status.scala 
b/core/src/main/scala/org/apache/spark/internal/config/Status.scala
new file mode 100644
index 0000000..c561572
--- /dev/null
+++ b/core/src/main/scala/org/apache/spark/internal/config/Status.scala
@@ -0,0 +1,58 @@
+/*
+ * 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.internal.config
+
+import java.util.concurrent.TimeUnit
+
+private[spark] object Status {
+
+  val ASYNC_TRACKING_ENABLED = 
ConfigBuilder("spark.appStateStore.asyncTracking.enable")
+    .booleanConf
+    .createWithDefault(true)
+
+  val LIVE_ENTITY_UPDATE_PERIOD = ConfigBuilder("spark.ui.liveUpdate.period")
+    .timeConf(TimeUnit.NANOSECONDS)
+    .createWithDefaultString("100ms")
+
+  val MAX_RETAINED_JOBS = ConfigBuilder("spark.ui.retainedJobs")
+    .intConf
+    .createWithDefault(1000)
+
+  val MAX_RETAINED_STAGES = ConfigBuilder("spark.ui.retainedStages")
+    .intConf
+    .createWithDefault(1000)
+
+  val MAX_RETAINED_TASKS_PER_STAGE = ConfigBuilder("spark.ui.retainedTasks")
+    .intConf
+    .createWithDefault(100000)
+
+  val MAX_RETAINED_DEAD_EXECUTORS = 
ConfigBuilder("spark.ui.retainedDeadExecutors")
+    .intConf
+    .createWithDefault(100)
+
+  val MAX_RETAINED_ROOT_NODES = 
ConfigBuilder("spark.ui.dagGraph.retainedRootRDDs")
+    .intConf
+    .createWithDefault(Int.MaxValue)
+
+  val APP_STATUS_METRICS_ENABLED =
+    ConfigBuilder("spark.app.status.metrics.enabled")
+      .doc("Whether Dropwizard/Codahale metrics " +
+        "will be reported for the status of the running spark app.")
+      .booleanConf
+      .createWithDefault(false)
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala 
b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
index e2c190e..81d39e0 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable.HashMap
 import org.apache.spark._
 import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
 import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.scheduler._
 import org.apache.spark.status.api.v1
 import org.apache.spark.storage._
@@ -47,8 +48,6 @@ private[spark] class AppStatusListener(
     appStatusSource: Option[AppStatusSource] = None,
     lastUpdateTime: Option[Long] = None) extends SparkListener with Logging {
 
-  import config._
-
   private var sparkVersion = SPARK_VERSION
   private var appInfo: v1.ApplicationInfo = null
   private var appSummary = new AppSummary(0, 0)

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala 
b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala
index 3ab293d..f6a2157 100644
--- a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala
+++ b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala
@@ -22,7 +22,7 @@ import AppStatusSource.getCounter
 import com.codahale.metrics.{Counter, Gauge, MetricRegistry}
 
 import org.apache.spark.SparkConf
-import org.apache.spark.internal.config.ConfigBuilder
+import org.apache.spark.internal.config.Status.APP_STATUS_METRICS_ENABLED
 import org.apache.spark.metrics.source.Source
 
 private [spark] class JobDuration(val value: AtomicLong) extends Gauge[Long] {
@@ -71,15 +71,8 @@ private[spark] object AppStatusSource {
   }
 
   def createSource(conf: SparkConf): Option[AppStatusSource] = {
-    Option(conf.get(AppStatusSource.APP_STATUS_METRICS_ENABLED))
+    Option(conf.get(APP_STATUS_METRICS_ENABLED))
       .filter(identity)
       .map { _ => new AppStatusSource() }
   }
-
-  val APP_STATUS_METRICS_ENABLED =
-    ConfigBuilder("spark.app.status.metrics.enabled")
-      .doc("Whether Dropwizard/Codahale metrics " +
-        "will be reported for the status of the running spark app.")
-      .booleanConf
-      .createWithDefault(false)
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
----------------------------------------------------------------------
diff --git 
a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala 
b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
index 863b096..5ec7d90 100644
--- a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
+++ b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala
@@ -24,6 +24,7 @@ import scala.collection.mutable.{HashMap, ListBuffer}
 import com.google.common.util.concurrent.MoreExecutors
 
 import org.apache.spark.SparkConf
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.util.{ThreadUtils, Utils}
 import org.apache.spark.util.kvstore._
 
@@ -45,8 +46,6 @@ import org.apache.spark.util.kvstore._
  */
 private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) 
extends KVStore {
 
-  import config._
-
   private val triggers = new HashMap[Class[_], Seq[Trigger[_]]]()
   private val flushTriggers = new ListBuffer[() => Unit]()
   private val executor = if (conf.get(ASYNC_TRACKING_ENABLED)) {

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/main/scala/org/apache/spark/status/config.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/status/config.scala 
b/core/src/main/scala/org/apache/spark/status/config.scala
deleted file mode 100644
index 67801b8..0000000
--- a/core/src/main/scala/org/apache/spark/status/config.scala
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * 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.status
-
-import java.util.concurrent.TimeUnit
-
-import org.apache.spark.internal.config._
-
-private[spark] object config {
-
-  val ASYNC_TRACKING_ENABLED = 
ConfigBuilder("spark.appStateStore.asyncTracking.enable")
-    .booleanConf
-    .createWithDefault(true)
-
-  val LIVE_ENTITY_UPDATE_PERIOD = ConfigBuilder("spark.ui.liveUpdate.period")
-    .timeConf(TimeUnit.NANOSECONDS)
-    .createWithDefaultString("100ms")
-
-  val MAX_RETAINED_JOBS = ConfigBuilder("spark.ui.retainedJobs")
-    .intConf
-    .createWithDefault(1000)
-
-  val MAX_RETAINED_STAGES = ConfigBuilder("spark.ui.retainedStages")
-    .intConf
-    .createWithDefault(1000)
-
-  val MAX_RETAINED_TASKS_PER_STAGE = ConfigBuilder("spark.ui.retainedTasks")
-    .intConf
-    .createWithDefault(100000)
-
-  val MAX_RETAINED_DEAD_EXECUTORS = 
ConfigBuilder("spark.ui.retainedDeadExecutors")
-    .intConf
-    .createWithDefault(100)
-
-  val MAX_RETAINED_ROOT_NODES = 
ConfigBuilder("spark.ui.dagGraph.retainedRootRDDs")
-    .intConf
-    .createWithDefault(Int.MaxValue)
-
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala 
b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
index 0d06b02..df274d9 100644
--- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
+++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala
@@ -26,8 +26,8 @@ import scala.util.{Random, Try}
 
 import com.esotericsoftware.kryo.Kryo
 
-import org.apache.spark.deploy.history.config._
 import org.apache.spark.internal.config._
+import org.apache.spark.internal.config.History._
 import org.apache.spark.network.util.ByteUnit
 import org.apache.spark.serializer.{JavaSerializer, KryoRegistrator, 
KryoSerializer}
 import org.apache.spark.util.{ResetSystemProperties, RpcUtils}

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
index 6a761d4..87d5852 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala
@@ -39,8 +39,8 @@ import org.scalatest.Matchers
 import org.scalatest.concurrent.Eventually._
 
 import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite}
-import org.apache.spark.deploy.history.config._
 import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.History._
 import org.apache.spark.io._
 import org.apache.spark.scheduler._
 import org.apache.spark.security.GroupMappingServiceProvider

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala
index 4b1b921..341a1e2 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala
@@ -25,14 +25,13 @@ import org.mockito.Mockito._
 import org.scalatest.BeforeAndAfter
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.internal.config.History._
 import org.apache.spark.status.KVUtils
 import org.apache.spark.util.{ManualClock, Utils}
 import org.apache.spark.util.kvstore.KVStore
 
 class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter {
 
-  import config._
-
   private val MAX_USAGE = 3L
 
   private var testDir: File = _

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
index 11a2db8..7c9f8ab 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala
@@ -45,7 +45,7 @@ import org.scalatest.mockito.MockitoSugar
 import org.scalatest.selenium.WebBrowser
 
 import org.apache.spark._
-import org.apache.spark.deploy.history.config._
+import org.apache.spark.internal.config.History._
 import org.apache.spark.status.api.v1.ApplicationInfo
 import org.apache.spark.status.api.v1.JobData
 import org.apache.spark.ui.SparkUI

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala 
b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
index 0b2bbd2..bfd7306 100644
--- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
+++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala
@@ -29,6 +29,7 @@ import org.scalatest.BeforeAndAfter
 
 import org.apache.spark._
 import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.metrics.ExecutorMetricType
 import org.apache.spark.scheduler._
 import org.apache.spark.scheduler.cluster._
@@ -38,8 +39,6 @@ import org.apache.spark.util.Utils
 
 class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter {
 
-  import config._
-
   private val conf = new SparkConf()
     .set(LIVE_ENTITY_UPDATE_PERIOD, 0L)
     .set(ASYNC_TRACKING_ENABLED, false)

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
----------------------------------------------------------------------
diff --git 
a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala 
b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
index 07a7b58..a99c1ec 100644
--- 
a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala
@@ -20,12 +20,11 @@ package org.apache.spark.status
 import org.mockito.Mockito._
 
 import org.apache.spark.{SparkConf, SparkFunSuite}
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.util.kvstore._
 
 class ElementTrackingStoreSuite extends SparkFunSuite {
 
-  import config._
-
   test("tracking for multiple types") {
     val store = mock(classOf[KVStore])
     val tracking = new ElementTrackingStore(store, new SparkConf()

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
index 6044563..2945c3e 100644
--- a/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/StagePageSuite.scala
@@ -26,10 +26,10 @@ import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS}
 
 import org.apache.spark._
 import org.apache.spark.executor.TaskMetrics
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.scheduler._
 import org.apache.spark.status.AppStatusStore
 import org.apache.spark.status.api.v1.{AccumulableInfo => UIAccumulableInfo, 
StageData, StageStatus}
-import org.apache.spark.status.config._
 import org.apache.spark.ui.jobs.{ApiHelper, StagePage, StagesTab, 
TaskPagedTable}
 
 class StagePageSuite extends SparkFunSuite with LocalSparkContext {

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala 
b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
index e86cadf..b04b065 100644
--- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
+++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala
@@ -40,9 +40,9 @@ import org.apache.spark.LocalSparkContext._
 import org.apache.spark.api.java.StorageLevels
 import org.apache.spark.deploy.history.HistoryServerSuite
 import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.shuffle.FetchFailedException
 import org.apache.spark.status.api.v1.{JacksonMessageWriter, 
RDDDataDistribution, StageStatus}
-import org.apache.spark.status.config._
 
 private[spark] class SparkUICssErrorHandler extends DefaultCssErrorHandler {
 

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
----------------------------------------------------------------------
diff --git 
a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
 
b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
index 8ef1e18..6344606 100644
--- 
a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
+++ 
b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala
@@ -37,7 +37,7 @@ import org.apache.mesos.protobuf.{ByteString, 
GeneratedMessageV3}
 import org.apache.spark.{SparkConf, SparkContext, SparkException}
 import org.apache.spark.TaskState
 import org.apache.spark.internal.Logging
-import org.apache.spark.internal.config._
+import org.apache.spark.internal.config.{Status => _, _}
 import org.apache.spark.util.Utils
 
 /**

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/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 1199eec..6978ec3 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
@@ -24,12 +24,12 @@ import scala.collection.JavaConverters._
 
 import org.apache.spark.{JobExecutionStatus, SparkConf}
 import org.apache.spark.internal.Logging
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.scheduler._
 import org.apache.spark.sql.execution.SQLExecution
 import org.apache.spark.sql.execution.metric._
 import org.apache.spark.sql.internal.StaticSQLConf._
 import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity}
-import org.apache.spark.status.config._
 
 class SQLAppStatusListener(
     conf: SparkConf,

http://git-wip-us.apache.org/repos/asf/spark/blob/78fa1be2/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
index 02df45d..d79c0cf 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListenerSuite.scala
@@ -26,6 +26,7 @@ import org.json4s.jackson.JsonMethods._
 import org.apache.spark._
 import org.apache.spark.LocalSparkContext._
 import org.apache.spark.internal.config
+import org.apache.spark.internal.config.Status._
 import org.apache.spark.rdd.RDD
 import org.apache.spark.scheduler._
 import org.apache.spark.sql.{DataFrame, SparkSession}
@@ -38,7 +39,6 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, 
SQLMetrics}
 import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS
 import org.apache.spark.sql.test.SharedSQLContext
 import org.apache.spark.status.ElementTrackingStore
-import org.apache.spark.status.config._
 import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, 
LongAccumulator}
 import org.apache.spark.util.kvstore.InMemoryStore
 


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

Reply via email to