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 653b31e18b3f [SPARK-45771][CORE] Enable 
`spark.eventLog.rolling.enabled` by default
653b31e18b3f is described below

commit 653b31e18b3fc2546bd6b13b384459f8afddabdc
Author: Dongjoon Hyun <[email protected]>
AuthorDate: Thu Nov 2 13:14:04 2023 -0700

    [SPARK-45771][CORE] Enable `spark.eventLog.rolling.enabled` by default
    
    ### What changes were proposed in this pull request?
    
    This PR aims to enable `spark.eventLog.rolling.enabled` by default for 
Apache Spark 4.0.0.
    
    ### Why are the changes needed?
    
    Since Apache Spark 3.0.0, we have been using event log rolling not only for 
**long-running jobs**, but also for **some failed jobs** to archive the partial 
event logs incrementally.
    - https://github.com/apache/spark/pull/25670
    
    ### Does this PR introduce _any_ user-facing change?
    
    - No because `spark.eventLog.enabled` is disabled by default.
    - For the users with `spark.eventLog.enabled=true`, yes, `spark-events` 
directory will have different layouts. However, all 3.3+ `Spark History Server` 
can read both old and new event logs. I believe that the event log users are 
already using this configuration to avoid the loss of event logs for 
long-running jobs and some failed jobs.
    
    ### How was this patch tested?
    
    Pass the CIs.
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    No.
    
    Closes #43638 from dongjoon-hyun/SPARK-45771.
    
    Authored-by: Dongjoon Hyun <[email protected]>
    Signed-off-by: Dongjoon Hyun <[email protected]>
---
 core/src/main/scala/org/apache/spark/internal/config/package.scala     | 2 +-
 core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala     | 1 +
 .../org/apache/spark/deploy/history/EventLogFileWritersSuite.scala     | 2 +-
 .../scala/org/apache/spark/deploy/history/EventLogTestHelper.scala     | 1 +
 .../scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala   | 3 ++-
 docs/core-migration-guide.md                                           | 2 ++
 6 files changed, 8 insertions(+), 3 deletions(-)

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 7b0fcf3433cf..143dd0c44ce8 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
@@ -238,7 +238,7 @@ package object config {
         "each event log file to the configured size.")
       .version("3.0.0")
       .booleanConf
-      .createWithDefault(false)
+      .createWithDefault(true)
 
   private[spark] val EVENT_LOG_ROLLING_MAX_FILE_SIZE =
     ConfigBuilder("spark.eventLog.rolling.maxFileSize")
diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala 
b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
index 88f015f864de..7ebb0165e620 100644
--- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
+++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala
@@ -731,6 +731,7 @@ class SparkSubmitSuite
         "--conf", "spark.master.rest.enabled=false",
         "--conf", 
"spark.executorEnv.HADOOP_CREDSTORE_PASSWORD=secret_password",
         "--conf", "spark.eventLog.enabled=true",
+        "--conf", "spark.eventLog.rolling.enabled=false",
         "--conf", "spark.eventLog.testing=true",
         "--conf", s"spark.eventLog.dir=${testDirPath.toUri.toString}",
         "--conf", "spark.hadoop.fs.defaultFS=unsupported://example.com",
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala
 
b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala
index 455e2e18b11e..b575cbc080c0 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/EventLogFileWritersSuite.scala
@@ -66,7 +66,7 @@ abstract class EventLogFileWritersSuite extends SparkFunSuite 
with LocalSparkCon
     conf.set(EVENT_LOG_DIR, testDir.toString)
 
     // default config
-    buildWriterAndVerify(conf, classOf[SingleEventLogFileWriter])
+    buildWriterAndVerify(conf, classOf[RollingEventLogFilesWriter])
 
     conf.set(EVENT_LOG_ENABLE_ROLLING, true)
     buildWriterAndVerify(conf, classOf[RollingEventLogFilesWriter])
diff --git 
a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala 
b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala
index ea8da0108592..ac89f60955ee 100644
--- 
a/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala
+++ 
b/core/src/test/scala/org/apache/spark/deploy/history/EventLogTestHelper.scala
@@ -38,6 +38,7 @@ object EventLogTestHelper {
   def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): 
SparkConf = {
     val conf = new SparkConf
     conf.set(EVENT_LOG_ENABLED, true)
+    conf.set(EVENT_LOG_ENABLE_ROLLING, false)
     conf.set(EVENT_LOG_BLOCK_UPDATES, true)
     conf.set(EVENT_LOG_TESTING, true)
     conf.set(EVENT_LOG_DIR, logDir.toString)
diff --git 
a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
 
b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
index cd8fac2c6570..939923e12b8e 100644
--- 
a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
+++ 
b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala
@@ -33,7 +33,7 @@ import org.apache.spark.deploy.SparkHadoopUtil
 import org.apache.spark.deploy.history.{EventLogFileReader, 
SingleEventLogFileWriter}
 import org.apache.spark.deploy.history.EventLogTestHelper._
 import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics}
-import org.apache.spark.internal.config.{EVENT_LOG_COMPRESS, EVENT_LOG_DIR, 
EVENT_LOG_ENABLED}
+import org.apache.spark.internal.config.{EVENT_LOG_COMPRESS, EVENT_LOG_DIR, 
EVENT_LOG_ENABLE_ROLLING, EVENT_LOG_ENABLED}
 import org.apache.spark.io._
 import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem}
 import org.apache.spark.resource.ResourceProfile
@@ -163,6 +163,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with 
LocalSparkContext wit
   test("SPARK-31764: isBarrier should be logged in event log") {
     val conf = new SparkConf()
     conf.set(EVENT_LOG_ENABLED, true)
+    conf.set(EVENT_LOG_ENABLE_ROLLING, false)
     conf.set(EVENT_LOG_COMPRESS, false)
     conf.set(EVENT_LOG_DIR, testDirPath.toString)
     val sc = new SparkContext("local", "test-SPARK-31764", conf)
diff --git a/docs/core-migration-guide.md b/docs/core-migration-guide.md
index fb9471d0c1ae..09ba4b474e97 100644
--- a/docs/core-migration-guide.md
+++ b/docs/core-migration-guide.md
@@ -24,6 +24,8 @@ license: |
 
 ## Upgrading from Core 3.5 to 4.0
 
+- Since Spark 4.0, Spark will roll event logs to archive them incrementally. 
To restore the behavior before Spark 4.0, you can set 
`spark.eventLog.rolling.enabled` to `false`.
+
 - Since Spark 4.0, Spark will compress event logs. To restore the behavior 
before Spark 4.0, you can set `spark.eventLog.compress` to `false`.
 
 - Since Spark 4.0, `spark.shuffle.service.db.backend` is set to `ROCKSDB` by 
default which means Spark will use RocksDB store for shuffle service. To 
restore the behavior before Spark 4.0, you can set 
`spark.shuffle.service.db.backend` to `LEVELDB`.


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to