-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-17711] Compress rolled executor log #15285
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
15034e5
1df96e6
7356cdf
490eeed
c9dd1b0
ae08495
ef4f2b9
e5676a6
7cc6935
89b9acd
60cc130
bd47bd4
41da0ea
a6fbebe
440fcfc
b645b76
81465ca
82d4575
b3041b7
4ead779
1e3302d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,6 +27,7 @@ import java.nio.file.Files | |
| import java.util.{Locale, Properties, Random, UUID} | ||
| import java.util.concurrent._ | ||
| import java.util.concurrent.atomic.AtomicBoolean | ||
| import java.util.zip.GZIPInputStream | ||
| import javax.net.ssl.HttpsURLConnection | ||
|
|
||
| import scala.annotation.tailrec | ||
|
|
@@ -38,8 +39,10 @@ import scala.reflect.ClassTag | |
| import scala.util.Try | ||
| import scala.util.control.{ControlThrowable, NonFatal} | ||
|
|
||
| import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache} | ||
| import com.google.common.io.{ByteStreams, Files => GFiles} | ||
| import com.google.common.net.InetAddresses | ||
| import org.apache.commons.io.IOUtils | ||
| import org.apache.commons.lang3.SystemUtils | ||
| import org.apache.hadoop.conf.Configuration | ||
| import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} | ||
|
|
@@ -55,6 +58,7 @@ import org.apache.spark.internal.Logging | |
| import org.apache.spark.internal.config.{DYN_ALLOCATION_INITIAL_EXECUTORS, DYN_ALLOCATION_MIN_EXECUTORS, EXECUTOR_INSTANCES} | ||
| import org.apache.spark.network.util.JavaUtils | ||
| import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} | ||
| import org.apache.spark.util.logging.RollingFileAppender | ||
|
|
||
| /** CallSite represents a place in user code. It can have a short and a long form. */ | ||
| private[spark] case class CallSite(shortForm: String, longForm: String) | ||
|
|
@@ -1448,14 +1452,72 @@ private[spark] object Utils extends Logging { | |
| CallSite(shortForm, longForm) | ||
| } | ||
|
|
||
| private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = | ||
| "spark.worker.ui.compressedLogFileLengthCacheSize" | ||
| private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 | ||
| private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null | ||
| private def getCompressedLogFileLengthCache( | ||
| sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { | ||
| if (compressedLogFileLengthCache == null) { | ||
| val compressedLogFileLengthCacheSize = sparkConf.getInt( | ||
| UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, | ||
| DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) | ||
| compressedLogFileLengthCache = CacheBuilder.newBuilder() | ||
| .maximumSize(compressedLogFileLengthCacheSize) | ||
| .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { | ||
| override def load(path: String): java.lang.Long = { | ||
| Utils.getCompressedFileLength(new File(path)) | ||
| } | ||
| }) | ||
| } | ||
| compressedLogFileLengthCache | ||
| } | ||
|
|
||
| /** | ||
| * Return the file length, if the file is compressed it returns the uncompressed file length. | ||
| * It also caches the uncompressed file size to avoid repeated decompression. The cache size is | ||
| * read from workerConf. | ||
| */ | ||
| def getFileLength(file: File, workConf: SparkConf): Long = { | ||
| if (file.getName.endsWith(".gz")) { | ||
| getCompressedLogFileLengthCache(workConf).get(file.getAbsolutePath) | ||
| } else { | ||
| file.length | ||
| } | ||
| } | ||
|
|
||
| /** Return uncompressed file length of a compressed file. */ | ||
| private def getCompressedFileLength(file: File): Long = { | ||
| try { | ||
| // Uncompress .gz file to determine file size. | ||
| var fileSize = 0L | ||
| val gzInputStream = new GZIPInputStream(new FileInputStream(file)) | ||
| val bufSize = 1024 | ||
| val buf = new Array[Byte](bufSize) | ||
| var numBytes = IOUtils.read(gzInputStream, buf) | ||
| while (numBytes > 0) { | ||
| fileSize += numBytes | ||
| numBytes = IOUtils.read(gzInputStream, buf) | ||
| } | ||
| fileSize | ||
| } catch { | ||
| case e: Throwable => | ||
| logError(s"Cannot get file length of ${file}", e) | ||
| throw e | ||
| } | ||
| } | ||
|
|
||
| /** Return a string containing part of a file from byte 'start' to 'end'. */ | ||
| def offsetBytes(path: String, start: Long, end: Long): String = { | ||
| def offsetBytes(path: String, length: Long, start: Long, end: Long): String = { | ||
| val file = new File(path) | ||
| val length = file.length() | ||
| val effectiveEnd = math.min(length, end) | ||
| val effectiveStart = math.max(0, start) | ||
| val buff = new Array[Byte]((effectiveEnd-effectiveStart).toInt) | ||
| val stream = new FileInputStream(file) | ||
| val stream = if (path.endsWith(".gz")) { | ||
| new GZIPInputStream(new FileInputStream(file)) | ||
| } else { | ||
| new FileInputStream(file) | ||
| } | ||
|
|
||
| try { | ||
| ByteStreams.skipFully(stream, effectiveStart) | ||
|
|
@@ -1471,16 +1533,16 @@ private[spark] object Utils extends Logging { | |
| * and `endIndex` is based on the cumulative size of all the files take in | ||
| * the given order. See figure below for more details. | ||
| */ | ||
| def offsetBytes(files: Seq[File], start: Long, end: Long): String = { | ||
| val fileLengths = files.map { _.length } | ||
| def offsetBytes(files: Seq[File], fileLengths: Seq[Long], start: Long, end: Long): String = { | ||
| assert(files.length == fileLengths.length) | ||
| val startIndex = math.max(start, 0) | ||
| val endIndex = math.min(end, fileLengths.sum) | ||
| val fileToLength = files.zip(fileLengths).toMap | ||
| logDebug("Log files: \n" + fileToLength.mkString("\n")) | ||
|
|
||
| val stringBuffer = new StringBuffer((endIndex - startIndex).toInt) | ||
| var sum = 0L | ||
| for (file <- files) { | ||
| files.zip(fileLengths).foreach { case (file, fileLength) => | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Missed this. We should assert that the number of files and lengths are same. Zip can silently drop stuff if they dont match |
||
| val startIndexOfFile = sum | ||
| val endIndexOfFile = sum + fileToLength(file) | ||
| logDebug(s"Processing file $file, " + | ||
|
|
@@ -1499,19 +1561,19 @@ private[spark] object Utils extends Logging { | |
|
|
||
| if (startIndex <= startIndexOfFile && endIndex >= endIndexOfFile) { | ||
| // Case C: read the whole file | ||
| stringBuffer.append(offsetBytes(file.getAbsolutePath, 0, fileToLength(file))) | ||
| stringBuffer.append(offsetBytes(file.getAbsolutePath, fileLength, 0, fileToLength(file))) | ||
| } else if (startIndex > startIndexOfFile && startIndex < endIndexOfFile) { | ||
| // Case A and B: read from [start of required range] to [end of file / end of range] | ||
| val effectiveStartIndex = startIndex - startIndexOfFile | ||
| val effectiveEndIndex = math.min(endIndex - startIndexOfFile, fileToLength(file)) | ||
| stringBuffer.append(Utils.offsetBytes( | ||
| file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) | ||
| file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) | ||
| } else if (endIndex > startIndexOfFile && endIndex < endIndexOfFile) { | ||
| // Case D: read from [start of file] to [end of require range] | ||
| val effectiveStartIndex = math.max(startIndex - startIndexOfFile, 0) | ||
| val effectiveEndIndex = endIndex - startIndexOfFile | ||
| stringBuffer.append(Utils.offsetBytes( | ||
| file.getAbsolutePath, effectiveStartIndex, effectiveEndIndex)) | ||
| file.getAbsolutePath, fileLength, effectiveStartIndex, effectiveEndIndex)) | ||
| } | ||
| sum += fileToLength(file) | ||
| logDebug(s"After processing file $file, string built is ${stringBuffer.toString}") | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,9 +17,11 @@ | |
|
|
||
| package org.apache.spark.util.logging | ||
|
|
||
| import java.io.{File, FileFilter, InputStream} | ||
| import java.io._ | ||
| import java.util.zip.GZIPOutputStream | ||
|
|
||
| import com.google.common.io.Files | ||
| import org.apache.commons.io.IOUtils | ||
|
|
||
| import org.apache.spark.SparkConf | ||
|
|
||
|
|
@@ -45,6 +47,7 @@ private[spark] class RollingFileAppender( | |
| import RollingFileAppender._ | ||
|
|
||
| private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) | ||
| private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should we enable this by default?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I don't want to existing behavior. |
||
|
|
||
| /** Stop the appender */ | ||
| override def stop() { | ||
|
|
@@ -76,15 +79,42 @@ private[spark] class RollingFileAppender( | |
| } | ||
| } | ||
|
|
||
| // Roll the log file and compress if enableCompression is true. | ||
| private def rotateFile(activeFile: File, rolloverFile: File): Unit = { | ||
| if (enableCompression) { | ||
| val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) | ||
| var gzOutputStream: GZIPOutputStream = null | ||
| var inputStream: InputStream = null | ||
| try { | ||
| inputStream = new FileInputStream(activeFile) | ||
| gzOutputStream = new GZIPOutputStream(new FileOutputStream(gzFile)) | ||
| IOUtils.copy(inputStream, gzOutputStream) | ||
| inputStream.close() | ||
| gzOutputStream.close() | ||
| activeFile.delete() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Are you sure this is a good idea to delete the activeFile before closing the inputStream? I am not sure this is the right thing to do.
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. In fact the docs of IOUtils.closeQuietly says that it should not be used as a replacement for normal closing. So this is not right. |
||
| } finally { | ||
| IOUtils.closeQuietly(inputStream) | ||
| IOUtils.closeQuietly(gzOutputStream) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What kinds of error do we expect? If there is an exception, we will lose log data, right? Seems we should at least have a log.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It may throw some kind of |
||
| } | ||
| } else { | ||
| Files.move(activeFile, rolloverFile) | ||
| } | ||
| } | ||
|
|
||
| // Check if the rollover file already exists. | ||
| private def rolloverFileExist(file: File): Boolean = { | ||
| file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists | ||
| } | ||
|
|
||
| /** Move the active log file to a new rollover file */ | ||
| private def moveFile() { | ||
| val rolloverSuffix = rollingPolicy.generateRolledOverFileSuffix() | ||
| val rolloverFile = new File( | ||
| activeFile.getParentFile, activeFile.getName + rolloverSuffix).getAbsoluteFile | ||
| logDebug(s"Attempting to rollover file $activeFile to file $rolloverFile") | ||
| if (activeFile.exists) { | ||
| if (!rolloverFile.exists) { | ||
| Files.move(activeFile, rolloverFile) | ||
| if (!rolloverFileExist(rolloverFile)) { | ||
| rotateFile(activeFile, rolloverFile) | ||
| logInfo(s"Rolled over $activeFile to $rolloverFile") | ||
| } else { | ||
| // In case the rollover file name clashes, make a unique file name. | ||
|
|
@@ -97,11 +127,11 @@ private[spark] class RollingFileAppender( | |
| altRolloverFile = new File(activeFile.getParent, | ||
| s"${activeFile.getName}$rolloverSuffix--$i").getAbsoluteFile | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Want to double check. If we enable compression, the suffix will still be gz even we use the alternative file name, right?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes, it will be whatever we have before + |
||
| i += 1 | ||
| } while (i < 10000 && altRolloverFile.exists) | ||
| } while (i < 10000 && rolloverFileExist(altRolloverFile)) | ||
|
|
||
| logWarning(s"Rollover file $rolloverFile already exists, " + | ||
| s"rolled over $activeFile to file $altRolloverFile") | ||
| Files.move(activeFile, altRolloverFile) | ||
| rotateFile(activeFile, altRolloverFile) | ||
| } | ||
| } else { | ||
| logWarning(s"File $activeFile does not exist") | ||
|
|
@@ -142,6 +172,9 @@ private[spark] object RollingFileAppender { | |
| val SIZE_DEFAULT = (1024 * 1024).toString | ||
| val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" | ||
| val DEFAULT_BUFFER_SIZE = 8192 | ||
| val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Shouldnt we document this in the spark docs? |
||
|
|
||
| val GZIP_LOG_SUFFIX = ".gz" | ||
|
|
||
| /** | ||
| * Get the sorted list of rolled over files. This assumes that the all the rolled | ||
|
|
@@ -158,6 +191,6 @@ private[spark] object RollingFileAppender { | |
| val file = new File(directory, activeFileName).getAbsoluteFile | ||
| if (file.exists) Some(file) else None | ||
| } | ||
| rolledOverFiles ++ activeFile | ||
| rolledOverFiles.sortBy(_.getName.stripSuffix(GZIP_LOG_SUFFIX)) ++ activeFile | ||
| } | ||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we use
GZIP_LOG_SUFFIX?There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
yes
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
oh, Utils.scala doesn't depend on
FileAppender. It does use string literal in other places. https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/util/Utils.scala#L463