[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 <ueshin@databricks.com>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
This commit is contained in:
Takuya UESHIN 2018-11-06 09:18:17 +08:00 committed by Wenchen Fan
parent c0d1bf0322
commit 78fa1be29b
20 changed files with 30 additions and 41 deletions

View file

@ -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

View file

@ -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.

View file

@ -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._

View file

@ -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).")

View file

@ -15,14 +15,13 @@
* limitations under the License.
*/
package org.apache.spark.deploy.history
package org.apache.spark.internal.config
import java.util.concurrent.TimeUnit
import org.apache.spark.internal.config.ConfigBuilder
import org.apache.spark.network.util.ByteUnit
private[spark] object config {
private[spark] object History {
val DEFAULT_LOG_DIR = "file:/tmp/spark-events"
@ -63,5 +62,4 @@ private[spark] object config {
"parts of event log files. It can be disabled by setting this config to 0.")
.bytesConf(ByteUnit.BYTE)
.createWithDefaultString("1m")
}

View file

@ -15,13 +15,11 @@
* limitations under the License.
*/
package org.apache.spark.status
package org.apache.spark.internal.config
import java.util.concurrent.TimeUnit
import org.apache.spark.internal.config._
private[spark] object config {
private[spark] object Status {
val ASYNC_TRACKING_ENABLED = ConfigBuilder("spark.appStateStore.asyncTracking.enable")
.booleanConf
@ -51,4 +49,10 @@ private[spark] object config {
.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)
}

View file

@ -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)

View file

@ -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)
}

View file

@ -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)) {

View file

@ -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}

View file

@ -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

View file

@ -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 = _

View file

@ -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

View file

@ -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)

View file

@ -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()

View file

@ -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 {

View file

@ -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 {

View file

@ -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
/**

View file

@ -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,

View file

@ -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