[SPARK-33790][CORE] Reduce the rpc call of getFileStatus in SingleFileEventLogFileReader
### What changes were proposed in this pull request? `FsHistoryProvider#checkForLogs` already has `FileStatus` when constructing `SingleFileEventLogFileReader`, and there is no need to get the `FileStatus` again when `SingleFileEventLogFileReader#fileSizeForLastIndex`. ### Why are the changes needed? This can reduce a lot of rpc calls and improve the speed of the history server. ### Does this PR introduce _any_ user-facing change? No ### How was this patch tested? exist ut Closes #30780 from cxzl25/SPARK-33790. Authored-by: sychen <sychen@ctrip.com> Signed-off-by: Jungtaek Lim <kabhwan.opensource@gmail.com>
This commit is contained in:
parent
477046c63f
commit
0c12900120
|
@ -116,7 +116,7 @@ object EventLogFileReader {
|
|||
|
||||
def apply(fs: FileSystem, status: FileStatus): Option[EventLogFileReader] = {
|
||||
if (isSingleEventLog(status)) {
|
||||
Some(new SingleFileEventLogFileReader(fs, status.getPath))
|
||||
Some(new SingleFileEventLogFileReader(fs, status.getPath, Option(status)))
|
||||
} else if (isRollingEventLogs(status)) {
|
||||
Some(new RollingEventLogFilesFileReader(fs, status.getPath))
|
||||
} else {
|
||||
|
@ -166,8 +166,9 @@ object EventLogFileReader {
|
|||
*/
|
||||
class SingleFileEventLogFileReader(
|
||||
fs: FileSystem,
|
||||
path: Path) extends EventLogFileReader(fs, path) {
|
||||
private lazy val status = fileSystem.getFileStatus(rootPath)
|
||||
path: Path,
|
||||
maybeStatus: Option[FileStatus] = None) extends EventLogFileReader(fs, path) {
|
||||
private lazy val status = maybeStatus.getOrElse(fileSystem.getFileStatus(rootPath))
|
||||
|
||||
override def lastIndex: Option[Long] = None
|
||||
|
||||
|
|
|
@ -109,7 +109,10 @@ object MimaExcludes {
|
|||
ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.classification.BinaryLogisticRegressionSummary.weightCol"),
|
||||
|
||||
// [SPARK-32879] Pass SparkSession.Builder options explicitly to SparkSession
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SparkSession.this")
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.SparkSession.this"),
|
||||
|
||||
// [SPARK-33790][CORE] Reduce the rpc call of getFileStatus in SingleFileEventLogFileReader
|
||||
ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.history.SingleFileEventLogFileReader.this")
|
||||
)
|
||||
|
||||
// Exclude rules for 3.0.x
|
||||
|
|
Loading…
Reference in a new issue