You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ankuriitg <gi...@git.apache.org> on 2018/09/20 22:09:22 UTC

[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

GitHub user ankuriitg opened a pull request:

    https://github.com/apache/spark/pull/22504

    [SPARK-25118][Submit] Persist Driver Logs in Yarn Client mode to Hdfs

    ## What changes were proposed in this pull request?
    Currently, we do not have a mechanism to collect driver logs if a user chooses
    to run their application in client mode. This is a big issue as admin teams need
    to create their own mechanisms to capture driver logs.
    
    This commit adds a logger which, if enabled, adds a local log appender to the
    root logger and asynchronously syncs it to hdfs (Spark Driver Log Dir). This log
    appender writes the logs to an application specific log file on hdfs and, if
    configured, this file is persisted in the Yarn Application Dir on Spark
    Application end, allowing for easy discovery/maintenability.
    
    If YARN application dir is not available, or we are not able to write to that
    directory, the logs will still be persisted to the configurable Spark Driver Log
    Dir (Hdfs).
    
    Additionally, this collects spark-shell driver logs at INFO level by default.
    The change is that instead of setting root logger level to WARN, we will set the
    consoleAppender threshold to WARN, in case of spark-shell. This ensures that
    only WARN logs are printed on CONSOLE but other log appenders still capture INFO
    (or the default log level logs).
    
    ## How was this patch tested?
    1. Verified that logs are written to local/remote dir and then moved to Yarn
    Application Dir on application end.
    2. Added a unit test case
    3. Verified this for spark-shell, client mode and pyspark.
    4. Verified in both non-kerberos and kerberos environment
    5. Verified with following unexpected termination conditions: Ctrl + C, Driver
    OOM, Large Log Files
    6. Ran an application at WARN level and ensured that driver logs were
    captured at INFO level
    7. Started the application at WARN level, programmatically changed the level to
    INFO and ensured that logs on console were printed at INFO level

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ankuriitg/spark ankurgupta/SPARK-25118

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/22504.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #22504
    
----
commit d792809532ca648d133260c564d33e014a35c135
Author: ankurgupta <an...@...>
Date:   2018-09-20T21:22:46Z

    [SPARK-25118][Submit] Persist Driver Logs in Yarn Client mode to Hdfs
    
    Currently, we do not have a mechanism to collect driver logs if a user chooses
    to run their application in client mode. This is a big issue as admin teams need
    to create their own mechanisms to capture driver logs.
    
    This commit adds a logger which, if enabled, adds a local log appender to the
    root logger and asynchronously syncs it to hdfs (Spark Driver Log Dir). This log
    appender writes the logs to an application specific log file on hdfs and, if
    configured, this file is persisted in the Yarn Application Dir on Spark
    Application end, allowing for easy discovery/maintenability.
    
    If YARN application dir is not available, or we are not able to write to that
    directory, the logs will still be persisted to the configurable Spark Driver Log
    Dir (Hdfs).
    
    Additionally, this collects spark-shell driver logs at INFO level by default.
    The change is that instead of setting root logger level to WARN, we will set the
    consoleAppender threshold to WARN, in case of spark-shell. This ensures that
    only WARN logs are printed on CONSOLE but other log appenders still capture INFO
    (or the default log level logs).
    
    Testing Done:
    1. Verified that logs are written to local/remote dir and then moved to Yarn
    Application Dir on application end.
    2. Added a unit test case
    3. Verified this for spark-shell, client mode and pyspark.
    4. Verified in both non-kerberos and kerberos environment
    5. Verified with following unexpected termination conditions: Ctrl + C, Driver
    OOM, Large Log Files
    6. Ran an application at WARN level and ensured that driver logs were
    captured at INFO level
    7. Started the application at WARN level, programmatically changed the level to
    INFO and ensured that logs on console were printed at INFO level

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226716880
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    +    driverLogDir.foreach { dl =>
    +      val maxTime = clock.getTimeMillis() -
    +        conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +      val appDirs = driverLogFs.get.listLocatedStatus(new Path(dl))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(driverLogFs.get, appDirStatus.getPath())
    --- End diff --
    
    So you *are* doing what the current code does (I haven't reviewed the code yet). Your comment made it sound like you were just waiting longer to delete the log.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96577/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96577 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96577/testReport)** for PR 22504 at commit [`e13d6c6`](https://github.com/apache/spark/commit/e13d6c6744baafa7e79e5d37fc62d1a2fecd6f36).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Can one of the admins verify this patch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96902 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96902/testReport)** for PR 22504 at commit [`eb751b2`](https://github.com/apache/spark/commit/eb751b293f99e412727cbbd1229dcdd73dfafb54).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97561 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97561/testReport)** for PR 22504 at commit [`c6c0432`](https://github.com/apache/spark/commit/c6c04328b2072583fbd213bcefa05fb2d4b82965).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226706867
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    +    driverLogDir.foreach { dl =>
    +      val maxTime = clock.getTimeMillis() -
    +        conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +      val appDirs = driverLogFs.get.listLocatedStatus(new Path(dl))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(driverLogFs.get, appDirStatus.getPath())
    --- End diff --
    
    Isn't that just delaying the problem I described?
    
    If you look at how this code handles that for event logs, it's way more complicated than that.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226463678
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if spark.history.fs.driverlog.cleaner.enabled is true.
    +    They are cleaned if they are older than max age configured at spark.history.fs.driverlog.cleaner.maxAge.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.driver.log.syncToDfs.enabled</code></td>
    +  <td>false</td>
    +  <td>
    +    If true, spark application running in client mode will sync driver logs to a persistent storage, configured
    --- End diff --
    
    Makes sense. Updated the config name and the wording.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228669149
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -274,11 +275,20 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
           pool.scheduleWithFixedDelay(
             getRunner(() => checkForLogs()), 0, UPDATE_INTERVAL_S, TimeUnit.SECONDS)
     
    -      if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) {
    +      if (conf.get(CLEANER_ENABLED)) {
             // A task that periodically cleans event logs on disk.
             pool.scheduleWithFixedDelay(
               getRunner(() => cleanLogs()), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS)
           }
    +
    +      conf.get(DRIVER_LOG_DFS_DIR).foreach { _ =>
    --- End diff --
    
    `if (conf.contains(DRIVER_LOG_DFS_DIR) && conf.get(DRIVER_LOG_CLEANER_ENABLED))`
    
    Don't use `foreach` as a boolean check.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226395070
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    +        " to YARN application dir as no appId defined")
    +      case ace: AccessControlException =>
    +        if (!syncToDfs) {
    +          logWarning(s"Couldn't move local driver log to YARN application dir," +
    +            s" trying Spark application dir now")
    +          moveToDfs()
    +        } else {
    +          logError(s"Couldn't move driver log to YARN application dir", ace)
    +        }
    +      case e: Exception =>
    +        logError(s"Couldn't move driver log to YARN application dir", e)
    +    }
    +  }
    +
    +  private def moveToDfs(): Unit = {
    +    try {
    +      fileSystem.copyFromLocalFile(true,
    +        new Path(localLogFile),
    +        new Path(hdfsLogFile))
    +      fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +      logInfo(
    +        s"Moved the local driver log file to spark dfs at: ${hdfsLogFile}")
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not move local driver log to spark dfs", e)
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter extends Runnable with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      } finally {
    +        inStream.close()
    +        outputStream.close()
    +        streamClosed = true
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          def run(): Unit = HdfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in closing driver log input/output stream", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if ((conf.get(DRIVER_LOG_SYNCTODFS) || conf.get(DRIVER_LOG_SYNCTOYARN))
    +      && Utils.isClientMode(conf)) {
    --- End diff --
    
    still looks like the wrong indent here


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r224189470
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(appDirStatus.getPath())
    --- End diff --
    
    Added the new configurations, with fallback option to existing ones.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r227036915
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,41 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
    +    is true. Within this base directory, Spark creates a sub-directory for each application, and logs the driver
    +    logs specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any Spark
    +    user to read/write files and the Spark History Server user to delete files. Additionally, older logs from
    --- End diff --
    
    we should add something about this to the security doc with specific information on permissions, like for event logging: https://spark.apache.org/docs/latest/security.html


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226458986
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    +    driverLogDir.foreach { dl =>
    +      val maxTime = clock.getTimeMillis() -
    +        conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +      val appDirs = driverLogFs.get.listLocatedStatus(new Path(dl))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(driverLogFs.get, appDirStatus.getPath())
    --- End diff --
    
    How are you differentiating logs that are being actively written to? There's this comment at the top of this class:
    
    "some filesystems do not appear to update the `modtime` value whenever data is flushed to an open file output stream"
    
    That was added because event logs actively being written to don't have their mtime updated. There's a lot of code in this class to deal with that.
    
    In that situation you may run into issues here; either deleting a log for an active app, which would then cause lots of errors in that app's logs, or spamming the SHS logs with errors that this log cannot be deleted (not sure what's HDFS's behavior, but I believe it's the former).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97210 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97210/testReport)** for PR 22504 at commit [`7fb94fd`](https://github.com/apache/spark/commit/7fb94fd5273f19d91ec74584d2efa6b33ce371c4).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97565/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Jenkins, add to whitelist


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226492521
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    +    driverLogDir.foreach { dl =>
    +      val maxTime = clock.getTimeMillis() -
    +        conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +      val appDirs = driverLogFs.get.listLocatedStatus(new Path(dl))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(driverLogFs.get, appDirStatus.getPath())
    --- End diff --
    
    Thanks for this. I did not realize that `modtime` is not updated always. Added a fix. The fix will clean the logs when they are older than max(2 * cleaner_interval, max_age).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226397391
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private val DRIVER_LOG_DIR_DEFAULT = "/tmp/hdfs_logs"
    +
    +  override def beforeAll(): Unit = {
    +    FileUtils.forceMkdir(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  override def afterAll(): Unit = {
    +    JavaUtils.deleteRecursively(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  test("driver logs are persisted locally and synced to hdfs") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // On application end, file is moved to Hdfs (which is a local dir for this test)
    --- End diff --
    
    comment is a bit misleading (or maybe out of date) -- its synced continuously, right?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223519892
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -192,7 +211,15 @@ private[spark] object Logging {
             defaultSparkLog4jConfig = false
             LogManager.resetConfiguration()
           } else {
    -        LogManager.getRootLogger().setLevel(defaultRootLevel)
    +        val rootLogger = LogManager.getRootLogger()
    +        rootLogger.setLevel(defaultRootLevel)
    +        rootLogger.getAllAppenders().asScala.foreach { tmp =>
    --- End diff --
    
    Not really, consoleAppenderToThreshold was added later. Additionally, currently I do not store any consoleAppender which have null threshold into this map. In order for doing this way, I will need to store those as well. Dropping this, as in most scenarios, this will require more storage and minor performance improvement.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96577 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96577/testReport)** for PR 22504 at commit [`e13d6c6`](https://github.com/apache/spark/commit/e13d6c6744baafa7e79e5d37fc62d1a2fecd6f36).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98234/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    @ankuriitg if so please fix it separately; this is a big change that's not directly related


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98407 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98407/testReport)** for PR 22504 at commit [`4df08bd`](https://github.com/apache/spark/commit/4df08bd56b4cd51c4072aa026bf7f46bc574421d).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226421401
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if spark.history.fs.driverlog.cleaner.enabled is true.
    +    They are cleaned if they are older than max age configured at spark.history.fs.driverlog.cleaner.maxAge.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.driver.log.syncToDfs.enabled</code></td>
    +  <td>false</td>
    +  <td>
    +    If true, spark application running in client mode will sync driver logs to a persistent storage, configured
    --- End diff --
    
    I guess, what I meant to say here is that this feature enables syncing to dfs. We do not support syncing to local disk and that is just a implementation detail, not a feature.
    
    I am good with using any alternate wording, if that will represent the intent better.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r230463888
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -812,18 +821,74 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
           .reverse()
           .first(maxTime)
           .asScala
    +      .filter(l => l.logType == LogType.EventLogs)
           .toList
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +    val driverLogFs = new Path(driverLogDir).getFileSystem(hadoopConf)
    +    val currentTime = clock.getTimeMillis()
    +    val maxTime = currentTime - conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +    val logFiles = driverLogFs.listLocatedStatus(new Path(driverLogDir))
    +    while (logFiles.hasNext()) {
    +      val f = logFiles.next()
    +      // Do not rely on 'modtime' as it is not updated for all filesystems when files are written to
    +      val deleteFile =
    +        try {
    +          val info = listing.read(classOf[LogInfo], f.getPath().toString())
    +          // Update the lastprocessedtime of file if it's length or modification time has changed
    +          if (info.fileSize < f.getLen() || info.lastProcessed < f.getModificationTime()) {
    +            listing.write(
    +              info.copy(lastProcessed = currentTime, fileSize = f.getLen()))
    +            false
    +          } else if (info.lastProcessed > maxTime) {
    +            false
    +          } else {
    +            true
    +          }
    +        } catch {
    +          case e: NoSuchElementException =>
    +            // For every new driver log file discovered, create a new entry in listing
    +            listing.write(LogInfo(f.getPath().toString(), currentTime, LogType.DriverLogs, None,
    +              None, f.getLen()))
    +          false
    +        }
    +      if (deleteFile) {
    +        logInfo(s"Deleting expired driver log for: ${f.getPath().getName()}")
    +        listing.delete(classOf[LogInfo], f.getPath().toString())
    +        deleteLog(driverLogFs, f.getPath())
    +      }
    +    }
    +
    +    // Delete driver log file entries that exceed the configured max age and
    +    // may have been deleted on filesystem externally.
    +    val stale = listing.view(classOf[LogInfo])
    +      .index("lastProcessed")
    +      .reverse()
    +      .first(maxTime)
    +      .asScala
    +      .filter(i => i.logType == LogType.DriverLogs)
    --- End diff --
    
    `.filter { i => ... }`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    while I see this as being a nice feature, I'm a bit hesitant of the yarn logs part of this. You are essentially duplicating the yarn nodemanager logic and assuming you know what its doing.   
    If you don't get permissions, format, compression - TFILE format, etc correct then its not going to be readable.
    
    For your use case how are you expecting to get the logs?  just via yarn logs?
    
    Did you look at just using the hadoop log aggregation classes directly?
    
    Putting into some random hdfs directory is obviously much easier the only thing there is user or someone has to make sure they cleanup.
    
    Note I haven't looked at the code in detail yet.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220013308
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -143,13 +143,28 @@ trait Logging {
             // overriding the root logger's config if they're different.
             val replLogger = LogManager.getLogger(logName)
             val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN)
    +        // Update the consoleAppender threshold to replLevel
             if (replLevel != rootLogger.getEffectiveLevel()) {
               if (!silent) {
    -            System.err.printf("Setting default log level to \"%s\".\n", replLevel)
    +            System.err.printf("Setting console log level to \"%s\".\n", replLevel)
    --- End diff --
    
    Hmm... with the change here this message now can be a little confusing.
    
    Because while this is correct, the instructions printed below will actually change the "default" (actually the root logger) level, not just the console appender's level.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Sure, let me do that


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97561 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97561/testReport)** for PR 22504 at commit [`c6c0432`](https://github.com/apache/spark/commit/c6c04328b2072583fbd213bcefa05fb2d4b82965).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226461965
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    --- End diff --
    
    Use the nicer closure syntax.
    
    ```
    blah {
      do stuff
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by bersprockets <gi...@git.apache.org>.
Github user bersprockets commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223844575
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    --- End diff --
    
    This doesn't make sense anymore since you do not instantiate the logger when it's false.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97210/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226421974
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    private def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          override def run(): Unit = DfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in shutting down threadpool", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if (conf.get(DRIVER_LOG_SYNCTODFS)
    +        && conf.get(DRIVER_LOG_DFS_DIR).isDefined
    +      && Utils.isClientMode(conf)) {
    +      Try[DriverLogger] { new DriverLogger(conf) }
    +        .recoverWith {
    +          case t: Throwable =>
    +            logError("Could not add driver logger", t)
    +            Failure(t)
    +        }.toOption
    --- End diff --
    
    I will leave it as it is for now unless others also feel the same way, as I came to this implementation based on review comments.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r230467097
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,41 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
    +    is true. Within this base directory, Spark creates a sub-directory for each application, and logs the driver
    +    logs specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any Spark
    +    user to read/write files and the Spark History Server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if <code>spark.history.fs.driverlog.cleaner.enabled</code>
    +    is true or if not configured, falling back to <code>spark.history.fs.cleaner.enabled</code>. They are cleaned
    +    if they are older than max age configured at <code>spark.history.fs.driverlog.cleaner.maxAge</code> or if not
    +    configured, falling back to <code>spark.history.fs.cleaner.maxAge</code>.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.driver.log.persistToDfs.enabled</code></td>
    +  <td>false</td>
    +  <td>
    +    If true, spark application running in client mode will write driver logs to a persistent storage, configured
    +    in <code>spark.driver.log.dfsDir</code>. If <code>spark.driver.log.dfsDir</code> is not configured, driver logs
    +    will not be persisted. Additionally, enable the cleaner by setting <code>spark.history.fs.driverlog.cleaner.enabled</code>
    --- End diff --
    
    Instead of mentioning the cleaner config, I'd add a link to the SHS config page. That makes it clear that the cleaner is not part of the application.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Got it, I didn't test that. Let me verify that and explore alternate options.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226456519
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -117,6 +118,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
       // Visible for testing
       private[history] val fs: FileSystem = new Path(logDir).getFileSystem(hadoopConf)
     
    +  private val driverLogFs: Option[FileSystem] =
    +    if (conf.get(DRIVER_LOG_DFS_DIR).isDefined) {
    +      Some(FileSystem.get(hadoopConf))
    --- End diff --
    
    This is not right. It assumes the directory defined is in the defaultFS. See call right above this one for what you should do here.
    
    I also don't see a good reason to keep this in a field. Just get the FS reference when the log cleaner runs.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226739229
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -798,14 +815,52 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +    val currentTime = clock.getTimeMillis()
    +    val maxTime = currentTime - conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +    val appDirs = driverLogFs.get.listLocatedStatus(new Path(driverLogDir))
    +    while (appDirs.hasNext()) {
    +      val appDirStatus = appDirs.next()
    +      if (appDirStatus.isDirectory()) {
    +        val logFiles = driverLogFs.get.listStatus(appDirStatus.getPath())
    --- End diff --
    
    Not sure how I missed this before, but why are you storing log files in a subdirectory? What's the advantage? You'll have a ton of directories with a single log file. Instead, just write the log files directly into the parent directory... that also makes this code simpler / faster (less RPCs).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98570 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98570/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97565 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97565/testReport)** for PR 22504 at commit [`a0a85b3`](https://github.com/apache/spark/commit/a0a85b3eff0f115b983ce5ba3214e09f8ee90dd2).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220016032
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    --- End diff --
    
    Move code after `=>` to next line. Where is this exception thrown btw? The log message is very specific...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226457468
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    --- End diff --
    
    If you're calling this you know the config is set. So you can avoid the extra indentation by just calling `.get`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98564 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98564/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220015864
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    --- End diff --
    
    `val`
    
    I don't know if `getCurrentUser` will ever be null. But if you're worried I'd just log something and do nothing here. The behavior of your code in that case does not look very appropriate (writing to "/yarn-root-dir/logs/blah") and would probably cause an exception anyway.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225311218
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -281,6 +288,16 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
             pool.scheduleWithFixedDelay(
               getRunner(() => cleanLogs()), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS)
           }
    +
    +      driverLogFs.foreach { _ =>
    +        if (conf.get(DRIVER_LOG_CLEANER_ENABLED).getOrElse(
    +            conf.getBoolean("spark.history.fs.cleaner.enabled", false))) {
    +          pool.scheduleWithFixedDelay(getRunner(() => cleanDriverLogs()),
    +            0,
    +            conf.get(DRIVER_LOG_CLEANER_INTERVAL).getOrElse(CLEAN_INTERVAL_S),
    --- End diff --
    
    Create config constant for existing config, use `fallbackConf`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    cc: @vanzin @squito 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97566 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97566/testReport)** for PR 22504 at commit [`65c0800`](https://github.com/apache/spark/commit/65c080032bbea82024f1ab14cb43c771f6157fc4).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97560 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97560/testReport)** for PR 22504 at commit [`07d2df8`](https://github.com/apache/spark/commit/07d2df87fddd540637b054b643eb5484c5e58eaf).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226737882
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -274,11 +282,20 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
           pool.scheduleWithFixedDelay(
             getRunner(() => checkForLogs()), 0, UPDATE_INTERVAL_S, TimeUnit.SECONDS)
     
    -      if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) {
    +      if (conf.get(CLEANER_ENABLED)) {
             // A task that periodically cleans event logs on disk.
             pool.scheduleWithFixedDelay(
               getRunner(() => cleanLogs()), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS)
           }
    +
    +      driverLogFs.foreach { _ =>
    --- End diff --
    
    Here it's clearer to use `if (driverLogFs.isDefined && ...)`.
    
    But really this is a bit confusing and I think my previous suggestion is clearer: don't keep `driverLogFs` in a field, just get it when running the cleaner task. And do this initialization based on the config entries, not `driverLogFs`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226492665
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    --- End diff --
    
    I did not understand this comment.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97624 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97624/testReport)** for PR 22504 at commit [`0bd33f6`](https://github.com/apache/spark/commit/0bd33f6b2fccc45fcd99de735b8a25465aedb325).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r231346117
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,40 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
    +    is true. Within this base directory, each application logs the driver logs to an application specific file.
    +    Users may want to set this to a unified location like an HDFS directory so driver log files can be persisted
    +    for later usage. This directory should allow any Spark user to read/write files and the Spark History Server
    +    user to delete files. Additionally, older logs from this directory are cleaned by
    --- End diff --
    
    ...cleaned by the...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97177 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97177/testReport)** for PR 22504 at commit [`3b9fc9f`](https://github.com/apache/spark/commit/3b9fc9f9beeb31c701850c431755be17bf5c1648).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223866624
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -48,6 +48,19 @@ package object config {
         .bytesConf(ByteUnit.MiB)
         .createOptional
     
    +  private[spark] val DRIVER_LOG_DFS_DIR =
    +    ConfigBuilder("spark.driver.log.dfsDir").stringConf.createOptional
    +
    +  private[spark] val DRIVER_LOG_LAYOUT =
    +    ConfigBuilder("spark.driver.log.layout")
    +      .stringConf
    +      .createWithDefault("%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n")
    +
    +  private[spark] val DRIVER_LOG_SYNCTODFS =
    --- End diff --
    
    Let's leave it. But there's some usage of the enabled flag in your code that you can still clean up.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98799 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98799/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97414 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97414/testReport)** for PR 22504 at commit [`a46734c`](https://github.com/apache/spark/commit/a46734c46355b31863178ff9f0169118ca15a695).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98799 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98799/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226468029
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, File, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private var rootHdfsDir : File = _
    +
    +  override def beforeAll(): Unit = {
    +    rootHdfsDir = Utils.createTempDir(namePrefix = "hdfs_logs")
    +  }
    +
    +  override def afterAll(): Unit = {
    +    JavaUtils.deleteRecursively(rootHdfsDir)
    +  }
    +
    +  test("driver logs are persisted locally and synced to hdfs") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // File is continuously synced to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    +    assert(hdfsFiles.length > 0)
    +    val driverLogFile = hdfsFiles.filter(f => f.getName.equals("driver.log")).head
    +    val hdfsIS = new BufferedInputStream(new FileInputStream(driverLogFile))
    +    assert(hdfsIS.available() > 0)
    +    JavaUtils.deleteRecursively(hdfsDir)
    --- End diff --
    
    This line and the next are unnecessary.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220016885
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    +        " to YARN application dir as no appId defined")
    +      case ace: AccessControlException =>
    +        if (!syncToDfs) {
    +          logWarning(s"Couldn't move local driver log to YARN application dir," +
    +            s" trying Spark application dir now")
    +          moveToDfs()
    +        } else {
    +          logError(s"Couldn't move driver log to YARN application dir", ace)
    +        }
    +      case e: Exception =>
    +        logError(s"Couldn't move driver log to YARN application dir", e)
    +    }
    +  }
    +
    +  private def moveToDfs(): Unit = {
    +    try {
    +      fileSystem.copyFromLocalFile(true,
    +        new Path(localLogFile),
    +        new Path(hdfsLogFile))
    +      fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +      logInfo(
    +        s"Moved the local driver log file to spark dfs at: ${hdfsLogFile}")
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not move local driver log to spark dfs", e)
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter extends Runnable with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      } finally {
    +        inStream.close()
    +        outputStream.close()
    +        streamClosed = true
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          def run(): Unit = HdfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in closing driver log input/output stream", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if ((conf.get(DRIVER_LOG_SYNCTODFS) || conf.get(DRIVER_LOG_SYNCTOYARN))
    +      && Utils.isClientMode(conf)) {
    +      Try[DriverLogger] { new DriverLogger(conf) }
    +        .recoverWith { case t: Throwable => logError("Could not add driver logger", t); Failure(t) }
    --- End diff --
    
    Break into multiple lines instead of using `;`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97557 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97557/testReport)** for PR 22504 at commit [`18b823f`](https://github.com/apache/spark/commit/18b823fd38d93d45c9121b9802bb9206755265bd).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98401/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98413 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98413/testReport)** for PR 22504 at commit [`70a227f`](https://github.com/apache/spark/commit/70a227fd83524f35264fac9717d07024f440d179).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97617 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97617/testReport)** for PR 22504 at commit [`a3b6550`](https://github.com/apache/spark/commit/a3b6550704031c2717364f582190be9661c4c934).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225312559
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,195 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  // Visible for testing
    +  private[spark] var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get.split(",").head
    --- End diff --
    
    Why are you calling `split` here?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Thanks for your comment @tgravescs 
    
    Moving the file to yarn logs app dir is optional and users can choose to disable that. In my case, If they only enable syncToHdfs, the file will be moved to the configured hdfs dir.
    
    Regarding cleaning, I have another PR ready, which adds to the existing logic of Spark History Server cleaner and cleans these driver logs as well. It will use the existing max age and polling interval configured for event logs. (I can merge that code in this PR as well)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    @srowen - the bug is causing a unit-test failure on this PR. How should I resolve it?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97934 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97934/testReport)** for PR 22504 at commit [`a620bcc`](https://github.com/apache/spark/commit/a620bcc947111ad9530841fd5de20c86c477226e).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97177 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97177/testReport)** for PR 22504 at commit [`3b9fc9f`](https://github.com/apache/spark/commit/3b9fc9f9beeb31c701850c431755be17bf5c1648).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96729/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98564 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98564/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226395358
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    private def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          override def run(): Unit = DfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in shutting down threadpool", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if (conf.get(DRIVER_LOG_SYNCTODFS)
    +        && conf.get(DRIVER_LOG_DFS_DIR).isDefined
    +      && Utils.isClientMode(conf)) {
    +      Try[DriverLogger] { new DriverLogger(conf) }
    +        .recoverWith {
    +          case t: Throwable =>
    +            logError("Could not add driver logger", t)
    +            Failure(t)
    +        }.toOption
    --- End diff --
    
    probably simpler as a regular `try / catch`
    
    ```scala
    try {
      Some(new DriverLogger(conf))
    }  catch {
      case t: Throwable =>
        logError("Could not add driver logger", t)
        None
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223840375
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    --- End diff --
    
    Config constant. `driverLogDir` is a better name.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223849163
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private var hdfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key, "/tmp/driver_logs").split(",").head
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    --- End diff --
    
    If this throws you'll leave the stream opened.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226400385
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if spark.history.fs.driverlog.cleaner.enabled is true.
    +    They are cleaned if they are older than max age configured at spark.history.fs.driverlog.cleaner.maxAge.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.driver.log.syncToDfs.enabled</code></td>
    +  <td>false</td>
    +  <td>
    +    If true, spark application running in client mode will sync driver logs to a persistent storage, configured
    --- End diff --
    
    it seems like there is a mismatch between what is described here and what is implemented.  Do you intend to support a configuration where you are only logging to local disk, and not syncing to dfs?  It doesn't seem like `DriverLogger.apply` allows that.  And if you do intend to support that, and want it controlled by this configuration, then I'd remove the "syncToDfs" from this name.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Yes, I will try to reproduce it locally


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225313921
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,195 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  // Visible for testing
    +  private[spark] var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get.split(",").head
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          override def run(): Unit = DfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in shutting down threadpool", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if (conf.get(DRIVER_LOG_SYNCTODFS)
    +      && conf.get(DRIVER_LOG_DFS_DIR).isDefined
    --- End diff --
    
    Indent these continuation lines one extra level.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    its good its optional, but I think if we say we support it internally to Spark we need to make sure its done properly and works for most yarn setups. 
    
    Did you look at using the hadoop classes at all?  
    
    Just skimming the code it looks like you just move it directly to the hadoop log dir which definitely won't work in our environment.  Does it work in yours?  The file needs to be named node_port, and it  has to be in TFILE format. If that format is not right yarn logs will stop reading once it hits that bad file.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226457291
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    +    driverLogDir.foreach { dl =>
    +      val maxTime = clock.getTimeMillis() -
    +        conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    --- End diff --
    
    Fits in previous line.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223853075
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite {
    +
    +  test("driver logs are persisted") {
    +    val sc = getSparkContext()
    +    // Wait for application to start
    +    Thread.sleep(1000)
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // On application end, file is moved to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile("/tmp/hdfs_logs/", app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    +    assert(hdfsFiles.length > 0)
    +    JavaUtils.deleteRecursively(hdfsDir)
    +    assert(!hdfsDir.exists())
    +  }
    +
    +  test("driver logs are synced to hdfs continuously") {
    +    val sc = getSparkContext()
    +    // Wait for application to start
    +    Thread.sleep(1000)
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +    for (i <- 1 to 1000) {
    +      logInfo("Log enough data to log file so that it can be flushed")
    +    }
    +
    +    // After 5 secs, file contents are synced to Hdfs (which is a local dir for this test)
    +    Thread.sleep(6000)
    --- End diff --
    
    I'm not a fan of code that relies on exact timing between threads for things to work.
    
    This would be better if you drove the `HdfsAsyncWriter` class manually from the test, instead of indirectly through a `SparkContext`. The you can also control the log file flushing explicitly instead of the hack you have above.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223902134
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private var hdfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key, "/tmp/driver_logs").split(",").head
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    --- End diff --
    
    It's good practice to write idempotent close methods. "Should not be called twice" is part of the famous last words list.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220016445
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    --- End diff --
    
    Will this parent dir already exist at the time this code runs?
    
    I'd be more comfortable with asserting that it does exist, in which case it would have been created by YARN itself, than to automatically create it by calling one of the `FileSystem` methods later in this method...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96958 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96958/testReport)** for PR 22504 at commit [`ead6fac`](https://github.com/apache/spark/commit/ead6face05f73b277bf856158e3dffe27a01c58f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97211/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98799/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223848006
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    --- End diff --
    
    Slight preference to use:
    
    ```
    new File(Utils.createTempDir(root = Utils.getLocalDir(conf), namePrefix = "driver_log"), DRIVER_LOG_FILE)
    ```
    
    That also means Spark takes care of deleting the directory for you later.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97575 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97575/testReport)** for PR 22504 at commit [`6e14137`](https://github.com/apache/spark/commit/6e141373910df8c159429c367a8e937ad71eccbe).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226468828
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, File, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private var rootHdfsDir : File = _
    +
    +  override def beforeAll(): Unit = {
    +    rootHdfsDir = Utils.createTempDir(namePrefix = "hdfs_logs")
    +  }
    +
    +  override def afterAll(): Unit = {
    +    JavaUtils.deleteRecursively(rootHdfsDir)
    +  }
    +
    +  test("driver logs are persisted locally and synced to hdfs") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // File is continuously synced to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    --- End diff --
    
    All the checks below are basically:
    
    ```
    hdfsFiles.exists { f => f.getName() == DRIVER_LOG_FILE && f.length() > 0 }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97617/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226737371
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -116,6 +117,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
       // Visible for testing
       private[history] val fs: FileSystem = new Path(logDir).getFileSystem(hadoopConf)
     
    +  private val driverLogFs: Option[FileSystem] =
    +    if (conf.get(DRIVER_LOG_DFS_DIR).isDefined) {
    --- End diff --
    
    Use `conf.get(DRIVER_LOG_DFS_DIR).map`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r230464353
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -1033,6 +1102,7 @@ private[history] case class FsHistoryProviderMetadata(
     private[history] case class LogInfo(
         @KVIndexParam logPath: String,
         @KVIndexParam("lastProcessed") lastProcessed: Long,
    +    logType: LogType.Value,
    --- End diff --
    
    What happens here when you have an existing listing database where this field is not recorded?
    
    Depending on what happens, you may need a default value here, or maybe handling nulls somewhere else, or even changing the DB version so old data is invalidated.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223851626
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite {
    +
    +  test("driver logs are persisted") {
    +    val sc = getSparkContext()
    +    // Wait for application to start
    +    Thread.sleep(1000)
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // On application end, file is moved to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile("/tmp/hdfs_logs/", app_id)
    --- End diff --
    
    You should create a constant for `/tmp/hdfs_logs/` or read it from the SparkConf object.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223848348
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    --- End diff --
    
    This is not needed if you follow my previous suggestion, but there's `Utils.tryLogNonFatalError`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97177/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223849768
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private var hdfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key, "/tmp/driver_logs").split(",").head
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          def run(): Unit = HdfsAsyncWriter.this.close()
    --- End diff --
    
    override


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r231346507
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,40 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
    +    is true. Within this base directory, each application logs the driver logs to an application specific file.
    +    Users may want to set this to a unified location like an HDFS directory so driver log files can be persisted
    +    for later usage. This directory should allow any Spark user to read/write files and the Spark History Server
    +    user to delete files. Additionally, older logs from this directory are cleaned by
    +    <a href="monitoring.html#spark-history-server-configuration-options"> Spark History Server</a>  if
    +    <code>spark.history.fs.driverlog.cleaner.enabled</code> is true and, if they are older than max age configured
    +    at <code>spark.history.fs.driverlog.cleaner.maxAge</code>.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.driver.log.persistToDfs.enabled</code></td>
    +  <td>false</td>
    +  <td>
    +    If true, spark application running in client mode will write driver logs to a persistent storage, configured
    +    in <code>spark.driver.log.dfsDir</code>. If <code>spark.driver.log.dfsDir</code> is not configured, driver logs
    +    will not be persisted. Additionally, enable the cleaner by setting <code>spark.history.fs.driverlog.cleaner.enabled</code>
    +    to true in <a href="monitoring.html#spark-history-server-configuration-options"> Spark History Server</a>.
    +  </td>
    +</tr>
    +<tr>
    +  <td><code>spark.driver.log.layout</code></td>
    +  <td>%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n</td>
    +  <td>
    +    The layout for the driver logs that are synced to <code>spark.driver.log.dfsDir</code>. If 
    +    <code>spark.driver.log.persistToDfs.enabled</code> is true and this configuration is used. If this is not configured,
    --- End diff --
    
    No need to mention the `enabled` option here.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97624/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226464013
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,202 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not persist driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +      with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to persist driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed writing driver logs to dfs", e)
    +      }
    +    }
    +
    +    private def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          override def run(): Unit = DfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in shutting down threadpool", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if (conf.get(DRIVER_LOG_PERSISTTODFS) && Utils.isClientMode(conf)) {
    +      if (conf.get(DRIVER_LOG_DFS_DIR).isDefined) {
    +        Try[DriverLogger] {
    --- End diff --
    
    This is actually simpler with a try..catch. There might have been a reason for this before but in it's current shape, a try..catch is simpler and actually easier to understand.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98564/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225310694
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -117,6 +117,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
       // Visible for testing
       private[history] val fs: FileSystem = new Path(logDir).getFileSystem(hadoopConf)
     
    +  private[history] val driverLogFs: Option[FileSystem] =
    --- End diff --
    
    Just `private`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98413/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226454499
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    private def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          override def run(): Unit = DfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in shutting down threadpool", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if (conf.get(DRIVER_LOG_SYNCTODFS)
    +        && conf.get(DRIVER_LOG_DFS_DIR).isDefined
    --- End diff --
    
    do you want to at least warn the user if sync_to_dfs is enabled but log_dfs_dir is undefined?  Or is there a useful scenario when you'd want that?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226492916
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if spark.history.fs.driverlog.cleaner.enabled is true.
    --- End diff --
    
    I provided the information that driverlog cleaner uses a fallback option in its own documentation.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97211 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97211/testReport)** for PR 22504 at commit [`2bbf137`](https://github.com/apache/spark/commit/2bbf1370ab72a0183cc3ccdc04851b2b6dadf5dc).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226463571
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,202 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not persist driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +      with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to persist driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed writing driver logs to dfs", e)
    +      }
    +    }
    +
    +    private def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    --- End diff --
    
    Why catch throwable?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220015442
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    --- End diff --
    
    Fits in previous line.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223843974
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -48,6 +48,19 @@ package object config {
         .bytesConf(ByteUnit.MiB)
         .createOptional
     
    +  private[spark] val DRIVER_LOG_DFS_DIR =
    +    ConfigBuilder("spark.driver.log.dfsDir").stringConf.createOptional
    +
    +  private[spark] val DRIVER_LOG_LAYOUT =
    +    ConfigBuilder("spark.driver.log.layout")
    +      .stringConf
    +      .createWithDefault("%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n")
    +
    +  private[spark] val DRIVER_LOG_SYNCTODFS =
    --- End diff --
    
    With the removal of the YARN code, does it even make sense to have this option? Seems like setting `DRIVER_LOG_DFS_DIR` would be enough to enable this.
    
    Only advantage of it is that it makes it simpler to disable the logger if the default config file has it enabled.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220014994
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -192,7 +207,16 @@ private[spark] object Logging {
             defaultSparkLog4jConfig = false
             LogManager.resetConfiguration()
           } else {
    -        LogManager.getRootLogger().setLevel(defaultRootLevel)
    +        val rootLogger = LogManager.getRootLogger()
    +        rootLogger.setLevel(defaultRootLevel)
    +        val appenders = rootLogger.getAllAppenders
    --- End diff --
    
    `...getAllAppenders().asScala.foreach`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r231346593
  
    --- Diff: docs/monitoring.md ---
    @@ -202,6 +202,28 @@ Security options for the Spark History Server are covered more detail in the
           applications that fail to rename their event logs listed as in-progress.
         </td>
       </tr>
    +  <tr>
    +    <td>spark.history.fs.driverlog.cleaner.enabled</td>
    +    <td><code>spark.history.fs.cleaner.enabled</code></td>
    +    <td>
    +      Specifies whether the History Server should periodically clean up driver logs from storage.
    +    </td>
    +  </tr>
    +  <tr>
    +    <td>spark.history.fs.driverlog.cleaner.interval</td>
    +    <td><code>spark.history.fs.cleaner.interval</code></td>
    +    <td>
    +      How often the filesystem driver log history cleaner checks for files to delete.
    --- End diff --
    
    driver log cleaner


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226464199
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, File, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private var rootHdfsDir : File = _
    --- End diff --
    
    s/hdfs/dfs in all this file.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by tgravescs <gi...@git.apache.org>.
Github user tgravescs commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    to clarify what I mean "by work", I mean in order for the "yarn logs" command to read it properly.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97210 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97210/testReport)** for PR 22504 at commit [`7fb94fd`](https://github.com/apache/spark/commit/7fb94fd5273f19d91ec74584d2efa6b33ce371c4).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223842304
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(appDirStatus.getPath())
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => // no-op
    +      case t: Throwable => logError("Failed to delete driver logs", t)
    --- End diff --
    
    Is there a good reason to catch `Throwable` instead of `Exception`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96956 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96956/testReport)** for PR 22504 at commit [`522fcea`](https://github.com/apache/spark/commit/522fcea207bdccf19b29ffc3ae132da01d254ee0).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223846450
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    --- End diff --
    
    I don't think this logic is right. It should be:
    
    - if the user has defined the config, use it
    - otherwise default to the first appender's layout
    - if that is not set, then use a hardcoded default
    
    The first two are inverted in your case. You could achieve that with `.createWithDefaultFunction` in the config builder. But really I'd just make it an optional config, and here do:
    
    - if the user has defined the config, use it
    - otherwise default to the first appender's layout
    - if that is not set, then use `new PatternLayout()`.
    
    Which I think is what I suggested before.



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    known test failure addressed here: https://github.com/apache/spark/pull/22808


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96717 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96717/testReport)** for PR 22504 at commit [`e13d6c6`](https://github.com/apache/spark/commit/e13d6c6744baafa7e79e5d37fc62d1a2fecd6f36).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97211 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97211/testReport)** for PR 22504 at commit [`2bbf137`](https://github.com/apache/spark/commit/2bbf1370ab72a0183cc3ccdc04851b2b6dadf5dc).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223850112
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    --- End diff --
    
    s/HDFS/DFS in these names.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97566/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96384 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96384/testReport)** for PR 22504 at commit [`d792809`](https://github.com/apache/spark/commit/d792809532ca648d133260c564d33e014a35c135).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226463522
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    private def close(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } finally {
    +        try {
    +          streamClosed = true
    +          inStream.close()
    +          outputStream.close()
    +        } catch {
    +          case t: Throwable =>
    +            logError("Error in closing driver log input/output stream", t)
    +        }
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          override def run(): Unit = DfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in shutting down threadpool", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if (conf.get(DRIVER_LOG_SYNCTODFS)
    +        && conf.get(DRIVER_LOG_DFS_DIR).isDefined
    --- End diff --
    
    Sure, added a warning message in that case


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226460844
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    --- End diff --
    
    This is not what I suggested and your way is actually very confusing for the user; especially since you say this is the default in the configuration, whereas in 99.9% of the cases it won't be, since the user will have at least one appender in the log4j conf.
    
    If you want this config to have a default value, then you have to use it; it should have higher priority than the first appender's layout.
    
    What I actually suggested was for this config to *not* have a default. That solves the confusion; because if you set it, it's used, otherwise, it defaults to the first appender's layout (or the default log4j layout).
    
    You could probably do that with `createWithDefaultFunction`, but `createOptional` + some code here also works.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226740562
  
    --- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
    @@ -2863,6 +2870,14 @@ private[spark] object Utils extends Logging {
       def stringHalfWidth(str: String): Int = {
         if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size
       }
    +
    +  private[spark] def sanitizeDirName(str: String): String = {
    --- End diff --
    
    This class is already `private[spark]` so the modifier is redundant.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96958/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225315220
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private val DRIVER_LOG_DIR_DEFAULT = "/tmp/hdfs_logs"
    +
    +  override def beforeAll(): Unit = {
    +    FileUtils.forceMkdir(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  override def afterAll(): Unit = {
    +    JavaUtils.deleteRecursively(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  test("driver logs are persisted") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // On application end, file is moved to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    +    assert(hdfsFiles.length > 0)
    +    JavaUtils.deleteRecursively(hdfsDir)
    +    assert(!hdfsDir.exists())
    +  }
    +
    +  test("driver logs are synced to hdfs continuously") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +    for (i <- 1 to 1000) {
    +      logInfo("Log enough data to log file so that it can be flushed")
    +    }
    +
    +    // Sync the driver logs manually instead of waiting for scheduler
    +    sc._driverLogger.foreach(_.writer.foreach(_.run()))
    +    val hdfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    +    assert(hdfsFiles.length > 0)
    +    val driverLogFile = hdfsFiles.filter(f => f.getName.equals("driver.log")).head
    +    val hdfsIS = new BufferedInputStream(new FileInputStream(driverLogFile))
    +    assert(hdfsIS.available() > 0)
    +
    +    sc.stop()
    +    // Ensure that the local file is deleted on application end
    +    assert(!driverLogsDir.exists())
    +    JavaUtils.deleteRecursively(hdfsDir)
    +    assert(!hdfsDir.exists())
    +  }
    +
    +  private def getSparkContext(): SparkContext = {
    +    val conf = new SparkConf()
    +    conf.set("spark.local.dir", "/tmp")
    --- End diff --
    
    Don't do this. Tests run with a pre-configured temp directory, and should not write to `/tmp`.
    
    If you need to create a temp directory, do so explicitly using the existing `Utils` API.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98407 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98407/testReport)** for PR 22504 at commit [`4df08bd`](https://github.com/apache/spark/commit/4df08bd56b4cd51c4072aa026bf7f46bc574421d).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merging to master.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98805 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98805/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226464800
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,202 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    --- End diff --
    
    I'd really be more comfortable with using a temp dir rather than a hardcoded directory name...
    
    Or at least use `__driver_logs__` since that syntax conventionally means things that user applications should not use.
    
    And use a constant so you don't have to hardcode the value in the test.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226462792
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +      with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    --- End diff --
    
    "spark dfs"?
    
    Just use a simpler message: "Started driver log file synchronization to $blah."


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97557 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97557/testReport)** for PR 22504 at commit [`18b823f`](https://github.com/apache/spark/commit/18b823fd38d93d45c9121b9802bb9206755265bd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220012951
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -2421,11 +2425,13 @@ class SparkContext(config: SparkConf) extends Logging {
         // the cluster manager to get an application ID (in case the cluster manager provides one).
         listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId),
           startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls))
    +    _driverLogger.map(_.startSync(_hadoopConfiguration))
    --- End diff --
    
    foreach


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220016802
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    +        " to YARN application dir as no appId defined")
    +      case ace: AccessControlException =>
    +        if (!syncToDfs) {
    +          logWarning(s"Couldn't move local driver log to YARN application dir," +
    +            s" trying Spark application dir now")
    +          moveToDfs()
    +        } else {
    +          logError(s"Couldn't move driver log to YARN application dir", ace)
    +        }
    +      case e: Exception =>
    +        logError(s"Couldn't move driver log to YARN application dir", e)
    +    }
    +  }
    +
    +  private def moveToDfs(): Unit = {
    +    try {
    +      fileSystem.copyFromLocalFile(true,
    +        new Path(localLogFile),
    +        new Path(hdfsLogFile))
    +      fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +      logInfo(
    +        s"Moved the local driver log file to spark dfs at: ${hdfsLogFile}")
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not move local driver log to spark dfs", e)
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter extends Runnable with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      } finally {
    +        inStream.close()
    +        outputStream.close()
    +        streamClosed = true
    +      }
    +    }
    +
    +    def closeWriter(): Unit = {
    +      try {
    +        threadpool.execute(new Runnable() {
    +          def run(): Unit = HdfsAsyncWriter.this.close()
    +        })
    +        threadpool.shutdown()
    +        threadpool.awaitTermination(1, TimeUnit.MINUTES)
    +      } catch {
    +        case e: Exception =>
    +          logError("Error in closing driver log input/output stream", e)
    +      }
    +    }
    +  }
    +
    +}
    +
    +private[spark] object DriverLogger extends Logging {
    +  val APPENDER_NAME = "_DriverLogAppender"
    +
    +  def apply(conf: SparkConf): Option[DriverLogger] = {
    +    if ((conf.get(DRIVER_LOG_SYNCTODFS) || conf.get(DRIVER_LOG_SYNCTOYARN))
    +      && Utils.isClientMode(conf)) {
    --- End diff --
    
    indent one more level


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96902/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96902 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96902/testReport)** for PR 22504 at commit [`eb751b2`](https://github.com/apache/spark/commit/eb751b293f99e412727cbbd1229dcdd73dfafb54).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96717 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96717/testReport)** for PR 22504 at commit [`e13d6c6`](https://github.com/apache/spark/commit/e13d6c6744baafa7e79e5d37fc62d1a2fecd6f36).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98401 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98401/testReport)** for PR 22504 at commit [`4df08bd`](https://github.com/apache/spark/commit/4df08bd56b4cd51c4072aa026bf7f46bc574421d).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #4409 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4409/testReport)** for PR 22504 at commit [`4df08bd`](https://github.com/apache/spark/commit/4df08bd56b4cd51c4072aa026bf7f46bc574421d).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r230465268
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,209 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DEFAULT_LAYOUT = "%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf),
    +    DriverLogger.DRIVER_LOG_DIR,
    +    DriverLogger.DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT).get)
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DEFAULT_LAYOUT)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not persist driver logs to dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      }
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +      with Logging {
    +
    +    private var streamClosed = false
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private var threadpool: ScheduledExecutorService = _
    +    init()
    +
    +    private def init(): Unit = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      val fileSystem: FileSystem = new Path(rootDir).getFileSystem(hadoopConf)
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to persist driver logs")
    +      }
    +      val dfsLogFile: String = FileUtils.getFile(rootDir, appId
    +        + DriverLogger.DRIVER_LOG_FILE_SUFFIX).getAbsolutePath()
    +      try {
    +        inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +        outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +        fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +      } catch {
    +        case e: Exception =>
    +          if (inStream != null) {
    --- End diff --
    
    `JavaUtils.closeQuietly`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by bersprockets <gi...@git.apache.org>.
Github user bersprockets commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    The Py4JJavaError StackOverflow happens pretty reliably. I am guessing its related to the change.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226467039
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    --- End diff --
    
    spark.driver.log.syncToDfs.enabled  does not exist.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228671856
  
    --- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
    @@ -2328,7 +2328,14 @@ private[spark] object Utils extends Logging {
        * configure a new log4j level
        */
       def setLogLevel(l: org.apache.log4j.Level) {
    -    org.apache.log4j.Logger.getRootLogger().setLevel(l)
    +    val rootLogger = org.apache.log4j.Logger.getRootLogger()
    +    rootLogger.setLevel(l)
    +    rootLogger.getAllAppenders().asScala.foreach { tmp =>
    +      tmp match {
    --- End diff --
    
    You can have the cases directly in the body of the foreach.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223901876
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private var hdfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key, "/tmp/driver_logs").split(",").head
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    --- End diff --
    
    This should not be called twice, so it is not needed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228672761
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,206 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DEFAULT_LAYOUT = "%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf),
    +    DriverLogger.DRIVER_LOG_DIR,
    +    DriverLogger.DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.get(DRIVER_LOG_LAYOUT).isDefined) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT).get)
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DEFAULT_LAYOUT)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not persist driver logs to dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      }
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +      with Logging {
    +
    +    private var streamClosed = false
    +    private var fileSystem: FileSystem = _
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      fileSystem = new Path(rootDir).getFileSystem(hadoopConf)
    --- End diff --
    
    This is actually super weird, initializing another field in the initializer of this field.
    
    It's looking like you should just have an `init()` function, or maybe have `rootDir` as a field so you can initialize `fileSystem` more directly...
    
    (Sometimes I really miss Java constructors.)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97557/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226472013
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,202 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not persist driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +      with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    --- End diff --
    
    See my previous comment for why this call is not right.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r231346161
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,40 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
    +    is true. Within this base directory, each application logs the driver logs to an application specific file.
    +    Users may want to set this to a unified location like an HDFS directory so driver log files can be persisted
    +    for later usage. This directory should allow any Spark user to read/write files and the Spark History Server
    +    user to delete files. Additionally, older logs from this directory are cleaned by
    +    <a href="monitoring.html#spark-history-server-configuration-options"> Spark History Server</a>  if
    +    <code>spark.history.fs.driverlog.cleaner.enabled</code> is true and, if they are older than max age configured
    +    at <code>spark.history.fs.driverlog.cleaner.maxAge</code>.
    --- End diff --
    
    s/at/by setting


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98407/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228673299
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala ---
    @@ -413,6 +417,68 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
         }
       }
     
    +  test("driver log cleaner") {
    +    val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10)
    +    val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20)
    +    val maxAge = TimeUnit.SECONDS.toSeconds(40)
    +    val clock = new ManualClock(0)
    +    val testConf = new SparkConf()
    +    testConf.set("spark.history.fs.logDirectory",
    +      Utils.createTempDir(namePrefix = "eventLog").getAbsolutePath())
    +    testConf.set(DRIVER_LOG_DFS_DIR, testDir.getAbsolutePath())
    +    testConf.set(DRIVER_LOG_CLEANER_ENABLED, true)
    +    testConf.set(DRIVER_LOG_CLEANER_INTERVAL, maxAge / 4)
    +    testConf.set(MAX_DRIVER_LOG_AGE_S, maxAge)
    +    val provider = new FsHistoryProvider(testConf, clock)
    +
    +    val log1 = FileUtils.getFile(testDir, "1" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log1)
    +    val modTime1 = System.currentTimeMillis()
    --- End diff --
    
    You're mixing a manual clock with system time. Why not use `File.setLastModified` like other tests?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97575 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97575/testReport)** for PR 22504 at commit [`6e14137`](https://github.com/apache/spark/commit/6e141373910df8c159429c367a8e937ad71eccbe).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98570 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98570/testReport)** for PR 22504 at commit [`616bd01`](https://github.com/apache/spark/commit/616bd01b5b23dc0ce0b50e95b7fd6684b1f7e9d5).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #4394 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4394/testReport)** for PR 22504 at commit [`a620bcc`](https://github.com/apache/spark/commit/a620bcc947111ad9530841fd5de20c86c477226e).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96384/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Also, for our use case, we are planning to get the logs just via yarn logs


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226394446
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    +        FileUtils.getFile(localLogFile).getParentFile()))
    +    }
    +  }
    +
    +  // Visible for testing
    +  private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val dfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +      if (!fileSystem.exists(new Path(rootDir))) {
    +        throw new RuntimeException(s"${rootDir} does not exist." +
    +          s" Please create this dir in order to sync driver logs")
    +      }
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var inStream: InputStream = null
    +    private var outputStream: FSDataOutputStream = null
    +    try {
    +      inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +      outputStream = fileSystem.create(new Path(dfsLogFile), true)
    +      fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS)
    +    } catch {
    +      case e: Exception =>
    +        if (inStream != null) {
    +          Utils.tryLogNonFatalError(inStream.close())
    +        }
    +        if (outputStream != null) {
    +          Utils.tryLogNonFatalError(outputStream.close())
    +        }
    +        throw e
    +    }
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${dfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    --- End diff --
    
    its not really "spark dfs".  Maybe "Failed copying driver logs to dfs".
    
    If there is a consistent error, you're going to see this error msg a lot as you'll keep calling `run()`.  I'm not sure how common that will be and if we should do some special handling (eg. only include the full stack trace the first time or something like that).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98230 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98230/testReport)** for PR 22504 at commit [`e898d47`](https://github.com/apache/spark/commit/e898d47529e9aaa13751cc89f5d5ce4b243e430d).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223849313
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        val appId = Utils.sanitizeDirName(conf.getAppId)
    +        writer = Some(new HdfsAsyncWriter(appId, hadoopConf))
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable
    +    with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private var hdfsLogFile: String = {
    +      val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key, "/tmp/driver_logs").split(",").head
    +      FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +    }
    +    private var fileSystem: FileSystem = FileSystem.get(hadoopConf)
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +    logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    --- End diff --
    
    Return early if `streamClosed`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    retest this please


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by squito <gi...@git.apache.org>.
Github user squito commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226396957
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,83 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private val DRIVER_LOG_DIR_DEFAULT = "/tmp/hdfs_logs"
    --- End diff --
    
    use a relative dir, not an absolute one, for testing data.  (what if a developer happened to have real data at /tmp/hdfs_logs before running this?)


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223842417
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(appDirStatus.getPath())
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => // no-op
    --- End diff --
    
    What call throws this exception?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228671145
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -796,16 +806,57 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
           .asScala
           .toList
         stale.foreach { log =>
    -      if (log.appId.isEmpty) {
    +      if (log.appId.isEmpty &&
    +          (!conf.get(DRIVER_LOG_DFS_DIR).isDefined ||
    +          !log.logPath.startsWith(new Path(conf.get(DRIVER_LOG_DFS_DIR).get).toString()))) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +    val driverLogFs = new Path(driverLogDir).getFileSystem(hadoopConf)
    +    val currentTime = clock.getTimeMillis()
    +    val maxTime = currentTime - conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +    val logFiles = driverLogFs.listLocatedStatus(new Path(driverLogDir))
    +    while (logFiles.hasNext()) {
    --- End diff --
    
    One issue here is that since you're basing this on the file system's contents, if these files are deleted outside of the SHS then you'll accumulate `LogInfo` entries for those deleted entries.
    
    The event log cleaner avoids that by basing this logic on the SHS's view of the file system, although I don't know if that same logic can be applied here.
    



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223871565
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,175 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    --- End diff --
    
    Utils.getLocalDir also does that actually. Spark deletes the directory that it creates.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by NiharS <gi...@git.apache.org>.
Github user NiharS commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223512824
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -192,7 +211,15 @@ private[spark] object Logging {
             defaultSparkLog4jConfig = false
             LogManager.resetConfiguration()
           } else {
    -        LogManager.getRootLogger().setLevel(defaultRootLevel)
    +        val rootLogger = LogManager.getRootLogger()
    +        rootLogger.setLevel(defaultRootLevel)
    +        rootLogger.getAllAppenders().asScala.foreach { tmp =>
    --- End diff --
    
    Any reason not to just iterate through `consoleAppenderToThreshold.keys()`? Not a huge deal but cuts down on a bit of work


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225343405
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private val DRIVER_LOG_DIR_DEFAULT = "/tmp/hdfs_logs"
    +
    +  override def beforeAll(): Unit = {
    +    FileUtils.forceMkdir(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  override def afterAll(): Unit = {
    +    JavaUtils.deleteRecursively(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  test("driver logs are persisted") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // On application end, file is moved to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    +    assert(hdfsFiles.length > 0)
    +    JavaUtils.deleteRecursively(hdfsDir)
    +    assert(!hdfsDir.exists())
    +  }
    +
    +  test("driver logs are synced to hdfs continuously") {
    +    val sc = getSparkContext()
    --- End diff --
    
    I changed the test to just check the final output, so it does not rely on loop logging anymore. That way, I don't need to expose `SparkContext.driverLogger` as well


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r230466226
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala ---
    @@ -413,6 +417,66 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
         }
       }
     
    +  test("driver log cleaner") {
    +    val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10)
    +    val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20)
    +    val maxAge = TimeUnit.SECONDS.toSeconds(40)
    +    val clock = new ManualClock(0)
    +    val testConf = new SparkConf()
    +    testConf.set("spark.history.fs.logDirectory",
    +      Utils.createTempDir(namePrefix = "eventLog").getAbsolutePath())
    +    testConf.set(DRIVER_LOG_DFS_DIR, testDir.getAbsolutePath())
    +    testConf.set(DRIVER_LOG_CLEANER_ENABLED, true)
    +    testConf.set(DRIVER_LOG_CLEANER_INTERVAL, maxAge / 4)
    +    testConf.set(MAX_DRIVER_LOG_AGE_S, maxAge)
    +    val provider = new FsHistoryProvider(testConf, clock)
    +
    +    val log1 = FileUtils.getFile(testDir, "1" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log1)
    +    clock.setTime(firstFileModifiedTime)
    +    log1.setLastModified(clock.getTimeMillis())
    +    provider.cleanDriverLogs()
    +
    +    val log2 = FileUtils.getFile(testDir, "2" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log2)
    +    val log3 = FileUtils.getFile(testDir, "3" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log3)
    +    clock.setTime(secondFileModifiedTime)
    +    log2.setLastModified(clock.getTimeMillis())
    +    log3.setLastModified(clock.getTimeMillis())
    +    provider.cleanDriverLogs()
    +
    +    // This should not trigger any cleanup
    +    provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(3)
    +
    +    // Should trigger cleanup for first file but not second one
    +    clock.setTime(firstFileModifiedTime + TimeUnit.SECONDS.toMillis(maxAge) + 1)
    +    provider.cleanDriverLogs()
    +    provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(2)
    +    assert(!log1.exists())
    +    assert(log2.exists())
    +    assert(log3.exists())
    +
    +    // Update the third file length while keeping the original modified time
    +    Utils.tryLogNonFatalError {
    --- End diff --
    
    You don't want to catch the error, do you? Otherwise, if it happens, the test won't be doing what you want it to do.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97565 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97565/testReport)** for PR 22504 at commit [`a0a85b3`](https://github.com/apache/spark/commit/a0a85b3eff0f115b983ce5ba3214e09f8ee90dd2).
     * This patch **fails PySpark unit tests**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226462505
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    --- End diff --
    
    "spark dfs"?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223866023
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/config/package.scala ---
    @@ -48,6 +48,19 @@ package object config {
         .bytesConf(ByteUnit.MiB)
         .createOptional
     
    +  private[spark] val DRIVER_LOG_DFS_DIR =
    +    ConfigBuilder("spark.driver.log.dfsDir").stringConf.createOptional
    +
    +  private[spark] val DRIVER_LOG_LAYOUT =
    +    ConfigBuilder("spark.driver.log.layout")
    +      .stringConf
    +      .createWithDefault("%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n")
    +
    +  private[spark] val DRIVER_LOG_SYNCTODFS =
    --- End diff --
    
    Yes, as you said I prefer to keep the functionality of disabling/enabling separate from log dir. But I do see that removing this reduces the number of configurations.
    
    I am ok either ways. Please let me know if I should remove this.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223842100
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(appDirStatus.getPath())
    --- End diff --
    
    So now, because of this functionality, it would be good to add documentation in `configuration.md` explaining the log directory option, and what are the expected owner, group and permissions on it. I don't remember if the `DriverLog` code automatically creates that directory, but it should not do that anymore, otherwise it may break this functionality.
    
    It may also be good to have a separate option to enable this cleaner instead of piggybacking on the event log cleaner config. Even the max age could be a different option that defaults to the same as the event log cleaner.
    
    Basically you now should have the same semantics as the event log directory.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225312422
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,195 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  // Visible for testing
    +  private[spark] var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    --- End diff --
    
    Shouldn't you close the appender you added, so the underlying file is closed / flushed too?
    
    Or is that done implicitly by this call somehow?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223901520
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(appDirStatus.getPath())
    --- End diff --
    
    I added the documentation but I am still relying on the existing history server configurations. This is because I am using an existing cleaner. Otherwise, I may have to create another cleaner for this, which can run at a different interval and has a different configured max age.
    
    Please let me know if I should create another cleaner for this or just provide a configuration for a different max age.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220016668
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    +        " to YARN application dir as no appId defined")
    +      case ace: AccessControlException =>
    +        if (!syncToDfs) {
    +          logWarning(s"Couldn't move local driver log to YARN application dir," +
    +            s" trying Spark application dir now")
    +          moveToDfs()
    +        } else {
    +          logError(s"Couldn't move driver log to YARN application dir", ace)
    +        }
    +      case e: Exception =>
    +        logError(s"Couldn't move driver log to YARN application dir", e)
    +    }
    +  }
    +
    +  private def moveToDfs(): Unit = {
    +    try {
    +      fileSystem.copyFromLocalFile(true,
    +        new Path(localLogFile),
    +        new Path(hdfsLogFile))
    +      fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +      logInfo(
    +        s"Moved the local driver log file to spark dfs at: ${hdfsLogFile}")
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not move local driver log to spark dfs", e)
    +    }
    +  }
    +
    +  private class HdfsAsyncWriter extends Runnable with Logging {
    +
    +    private var streamClosed = false
    +    private val inStream = new BufferedInputStream(new FileInputStream(localLogFile))
    +    private val outputStream: FSDataOutputStream = fileSystem.create(new Path(hdfsLogFile), true)
    +    fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +    private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE)
    +    private val threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread")
    +    threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS,
    +      TimeUnit.SECONDS)
    +
    +    def run(): Unit = {
    +      if (streamClosed) {
    +        return
    +      }
    +      try {
    +        var remaining = inStream.available()
    +        while (remaining > 0) {
    +          val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE))
    +          outputStream.write(tmpBuffer, 0, read)
    +          remaining -= read
    +        }
    +        outputStream.hflush()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    +      }
    +    }
    +
    +    def close(): Unit = {
    +      try {
    +        // Write all remaining bytes
    +        run()
    +      } catch {
    +        case e: Exception => logError("Failed to write to spark dfs", e)
    --- End diff --
    
    `run()` doesn't throw anything.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96576 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96576/testReport)** for PR 22504 at commit [`7da9781`](https://github.com/apache/spark/commit/7da9781f1ffe0e93c0ab7e281267bbddf52e0349).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220013762
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -192,7 +207,16 @@ private[spark] object Logging {
             defaultSparkLog4jConfig = false
             LogManager.resetConfiguration()
           } else {
    -        LogManager.getRootLogger().setLevel(defaultRootLevel)
    +        val rootLogger = LogManager.getRootLogger()
    +        rootLogger.setLevel(defaultRootLevel)
    +        val appenders = rootLogger.getAllAppenders
    +        while (appenders.hasMoreElements()) {
    +          val tmp = appenders.nextElement()
    +          tmp match {
    +            case ca: ConsoleAppender => ca.setThreshold(defaultRootLevel)
    --- End diff --
    
    You'll need to reset this to the appender's original level (not to the root logger's original level, which may be different).
    
    In fact you're not really setting the root logger level anymore, right? So `defaultRootLevel` doesn't really make much sense with your changes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by srowen <gi...@git.apache.org>.
Github user srowen commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Just open a separate JIRA, PR to fix it. I suspect we shouldn't run a Spark job in toString.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225311036
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -281,6 +288,16 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
             pool.scheduleWithFixedDelay(
               getRunner(() => cleanLogs()), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS)
           }
    +
    +      driverLogFs.foreach { _ =>
    +        if (conf.get(DRIVER_LOG_CLEANER_ENABLED).getOrElse(
    +            conf.getBoolean("spark.history.fs.cleaner.enabled", false))) {
    --- End diff --
    
    Turn `spark.history.fs.cleaner.enabled` into a config constant and use `fallbackConf`.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97617 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97617/testReport)** for PR 22504 at commit [`a3b6550`](https://github.com/apache/spark/commit/a3b6550704031c2717364f582190be9661c4c934).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98401 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98401/testReport)** for PR 22504 at commit [`4df08bd`](https://github.com/apache/spark/commit/4df08bd56b4cd51c4072aa026bf7f46bc574421d).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97561/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96729 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96729/testReport)** for PR 22504 at commit [`e13d6c6`](https://github.com/apache/spark/commit/e13d6c6744baafa7e79e5d37fc62d1a2fecd6f36).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97414/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220012966
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -2421,11 +2425,13 @@ class SparkContext(config: SparkConf) extends Logging {
         // the cluster manager to get an application ID (in case the cluster manager provides one).
         listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId),
           startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls))
    +    _driverLogger.map(_.startSync(_hadoopConfiguration))
       }
     
       /** Post the application end event */
       private def postApplicationEnd() {
         listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis))
    +    _driverLogger.map(_.stop())
    --- End diff --
    
    foreach


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98562/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98805/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223842956
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -166,6 +184,7 @@ private[spark] object Logging {
       @volatile private var initialized = false
       @volatile private var defaultRootLevel: Level = null
       @volatile private var defaultSparkLog4jConfig = false
    +  val consoleAppenderToThreshold = new ConcurrentHashMap[ConsoleAppender, Priority]()
    --- End diff --
    
    `private`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223900576
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    --- End diff --
    
    Could be. If it doesn't work this is fine.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223840922
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    --- End diff --
    
    `.appDirs.asScala.foreach`?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98230 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98230/testReport)** for PR 22504 at commit [`e898d47`](https://github.com/apache/spark/commit/e898d47529e9aaa13751cc89f5d5ce4b243e430d).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Can one of the admins verify this patch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223859868
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -806,6 +806,22 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
    +
    +    // Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +    try {
    +      val hdfsDir = conf.get("spark.driver.log.dfsDir")
    +      val appDirs = fs.listLocatedStatus(new Path(hdfsDir))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(appDirStatus.getPath())
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => // no-op
    --- End diff --
    
    conf.get("spark.driver.log.dfsDir")


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220014946
  
    --- Diff: core/src/main/scala/org/apache/spark/util/Utils.scala ---
    @@ -2328,7 +2328,16 @@ private[spark] object Utils extends Logging {
        * configure a new log4j level
        */
       def setLogLevel(l: org.apache.log4j.Level) {
    -    org.apache.log4j.Logger.getRootLogger().setLevel(l)
    +    val rootLogger = org.apache.log4j.Logger.getRootLogger()
    +    rootLogger.setLevel(l)
    +    val appenders = rootLogger.getAllAppenders
    --- End diff --
    
    `rootLogger.getAllAppenders().asScala.foreach { ... }`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/96576/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220274067
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    --- End diff --
    
    This exception can be thrown in conf.getAppId. The scenario is if there is an exception before Spark application starts, then this exception will be thrown while trying to close the logger.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97575/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #96956 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/96956/testReport)** for PR 22504 at commit [`522fcea`](https://github.com/apache/spark/commit/522fcea207bdccf19b29ffc3ae132da01d254ee0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/97560/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226741280
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,79 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, File, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private var rootDfsDir : File = _
    +
    +  override def beforeAll(): Unit = {
    +    super.beforeAll()
    +    rootDfsDir = Utils.createTempDir(namePrefix = "dfs_logs")
    +  }
    +
    +  override def afterAll(): Unit = {
    +    super.afterAll()
    +    JavaUtils.deleteRecursively(rootDfsDir)
    +  }
    +
    +  test("driver logs are persisted locally and synced to dfs") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, DriverLogger.DRIVER_LOG_DIR)
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals(DriverLogger.DRIVER_LOG_FILE))
    +
    +    sc.stop()
    +    assert(!driverLogsDir.exists())
    +    val dfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(dfsDir.exists())
    +    val dfsFiles = dfsDir.listFiles()
    +    dfsFiles.exists{ f => f.getName().equals(DriverLogger.DRIVER_LOG_FILE) && f.length() > 0 }
    --- End diff --
    
    space before `{`
    
    You also need to assert the condition...


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226466609
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if spark.history.fs.driverlog.cleaner.enabled is true.
    --- End diff --
    
    Put config names between `<code></code>`.
    
    This is also wrong, since you do not have an explanation of this option in the documentation. It will be true also if you just enable the normal SHS cleaner. You should explain that option explicitly and say its default value is the value of the other config.
    
    The SHS options are in monitoring.md.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226456102
  
    --- Diff: core/src/main/scala/org/apache/spark/SparkContext.scala ---
    @@ -2421,11 +2425,13 @@ class SparkContext(config: SparkConf) extends Logging {
         // the cluster manager to get an application ID (in case the cluster manager provides one).
         listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId),
           startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls))
    +    _driverLogger.foreach(_.startSync(_hadoopConfiguration))
       }
     
       /** Post the application end event */
       private def postApplicationEnd() {
         listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis))
    +    _driverLogger.foreach(_.stop())
    --- End diff --
    
    I think it may be better call this in `stop()`. Stopping that logger is not part of posting an event to the listener bus.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98230/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r231346067
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,40 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if <code>spark.driver.log.persistToDfs.enabled</code>
    +    is true. Within this base directory, each application logs the driver logs to an application specific file.
    +    Users may want to set this to a unified location like an HDFS directory so driver log files can be persisted
    +    for later usage. This directory should allow any Spark user to read/write files and the Spark History Server
    +    user to delete files. Additionally, older logs from this directory are cleaned by
    +    <a href="monitoring.html#spark-history-server-configuration-options"> Spark History Server</a>  if
    --- End diff --
    
    remove space after `>`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226504971
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    +    this directory are cleaned by Spark History Server if spark.history.fs.driverlog.cleaner.enabled is true.
    --- End diff --
    
    That documentation is not user visible.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #98562 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/98562/testReport)** for PR 22504 at commit [`81ed77b`](https://github.com/apache/spark/commit/81ed77b51c305ce4c48135150604a7607d1e44be).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226505024
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,196 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf) extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var writer: Option[DfsAsyncWriter] = None
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    } else if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(DRIVER_LOG_LAYOUT.defaultValueString)
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      // Setup a writer which moves the local file to hdfs continuously
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      writer = Some(new DfsAsyncWriter(appId, hadoopConf))
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME)
    +      LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME)
    +      Utils.tryLogNonFatalError(fa.close())
    +      writer.foreach(_.closeWriter())
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      Utils.tryLogNonFatalError(JavaUtils.deleteRecursively(
    --- End diff --
    
    ```
    Utils.tryLogNonFatalError {
       // code goes here
    }
    ```


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #4394 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/4394/testReport)** for PR 22504 at commit [`a620bcc`](https://github.com/apache/spark/commit/a620bcc947111ad9530841fd5de20c86c477226e).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223842662
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/Logging.scala ---
    @@ -143,13 +147,27 @@ trait Logging {
             // overriding the root logger's config if they're different.
             val replLogger = LogManager.getLogger(logName)
             val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN)
    +        // Update the consoleAppender threshold to replLevel
             if (replLevel != rootLogger.getEffectiveLevel()) {
               if (!silent) {
                 System.err.printf("Setting default log level to \"%s\".\n", replLevel)
                 System.err.println("To adjust logging level use sc.setLogLevel(newLevel). " +
                   "For SparkR, use setLogLevel(newLevel).")
               }
    -          rootLogger.setLevel(replLevel)
    +          rootLogger.getAllAppenders().asScala.foreach { tmp =>
    +            tmp match {
    --- End diff --
    
    You can match directly in the body of the `foreach`, no need for the extra level.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228673330
  
    --- Diff: core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala ---
    @@ -413,6 +417,68 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc
         }
       }
     
    +  test("driver log cleaner") {
    +    val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10)
    +    val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20)
    +    val maxAge = TimeUnit.SECONDS.toSeconds(40)
    +    val clock = new ManualClock(0)
    +    val testConf = new SparkConf()
    +    testConf.set("spark.history.fs.logDirectory",
    +      Utils.createTempDir(namePrefix = "eventLog").getAbsolutePath())
    +    testConf.set(DRIVER_LOG_DFS_DIR, testDir.getAbsolutePath())
    +    testConf.set(DRIVER_LOG_CLEANER_ENABLED, true)
    +    testConf.set(DRIVER_LOG_CLEANER_INTERVAL, maxAge / 4)
    +    testConf.set(MAX_DRIVER_LOG_AGE_S, maxAge)
    +    val provider = new FsHistoryProvider(testConf, clock)
    +
    +    val log1 = FileUtils.getFile(testDir, "1" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log1)
    +    val modTime1 = System.currentTimeMillis()
    +
    +    clock.setTime(modTime1 + firstFileModifiedTime)
    +    provider.cleanDriverLogs()
    +
    +    val log2 = FileUtils.getFile(testDir, "2" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log2)
    +    val log3 = FileUtils.getFile(testDir, "3" + DriverLogger.DRIVER_LOG_FILE_SUFFIX)
    +    createEmptyFile(log3)
    +    val modTime2 = System.currentTimeMillis()
    +
    +    clock.setTime(modTime1 + secondFileModifiedTime)
    +    provider.cleanDriverLogs()
    +
    +    // This should not trigger any cleanup
    +    provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(3)
    +
    +    // Should trigger cleanup for first file but not second one
    +    clock.setTime(modTime1 + firstFileModifiedTime + TimeUnit.SECONDS.toMillis(maxAge) + 1)
    +    provider.cleanDriverLogs()
    +    provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(2)
    +    assert(!log1.exists())
    +    assert(log2.exists())
    +    assert(log3.exists())
    +
    +    // Should cleanup the second file but not the third file, as filelength changed.
    +    val writer = new OutputStreamWriter(new BufferedOutputStream(new FileOutputStream(log3)))
    --- End diff --
    
    `Files.write` is shorter and nicer.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r228669828
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -796,16 +806,57 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
           .asScala
           .toList
         stale.foreach { log =>
    -      if (log.appId.isEmpty) {
    +      if (log.appId.isEmpty &&
    +          (!conf.get(DRIVER_LOG_DFS_DIR).isDefined ||
    --- End diff --
    
    This check is kinda awkward. How about a new property in `LogInfo` with the type of the log?
    
    You could then also just filter those out above before `.toList`, since the `clearBlacklist` call is unrelated to the logs you're adding.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/98374/
    Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by ankuriitg <gi...@git.apache.org>.
Github user ankuriitg commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226718233
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -800,14 +817,33 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR)
    +    driverLogDir.foreach { dl =>
    +      val maxTime = clock.getTimeMillis() -
    +        conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +      val appDirs = driverLogFs.get.listLocatedStatus(new Path(dl))
    +      while (appDirs.hasNext()) {
    +        val appDirStatus = appDirs.next()
    +        if (appDirStatus.getModificationTime() < maxTime) {
    +          logInfo(s"Deleting expired driver log for: ${appDirStatus.getPath().getName()}")
    +          deleteLog(driverLogFs.get, appDirStatus.getPath())
    --- End diff --
    
    Ohh right, the fix is similar to what exists currently. I just wanted to add the note that logs will be deleted after that time period.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226466182
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    +    user to read/write files and the spark history server user to delete files. Additionally, older logs from
    --- End diff --
    
    Spark History Server


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r230463338
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -812,18 +821,74 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
           .reverse()
           .first(maxTime)
           .asScala
    +      .filter(l => l.logType == LogType.EventLogs)
    --- End diff --
    
    `.filter { l => ... }`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    **[Test build #97414 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97414/testReport)** for PR 22504 at commit [`a46734c`](https://github.com/apache/spark/commit/a46734c46355b31863178ff9f0169118ca15a695).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226465465
  
    --- Diff: docs/configuration.md ---
    @@ -266,6 +266,37 @@ of the most common options to set are:
         Only has effect in Spark standalone mode or Mesos cluster deploy mode.
       </td>
     </tr>
    +<tr>
    +  <td><code>spark.driver.log.dfsDir</code></td>
    +  <td>(none)</td>
    +  <td>
    +    Base directory in which Spark driver logs are synced, if spark.driver.log.syncToDfs.enabled is true.
    +    Within this base directory, Spark creates a sub-directory for each application, and logs the driver logs
    +    specific to the application in this directory. Users may want to set this to a unified location like an
    +    HDFS directory so driver log files can be persisted for later usage. This directory should allow any spark
    --- End diff --
    
    Spark


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Client mode...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Merged build finished. Test FAILed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r225314910
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,116 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite with LocalSparkContext {
    +
    +  private val DRIVER_LOG_DIR_DEFAULT = "/tmp/hdfs_logs"
    +
    +  override def beforeAll(): Unit = {
    +    FileUtils.forceMkdir(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  override def afterAll(): Unit = {
    +    JavaUtils.deleteRecursively(FileUtils.getFile(DRIVER_LOG_DIR_DEFAULT))
    +  }
    +
    +  test("driver logs are persisted") {
    +    val sc = getSparkContext()
    +
    +    val app_id = sc.applicationId
    +    // Run a simple spark application
    +    sc.parallelize(1 to 1000).count()
    +
    +    // Assert driver log file exists
    +    val rootDir = Utils.getLocalDir(sc.getConf)
    +    val driverLogsDir = FileUtils.getFile(rootDir, "driver_logs")
    +    assert(driverLogsDir.exists())
    +    val files = driverLogsDir.listFiles()
    +    assert(files.length === 1)
    +    assert(files(0).getName.equals("driver.log"))
    +
    +    sc.stop()
    +    // On application end, file is moved to Hdfs (which is a local dir for this test)
    +    assert(!driverLogsDir.exists())
    +    val hdfsDir = FileUtils.getFile(sc.getConf.get(DRIVER_LOG_DFS_DIR).get, app_id)
    +    assert(hdfsDir.exists())
    +    val hdfsFiles = hdfsDir.listFiles()
    +    assert(hdfsFiles.length > 0)
    +    JavaUtils.deleteRecursively(hdfsDir)
    +    assert(!hdfsDir.exists())
    +  }
    +
    +  test("driver logs are synced to hdfs continuously") {
    +    val sc = getSparkContext()
    --- End diff --
    
    I'm not a fan of what you did, exposing `SparkContext.driverLogger` for tests.
    
    It should be possible to write this test without using `SparkContext` at all. Just create a `DriverLogger` and control it from the test.
    
    Also, you did not address all of my previous feedback. e.g., the loop logging a bunch of things should not be needed. You should have some explicit action that causes the flush instead of relying on side effects of other calls.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Clie...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r226767375
  
    --- Diff: core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala ---
    @@ -798,14 +815,52 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock)
         stale.foreach { log =>
           if (log.appId.isEmpty) {
             logInfo(s"Deleting invalid / corrupt event log ${log.logPath}")
    -        deleteLog(new Path(log.logPath))
    +        deleteLog(fs, new Path(log.logPath))
             listing.delete(classOf[LogInfo], log.logPath)
           }
         }
         // Clean the blacklist from the expired entries.
         clearBlacklist(CLEAN_INTERVAL_S)
       }
     
    +  /**
    +   * Delete driver logs from the configured spark dfs dir that exceed the configured max age
    +   */
    +  private[history] def cleanDriverLogs(): Unit = Utils.tryLog {
    +    val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR).get
    +    val currentTime = clock.getTimeMillis()
    +    val maxTime = currentTime - conf.get(MAX_DRIVER_LOG_AGE_S) * 1000
    +    val appDirs = driverLogFs.get.listLocatedStatus(new Path(driverLogDir))
    +    while (appDirs.hasNext()) {
    +      val appDirStatus = appDirs.next()
    +      if (appDirStatus.isDirectory()) {
    +        val logFiles = driverLogFs.get.listStatus(appDirStatus.getPath())
    +        var deleteDir = true
    +        logFiles.foreach { f =>
    +          try {
    +            val info = listing.read(classOf[LogInfo], f.getPath().toString())
    --- End diff --
    
    Oh, an you probably want unit tests for this new code.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r223850451
  
    --- Diff: core/src/test/scala/org/apache/spark/util/logging/DriverLoggerSuite.scala ---
    @@ -0,0 +1,109 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io.{BufferedInputStream, FileInputStream}
    +
    +import org.apache.commons.io.FileUtils
    +
    +import org.apache.spark._
    +import org.apache.spark.{SparkContext, SparkFunSuite}
    +import org.apache.spark.internal.config._
    +import org.apache.spark.launcher.SparkLauncher
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.Utils
    +
    +class DriverLoggerSuite extends SparkFunSuite {
    +
    +  test("driver logs are persisted") {
    +    val sc = getSparkContext()
    +    // Wait for application to start
    --- End diff --
    
    What exactly is this waiting for? Sleeping is generally the wrong way to wait for something.
    
    You should also extend `LocalSparkContext`.
    
    If you really need to wait, you should do it in `getSparkContext()` to avoid having to add the wait logic to every test.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/22504#discussion_r220016494
  
    --- Diff: core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala ---
    @@ -0,0 +1,241 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.util.logging
    +
    +import java.io._
    +import java.util.concurrent.TimeUnit
    +
    +import scala.util.{Failure, Try}
    +
    +import org.apache.commons.io.FileUtils
    +import org.apache.hadoop.conf.Configuration
    +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path}
    +import org.apache.hadoop.fs.permission.FsPermission
    +import org.apache.hadoop.security.{AccessControlException, UserGroupInformation}
    +import org.apache.hadoop.yarn.conf.YarnConfiguration
    +import org.apache.log4j.{FileAppender => Log4jFileAppender, _}
    +
    +import org.apache.spark.SparkConf
    +import org.apache.spark.internal.Logging
    +import org.apache.spark.internal.config._
    +import org.apache.spark.network.util.JavaUtils
    +import org.apache.spark.util.{ThreadUtils, Utils}
    +
    +private[spark] class DriverLogger(conf: SparkConf)
    +    extends Logging {
    +
    +  private val UPLOAD_CHUNK_SIZE = 1024 * 1024
    +  private val UPLOAD_INTERVAL_IN_SECS = 5
    +  private val DRIVER_LOG_FILE = "driver.log"
    +  private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort)
    +
    +  private var localLogFile: String = FileUtils.getFile(
    +    Utils.getLocalDir(conf), "driver_logs", DRIVER_LOG_FILE).getAbsolutePath()
    +  private var hdfsLogFile: String = _
    +  private var writer: Option[HdfsAsyncWriter] = None
    +  private var hadoopConfiguration: Configuration = _
    +  private var fileSystem: FileSystem = _
    +
    +  private val syncToDfs: Boolean = conf.get(DRIVER_LOG_SYNCTODFS)
    +  private val syncToYarn: Boolean = conf.get(DRIVER_LOG_SYNCTOYARN)
    +
    +  addLogAppender()
    +
    +  private def addLogAppender(): Unit = {
    +    val appenders = LogManager.getRootLogger().getAllAppenders()
    +    val layout = if (appenders.hasMoreElements()) {
    +      appenders.nextElement().asInstanceOf[Appender].getLayout()
    +    } else {
    +      new PatternLayout(conf.get(DRIVER_LOG_LAYOUT))
    +    }
    +    val fa = new Log4jFileAppender(layout, localLogFile)
    +    fa.setName(DriverLogger.APPENDER_NAME)
    +    LogManager.getRootLogger().addAppender(fa)
    +    logInfo(s"Added a local log appender at: ${localLogFile}")
    +  }
    +
    +  def startSync(hadoopConf: Configuration): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      hdfsLogFile = {
    +        val rootDir = conf.get(DRIVER_LOG_DFS_DIR.key,
    +          "/tmp/driver_logs").split(",").head
    +        FileUtils.getFile(rootDir, appId, DRIVER_LOG_FILE).getAbsolutePath()
    +      }
    +      hadoopConfiguration = hadoopConf
    +      fileSystem = FileSystem.get(hadoopConf)
    +
    +      // Setup a writer which moves the local file to hdfs continuously
    +      if (syncToDfs) {
    +        writer = Some(new HdfsAsyncWriter())
    +        logInfo(s"The local driver log file is being synced to spark dfs at: ${hdfsLogFile}")
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Could not sync driver logs to spark dfs", e)
    +    }
    +  }
    +
    +  def stop(): Unit = {
    +    try {
    +      writer.map(_.closeWriter())
    +      if (syncToYarn) {
    +        moveToYarnAppDir()
    +      }
    +    } catch {
    +      case e: Exception =>
    +        logError(s"Error in persisting driver logs", e)
    +    } finally {
    +      try {
    +        JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile())
    +      } catch {
    +        case e: Exception =>
    +          logError(s"Error in deleting local driver log dir", e)
    +      }
    +    }
    +  }
    +
    +  private def moveToYarnAppDir(): Unit = {
    +    try {
    +      val appId = Utils.sanitizeDirName(conf.getAppId)
    +      val yarnConf = new YarnConfiguration(hadoopConfiguration)
    +      val rootDir = yarnConf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR)
    +      if (rootDir != null && !rootDir.isEmpty()) {
    +        var parentDir = if (UserGroupInformation.getCurrentUser() != null) {
    +          FileUtils.getFile(
    +            rootDir,
    +            UserGroupInformation.getCurrentUser().getShortUserName(),
    +            "logs",
    +            appId)
    +        } else {
    +          FileUtils.getFile(rootDir, "logs", appId)
    +        }
    +        if (syncToDfs) {
    +          fileSystem.rename(new Path(hdfsLogFile), new Path(parentDir.getAbsolutePath()))
    +          fileSystem.delete(new Path(FileUtils.getFile(hdfsLogFile).getParent()), true)
    +          logInfo(
    +            s"Moved the driver log file to: ${parentDir.getAbsolutePath()}")
    +        } else {
    +          fileSystem.copyFromLocalFile(true,
    +            new Path(localLogFile),
    +            new Path(parentDir.getAbsolutePath()))
    +          logInfo(
    +            s"Moved the local driver log file to spark dfs at: ${parentDir.getAbsolutePath()}")
    +        }
    +      }
    +    } catch {
    +      case nse: NoSuchElementException => logWarning("Couldn't move driver log" +
    +        " to YARN application dir as no appId defined")
    +      case ace: AccessControlException =>
    +        if (!syncToDfs) {
    +          logWarning(s"Couldn't move local driver log to YARN application dir," +
    +            s" trying Spark application dir now")
    +          moveToDfs()
    +        } else {
    +          logError(s"Couldn't move driver log to YARN application dir", ace)
    +        }
    +      case e: Exception =>
    +        logError(s"Couldn't move driver log to YARN application dir", e)
    +    }
    +  }
    +
    +  private def moveToDfs(): Unit = {
    +    try {
    +      fileSystem.copyFromLocalFile(true,
    +        new Path(localLogFile),
    +        new Path(hdfsLogFile))
    +      fileSystem.setPermission(new Path(hdfsLogFile), LOG_FILE_PERMISSIONS)
    +      logInfo(
    +        s"Moved the local driver log file to spark dfs at: ${hdfsLogFile}")
    --- End diff --
    
    Fits in previous line.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    Can one of the admins verify this patch?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #22504: [SPARK-25118][Submit] Persist Driver Logs in Yarn Client...

Posted by vanzin <gi...@git.apache.org>.
Github user vanzin commented on the issue:

    https://github.com/apache/spark/pull/22504
  
    ok to test


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org