[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:
parent
c0d1bf0322
commit
78fa1be29b
|
@ -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
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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._
|
||||
|
|
|
@ -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).")
|
||||
|
|
|
@ -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")
|
||||
|
||||
}
|
|
@ -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)
|
||||
}
|
|
@ -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)
|
||||
|
|
|
@ -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)
|
||||
}
|
||||
|
|
|
@ -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)) {
|
||||
|
|
|
@ -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}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 = _
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
||||
|
|
|
@ -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
|
||||
|
||||
/**
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
||||
|
|
Loading…
Reference in a new issue