-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-51896][CORE][SQL] Add Java Enum Support for TypedConfigBuilder #50691
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -59,7 +59,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft | |
| } | ||
|
|
||
| test("local mode, FIFO scheduler") { | ||
| val conf = new SparkConf().set(SCHEDULER_MODE, "FIFO") | ||
| val conf = new SparkConf().set(SCHEDULER_MODE.key, "FIFO") | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ur, is this a breaking change, @yaooqinn ?
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems that I missed this in Scala version PR. I'm curious if we can avoid this kind of programing behavior change.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi @dongjoon-hyun,set with ConfigEntry is in private scope |
||
| sc = new SparkContext("local[2]", "test", conf) | ||
| testCount() | ||
| testTake() | ||
|
|
@@ -68,7 +68,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft | |
| } | ||
|
|
||
| test("local mode, fair scheduler") { | ||
| val conf = new SparkConf().set(SCHEDULER_MODE, "FAIR") | ||
| val conf = new SparkConf().set(SCHEDULER_MODE.key, "FAIR") | ||
| val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() | ||
| conf.set(SCHEDULER_ALLOCATION_FILE, xmlPath) | ||
| sc = new SparkContext("local[2]", "test", conf) | ||
|
|
@@ -79,7 +79,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft | |
| } | ||
|
|
||
| test("cluster mode, FIFO scheduler") { | ||
| val conf = new SparkConf().set(SCHEDULER_MODE, "FIFO") | ||
| val conf = new SparkConf().set(SCHEDULER_MODE.key, "FIFO") | ||
| sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) | ||
| testCount() | ||
| testTake() | ||
|
|
@@ -88,7 +88,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft | |
| } | ||
|
|
||
| test("cluster mode, fair scheduler") { | ||
| val conf = new SparkConf().set(SCHEDULER_MODE, "FAIR") | ||
| val conf = new SparkConf().set(SCHEDULER_MODE.key, "FAIR") | ||
| val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() | ||
| conf.set(SCHEDULER_ALLOCATION_FILE, xmlPath) | ||
| sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -415,21 +415,14 @@ object SQLConf { | |
|
|
||
| private val VALID_LOG_LEVELS: Array[String] = Level.values.map(_.toString) | ||
|
|
||
| private def isValidLogLevel(level: String): Boolean = | ||
| VALID_LOG_LEVELS.contains(level.toUpperCase(Locale.ROOT)) | ||
|
|
||
| val PLAN_CHANGE_LOG_LEVEL = buildConf("spark.sql.planChangeLog.level") | ||
| .internal() | ||
| .doc("Configures the log level for logging the change from the original plan to the new " + | ||
| s"plan after a rule or batch is applied. The value can be " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .version("3.1.0") | ||
| .stringConf | ||
| .transform(_.toUpperCase(Locale.ROOT)) | ||
| .checkValue(isValidLogLevel, | ||
| "Invalid value for 'spark.sql.planChangeLog.level'. Valid values are " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .createWithDefault("trace") | ||
| .enumConf(classOf[Level]) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It's org.slf4j.event.Level
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. If slf4j is no longer used in the future, will there be a risk of introducing breaking changes?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Hi, there is no such risk since these changes are all targeting private APIs. Assuming removing slf4j does break stuff here, it will break w/ or w/o this PR, see
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Got it |
||
| .createWithDefault(Level.TRACE) | ||
|
|
||
| val PLAN_CHANGE_LOG_RULES = buildConf("spark.sql.planChangeLog.rules") | ||
| .internal() | ||
|
|
@@ -461,12 +454,8 @@ object SQLConf { | |
| "the resolved expression tree in the single-pass bottom-up Resolver. The value can be " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .version("4.0.0") | ||
| .stringConf | ||
| .transform(_.toUpperCase(Locale.ROOT)) | ||
| .checkValue(isValidLogLevel, | ||
| "Invalid value for 'spark.sql.expressionTreeChangeLog.level'. Valid values are " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .createWithDefault("trace") | ||
| .enumConf(classOf[Level]) | ||
| .createWithDefault(Level.TRACE) | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What do we get if we get this conf from PySpark (or SparkR)?
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. set/get as string is still available due to the underlying T=>str/str=>T conversion |
||
|
|
||
| val LIGHTWEIGHT_PLAN_CHANGE_VALIDATION = buildConf("spark.sql.lightweightPlanChangeValidation") | ||
| .internal() | ||
|
|
@@ -838,12 +827,8 @@ object SQLConf { | |
| .doc("Configures the log level for adaptive execution logging of plan changes. The value " + | ||
| s"can be ${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .version("3.0.0") | ||
| .stringConf | ||
| .transform(_.toUpperCase(Locale.ROOT)) | ||
| .checkValue(isValidLogLevel, | ||
| "Invalid value for 'spark.sql.adaptive.logLevel'. Valid values are " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .createWithDefault("debug") | ||
| .enumConf(classOf[Level]) | ||
| .createWithDefault(Level.DEBUG) | ||
|
|
||
| val ADVISORY_PARTITION_SIZE_IN_BYTES = | ||
| buildConf("spark.sql.adaptive.advisoryPartitionSizeInBytes") | ||
|
|
@@ -1856,10 +1841,8 @@ object SQLConf { | |
| .doc("The default storage level of `dataset.cache()`, `catalog.cacheTable()` and " + | ||
| "sql query `CACHE TABLE t`.") | ||
| .version("4.0.0") | ||
| .stringConf | ||
| .transform(_.toUpperCase(Locale.ROOT)) | ||
| .checkValues(StorageLevelMapper.values.map(_.name()).toSet) | ||
| .createWithDefault(StorageLevelMapper.MEMORY_AND_DISK.name()) | ||
| .enumConf(classOf[StorageLevelMapper]) | ||
| .createWithDefault(StorageLevelMapper.MEMORY_AND_DISK) | ||
|
|
||
| val DATAFRAME_CACHE_LOG_LEVEL = buildConf("spark.sql.dataframeCache.logLevel") | ||
| .internal() | ||
|
|
@@ -1868,12 +1851,8 @@ object SQLConf { | |
| "used for debugging purposes and not in the production environment, since it generates a " + | ||
| "large amount of logs.") | ||
| .version("4.0.0") | ||
| .stringConf | ||
| .transform(_.toUpperCase(Locale.ROOT)) | ||
| .checkValue(isValidLogLevel, | ||
| "Invalid value for 'spark.sql.dataframeCache.logLevel'. Valid values are " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .createWithDefault("trace") | ||
| .enumConf(classOf[Level]) | ||
| .createWithDefault(Level.TRACE) | ||
|
|
||
| val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") | ||
| .internal() | ||
|
|
@@ -2076,12 +2055,8 @@ object SQLConf { | |
| .doc("Configures the log level for logging of codegen. " + | ||
| s"The value can be ${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .version("4.1.0") | ||
| .stringConf | ||
| .transform(_.toUpperCase(Locale.ROOT)) | ||
| .checkValue(isValidLogLevel, | ||
| "Invalid value for 'spark.sql.codegen.logLevel'. Valid values are " + | ||
| s"${VALID_LOG_LEVELS.mkString(", ")}.") | ||
| .createWithDefault("DEBUG") | ||
| .enumConf(classOf[Level]) | ||
| .createWithDefault(Level.DEBUG) | ||
|
|
||
| val CODEGEN_LOGGING_MAX_LINES = buildConf("spark.sql.codegen.logging.maxLines") | ||
| .internal() | ||
|
|
@@ -5955,13 +5930,13 @@ class SQLConf extends Serializable with Logging with SqlApiConf { | |
|
|
||
| def optimizerInSetSwitchThreshold: Int = getConf(OPTIMIZER_INSET_SWITCH_THRESHOLD) | ||
|
|
||
| def planChangeLogLevel: Level = Level.valueOf(getConf(PLAN_CHANGE_LOG_LEVEL)) | ||
| def planChangeLogLevel: Level = getConf(PLAN_CHANGE_LOG_LEVEL) | ||
|
|
||
| def planChangeRules: Option[String] = getConf(PLAN_CHANGE_LOG_RULES) | ||
|
|
||
| def planChangeBatches: Option[String] = getConf(PLAN_CHANGE_LOG_BATCHES) | ||
|
|
||
| def expressionTreeChangeLogLevel: Level = Level.valueOf(getConf(EXPRESSION_TREE_CHANGE_LOG_LEVEL)) | ||
| def expressionTreeChangeLogLevel: Level = getConf(EXPRESSION_TREE_CHANGE_LOG_LEVEL) | ||
|
|
||
| def dynamicPartitionPruningEnabled: Boolean = getConf(DYNAMIC_PARTITION_PRUNING_ENABLED) | ||
|
|
||
|
|
@@ -6118,7 +6093,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { | |
|
|
||
| def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) | ||
|
|
||
| def adaptiveExecutionLogLevel: String = getConf(ADAPTIVE_EXECUTION_LOG_LEVEL) | ||
| def adaptiveExecutionLogLevel: Level = getConf(ADAPTIVE_EXECUTION_LOG_LEVEL) | ||
|
|
||
| def fetchShuffleBlocksInBatch: Boolean = getConf(FETCH_SHUFFLE_BLOCKS_IN_BATCH) | ||
|
|
||
|
|
@@ -6207,7 +6182,7 @@ class SQLConf extends Serializable with Logging with SqlApiConf { | |
|
|
||
| def codegenComments: Boolean = getConf(StaticSQLConf.CODEGEN_COMMENTS) | ||
|
|
||
| def codegenLogLevel: Level = Level.valueOf(getConf(CODEGEN_LOG_LEVEL)) | ||
| def codegenLogLevel: Level = getConf(CODEGEN_LOG_LEVEL) | ||
|
|
||
| def loggingMaxLinesForCodegen: Int = getConf(CODEGEN_LOGGING_MAX_LINES) | ||
|
|
||
|
|
@@ -6434,9 +6409,9 @@ class SQLConf extends Serializable with Logging with SqlApiConf { | |
| def viewSchemaCompensation: Boolean = getConf(VIEW_SCHEMA_COMPENSATION) | ||
|
|
||
| def defaultCacheStorageLevel: StorageLevel = | ||
| StorageLevel.fromString(getConf(DEFAULT_CACHE_STORAGE_LEVEL)) | ||
| StorageLevel.fromString(getConf(DEFAULT_CACHE_STORAGE_LEVEL).name()) | ||
|
|
||
| def dataframeCacheLogLevel: String = getConf(DATAFRAME_CACHE_LOG_LEVEL) | ||
| def dataframeCacheLogLevel: Level = getConf(DATAFRAME_CACHE_LOG_LEVEL) | ||
|
|
||
| def crossJoinEnabled: Boolean = getConf(SQLConf.CROSS_JOINS_ENABLED) | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
should we have an error condition for it?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ditto for the scala enum
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We can have a separate PR for this to make all the condition-less errors uniform