-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-24787][CORE] Revert hsync in EventLoggingListener and make FsHistoryProvider to read lastBlockBeingWritten data for logs #22752
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 1 commit
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 |
|---|---|---|
|
|
@@ -34,7 +34,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore | |
| import com.google.common.io.ByteStreams | ||
| import com.google.common.util.concurrent.MoreExecutors | ||
| import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} | ||
| import org.apache.hadoop.hdfs.DistributedFileSystem | ||
| import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} | ||
| import org.apache.hadoop.hdfs.protocol.HdfsConstants | ||
| import org.apache.hadoop.security.AccessControlException | ||
| import org.fusesource.leveldbjni.internal.NativeDB | ||
|
|
@@ -129,6 +129,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) | |
|
|
||
| private val storePath = conf.get(LOCAL_STORE_DIR).map(new File(_)) | ||
| private val fastInProgressParsing = conf.get(FAST_IN_PROGRESS_PARSING) | ||
| private val inProgressAbsoluteLengthCheck = conf.get(IN_PROGRESS_ABSOLUTE_LENGTH_CHECK) | ||
|
|
||
| // Visible for testing. | ||
| private[history] val listing: KVStore = storePath.map { path => | ||
|
|
@@ -449,7 +450,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) | |
| listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = entry.getLen())) | ||
| } | ||
|
|
||
| if (info.fileSize < entry.getLen()) { | ||
| if (info.fileSize < entry.getLen() || checkAbsoluteLength(info, entry)) { | ||
| if (info.appId.isDefined && fastInProgressParsing) { | ||
| // When fast in-progress parsing is on, we don't need to re-parse when the | ||
| // size changes, but we do need to invalidate any existing UIs. | ||
|
|
@@ -541,6 +542,23 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) | |
| } | ||
| } | ||
|
|
||
| private[history] def checkAbsoluteLength(info: LogInfo, entry: FileStatus): Boolean = { | ||
|
||
| var result = false | ||
| if (inProgressAbsoluteLengthCheck && info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { | ||
| try { | ||
| result = Utils.tryWithResource(fs.open(entry.getPath)) { in => | ||
| in.getWrappedStream match { | ||
| case dfsIn: DFSInputStream => dfsIn.getFileLength > info.fileSize | ||
|
||
| } | ||
| } | ||
| } catch { | ||
| case e: Exception => | ||
| logDebug(s"Failed to check the length for the file : ${info.logPath}", e) | ||
| } | ||
| } | ||
| result | ||
| } | ||
|
|
||
| private def cleanAppData(appId: String, attemptId: Option[String], logPath: String): Unit = { | ||
| try { | ||
| val app = load(appId) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -64,4 +64,11 @@ private[spark] object config { | |
| .bytesConf(ByteUnit.BYTE) | ||
| .createWithDefaultString("1m") | ||
|
|
||
| val IN_PROGRESS_ABSOLUTE_LENGTH_CHECK = | ||
| ConfigBuilder("spark.history.fs.inProgressAbsoluteLengthCheck.enabled") | ||
|
||
| .doc("Enable to check the absolute length of the in-progress" + | ||
|
||
| " logs while considering for re-parsing.") | ||
| .booleanConf | ||
| .createWithDefault(false) | ||
|
|
||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.
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.
I think we can create a function to get the length of given file:
getFileLength(ormax(getFileLength, entry.getLen()))entry.getLen()The logic can be simpler.
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.
Have you looked @ this getFileLength() call to see how well it updates?
FwIW HADOOP-15606 proposes adding a method like this for all streams, though that proposal includes the need for specification and tests. Generally the HDFS team are a bit lax about that spec -> test workflow, which doesn't help downstream code or other implementations.
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.
Thanks @gengliangwang for looking into this. Here it doesn't need to check the checkAbsoluteLength if the FileStatus.getLen() is more than the cached fileSize, if we update to
max(getFileLength, entry.getLen()))it checks the absoluteLength always which may not be necessary.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.
Thanks @steveloughran for looking into this.
I looked at the DFSInputStream.getFileLength() api, it gives locatedBlocks.getFileLength() + lastBlockBeingWrittenLength, here locatedBlocks.getFileLength() is the value got from NameNode for all the completed blocks and lastBlockBeingWrittenLength is the lastblock lenth from DataNode which is not the completed block.
Thanks for the pointer, once this is available we can update to use it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
...there's no timetable for that getLength thing, but if HDFS already supports the API, I'm more motivated to implement it. It has benefits in cloud stores in general