This is an automated email from the ASF dual-hosted git repository. dongjoon pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/master by this push: new f4f3b28aaac [SPARK-45261][CORE][FOLLOWUP] Avoid `transform` of conf value f4f3b28aaac is described below commit f4f3b28aaacc66692717584962f27d155558bc6e Author: Dongjoon Hyun <dh...@apple.com> AuthorDate: Thu Sep 21 17:58:04 2023 -0700 [SPARK-45261][CORE][FOLLOWUP] Avoid `transform` of conf value ### What changes were proposed in this pull request? This is a follow-up of #43038 to preserve the config value. ### Why are the changes needed? `spark.eventLog.compression.codec` allows fully-quilified class names which are case-sensitive. ### Does this PR introduce _any_ user-facing change? To preserve the existing behavior. ### How was this patch tested? Pass the CIs. Currently, `ReplayListenerSuite` is broken. ``` [info] ReplayListenerSuite: [info] - Simple replay (25 milliseconds) [info] - Replay compressed inprogress log file succeeding on partial read (35 milliseconds) [info] - Replay incompatible event log (19 milliseconds) [info] - End-to-end replay (11 seconds, 58 milliseconds) [info] - End-to-end replay with compression *** FAILED *** (29 milliseconds) [info] org.apache.spark.SparkIllegalArgumentException: [CODEC_SHORT_NAME_NOT_FOUND] Cannot find a short name for the codec org.apache.spark.io.lz4compressioncodec. ``` ### Was this patch authored or co-authored using generative AI tooling? No. Closes #43041 from dongjoon-hyun/SPARK-45261-FOLLOWUP. Authored-by: Dongjoon Hyun <dh...@apple.com> Signed-off-by: Dongjoon Hyun <dh...@apple.com> --- core/src/main/scala/org/apache/spark/SparkContext.scala | 2 +- .../scala/org/apache/spark/deploy/history/EventLogFileWriters.scala | 2 +- core/src/main/scala/org/apache/spark/internal/config/package.scala | 1 - 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 39c8e483ebd..640cfc2afaa 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -465,7 +465,7 @@ class SparkContext(config: SparkConf) extends Logging { _eventLogCodec = { val compress = _conf.get(EVENT_LOG_COMPRESS) && - !_conf.get(EVENT_LOG_COMPRESSION_CODEC).equals("none") + !_conf.get(EVENT_LOG_COMPRESSION_CODEC).equalsIgnoreCase("none") if (compress && isEventLogEnabled) { Some(_conf.get(EVENT_LOG_COMPRESSION_CODEC)).map(CompressionCodec.getShortName) } else { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala index 418d9171842..144dadf29bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/EventLogFileWriters.scala @@ -55,7 +55,7 @@ abstract class EventLogFileWriter( hadoopConf: Configuration) extends Logging { protected val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) && - !sparkConf.get(EVENT_LOG_COMPRESSION_CODEC).equals("none") + !sparkConf.get(EVENT_LOG_COMPRESSION_CODEC).equalsIgnoreCase("none") protected val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) protected val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt protected val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3da61f6c81d..05b2624b403 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -1907,7 +1907,6 @@ package object config { "the codec.") .version("3.0.0") .stringConf - .transform(_.toLowerCase(Locale.ROOT)) .createWithDefault("zstd") private[spark] val BUFFER_SIZE = --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org