Skip to content

Commit

Permalink
Trying FileSystem.closeAllForUGI to prevent fscache leak with inProce…
Browse files Browse the repository at this point in the history
…ssLauncher
  • Loading branch information
risyomei committed Jun 29, 2023
1 parent c973400 commit 6e4c34d
Show file tree
Hide file tree
Showing 2 changed files with 6 additions and 0 deletions.
2 changes: 2 additions & 0 deletions core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala
Original file line number Diff line number Diff line change
Expand Up @@ -186,6 +186,8 @@ private[spark] class SparkSubmit extends Logging {
} else {
throw e
}
} finally {
FileSystem.closeAllForUGI(proxyUser)
}
}
} else {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import java.util.concurrent.{ScheduledExecutorService, TimeUnit}
import scala.collection.mutable

import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileSystem
import org.apache.hadoop.security.{Credentials, UserGroupInformation}

import org.apache.spark.SparkConf
Expand Down Expand Up @@ -149,6 +150,9 @@ private[spark] class HadoopDelegationTokenManager(
creds.addAll(newTokens)
}
})
if(!currentUser.equals(freshUGI)) {
FileSystem.closeAllForUGI(freshUGI)
}
}
}

Expand Down

0 comments on commit 6e4c34d

Please sign in to comment.