Skip to content
Snippets Groups Projects
Commit 51b53a75 authored by Thomas Graves's avatar Thomas Graves
Browse files

[SPARK-3260] yarn - pass acls along with executor launch

Pass along the acl settings when we launch a container so that they can be applied to viewing the logs on a running NodeManager.

Author: Thomas Graves <tgraves@apache.org>

Closes #2185 from tgravescs/SPARK-3260 and squashes the following commits:

6f94b5a [Thomas Graves] make unit test more robust
28b9dd3 [Thomas Graves] yarn - pass acls along with executor launch
parent 6a37ed83
No related branches found
No related tags found
No related merge requests found
Showing
with 129 additions and 34 deletions
...@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration ...@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils}
import org.apache.spark.{SparkConf, Logging} import org.apache.spark.{SecurityManager, SparkConf, Logging}
class ExecutorRunnable( class ExecutorRunnable(
...@@ -46,7 +46,8 @@ class ExecutorRunnable( ...@@ -46,7 +46,8 @@ class ExecutorRunnable(
slaveId: String, slaveId: String,
hostname: String, hostname: String,
executorMemory: Int, executorMemory: Int,
executorCores: Int) executorCores: Int,
securityMgr: SecurityManager)
extends Runnable with ExecutorRunnableUtil with Logging { extends Runnable with ExecutorRunnableUtil with Logging {
var rpc: YarnRPC = YarnRPC.create(conf) var rpc: YarnRPC = YarnRPC.create(conf)
...@@ -86,6 +87,8 @@ class ExecutorRunnable( ...@@ -86,6 +87,8 @@ class ExecutorRunnable(
logInfo("Setting up executor with commands: " + commands) logInfo("Setting up executor with commands: " + commands)
ctx.setCommands(commands) ctx.setCommands(commands)
ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr))
// Send the start request to the ContainerManager // Send the start request to the ContainerManager
val startReq = Records.newRecord(classOf[StartContainerRequest]) val startReq = Records.newRecord(classOf[StartContainerRequest])
.asInstanceOf[StartContainerRequest] .asInstanceOf[StartContainerRequest]
......
...@@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger ...@@ -23,7 +23,7 @@ import java.util.concurrent.atomic.AtomicInteger
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.mutable.{ArrayBuffer, HashMap}
import org.apache.spark.SparkConf import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.scheduler.SplitInfo import org.apache.spark.scheduler.SplitInfo
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
...@@ -41,8 +41,9 @@ private[yarn] class YarnAllocationHandler( ...@@ -41,8 +41,9 @@ private[yarn] class YarnAllocationHandler(
resourceManager: AMRMProtocol, resourceManager: AMRMProtocol,
appAttemptId: ApplicationAttemptId, appAttemptId: ApplicationAttemptId,
args: ApplicationMasterArguments, args: ApplicationMasterArguments,
preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) preferredNodes: collection.Map[String, collection.Set[SplitInfo]],
extends YarnAllocator(conf, sparkConf, args, preferredNodes) { securityMgr: SecurityManager)
extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) {
private val lastResponseId = new AtomicInteger() private val lastResponseId = new AtomicInteger()
private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList()
......
...@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration ...@@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.hadoop.yarn.util.{ConverterUtils, Records}
import org.apache.spark.{Logging, SparkConf} import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.scheduler.SplitInfo import org.apache.spark.scheduler.SplitInfo
import org.apache.spark.util.Utils import org.apache.spark.util.Utils
...@@ -45,7 +45,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC ...@@ -45,7 +45,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC
sparkConf: SparkConf, sparkConf: SparkConf,
preferredNodeLocations: Map[String, Set[SplitInfo]], preferredNodeLocations: Map[String, Set[SplitInfo]],
uiAddress: String, uiAddress: String,
uiHistoryAddress: String) = { uiHistoryAddress: String,
securityMgr: SecurityManager) = {
this.rpc = YarnRPC.create(conf) this.rpc = YarnRPC.create(conf)
this.uiHistoryAddress = uiHistoryAddress this.uiHistoryAddress = uiHistoryAddress
...@@ -53,7 +54,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC ...@@ -53,7 +54,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC
registerApplicationMaster(uiAddress) registerApplicationMaster(uiAddress)
new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args,
preferredNodeLocations) preferredNodeLocations, securityMgr)
} }
override def getAttemptId() = { override def getAttemptId() = {
......
...@@ -116,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, ...@@ -116,7 +116,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
val securityMgr = new SecurityManager(sparkConf) val securityMgr = new SecurityManager(sparkConf)
if (isDriver) { if (isDriver) {
runDriver() runDriver(securityMgr)
} else { } else {
runExecutorLauncher(securityMgr) runExecutorLauncher(securityMgr)
} }
...@@ -157,7 +157,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, ...@@ -157,7 +157,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
sparkContextRef.compareAndSet(sc, null) sparkContextRef.compareAndSet(sc, null)
} }
private def registerAM(uiAddress: String) = { private def registerAM(uiAddress: String, securityMgr: SecurityManager) = {
val sc = sparkContextRef.get() val sc = sparkContextRef.get()
val appId = client.getAttemptId().getApplicationId().toString() val appId = client.getAttemptId().getApplicationId().toString()
...@@ -170,13 +170,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, ...@@ -170,13 +170,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
if (sc != null) sc.getConf else sparkConf, if (sc != null) sc.getConf else sparkConf,
if (sc != null) sc.preferredNodeLocationData else Map(), if (sc != null) sc.preferredNodeLocationData else Map(),
uiAddress, uiAddress,
historyAddress) historyAddress,
securityMgr)
allocator.allocateResources() allocator.allocateResources()
reporterThread = launchReporterThread() reporterThread = launchReporterThread()
} }
private def runDriver(): Unit = { private def runDriver(securityMgr: SecurityManager): Unit = {
addAmIpFilter() addAmIpFilter()
val userThread = startUserClass() val userThread = startUserClass()
...@@ -188,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, ...@@ -188,7 +189,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
if (sc == null) { if (sc == null) {
finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.") finish(FinalApplicationStatus.FAILED, "Timed out waiting for SparkContext.")
} else { } else {
registerAM(sc.ui.appUIHostPort) registerAM(sc.ui.appUIHostPort, securityMgr)
try { try {
userThread.join() userThread.join()
} finally { } finally {
...@@ -203,7 +204,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments, ...@@ -203,7 +204,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments,
conf = sparkConf, securityManager = securityMgr)._1 conf = sparkConf, securityManager = securityMgr)._1
actor = waitForSparkDriver() actor = waitForSparkDriver()
addAmIpFilter() addAmIpFilter()
registerAM(sparkConf.get("spark.driver.appUIAddress", "")) registerAM(sparkConf.get("spark.driver.appUIAddress", ""), securityMgr)
// In client mode the actor will stop the reporter thread. // In client mode the actor will stop the reporter thread.
reporterThread.join() reporterThread.join()
......
...@@ -430,10 +430,8 @@ trait ClientBase extends Logging { ...@@ -430,10 +430,8 @@ trait ClientBase extends Logging {
// send the acl settings into YARN to control who has access via YARN interfaces // send the acl settings into YARN to control who has access via YARN interfaces
val securityManager = new SecurityManager(sparkConf) val securityManager = new SecurityManager(sparkConf)
val acls = Map[ApplicationAccessType, String] ( amContainer.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityManager))
ApplicationAccessType.VIEW_APP -> securityManager.getViewAcls,
ApplicationAccessType.MODIFY_APP -> securityManager.getModifyAcls)
amContainer.setApplicationACLs(acls)
amContainer amContainer
} }
} }
......
...@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration ...@@ -28,7 +28,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse
import org.apache.spark.{Logging, SparkConf, SparkEnv} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv}
import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl} import org.apache.spark.scheduler.{SplitInfo, TaskSchedulerImpl}
import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend
...@@ -55,7 +55,8 @@ private[yarn] abstract class YarnAllocator( ...@@ -55,7 +55,8 @@ private[yarn] abstract class YarnAllocator(
conf: Configuration, conf: Configuration,
sparkConf: SparkConf, sparkConf: SparkConf,
args: ApplicationMasterArguments, args: ApplicationMasterArguments,
preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) preferredNodes: collection.Map[String, collection.Set[SplitInfo]],
securityMgr: SecurityManager)
extends Logging { extends Logging {
// These three are locked on allocatedHostToContainersMap. Complementary data structures // These three are locked on allocatedHostToContainersMap. Complementary data structures
...@@ -280,7 +281,8 @@ private[yarn] abstract class YarnAllocator( ...@@ -280,7 +281,8 @@ private[yarn] abstract class YarnAllocator(
executorId, executorId,
executorHostname, executorHostname,
executorMemory, executorMemory,
executorCores) executorCores,
securityMgr)
new Thread(executorRunnable).start() new Thread(executorRunnable).start()
} }
} }
...@@ -444,4 +446,4 @@ private[yarn] abstract class YarnAllocator( ...@@ -444,4 +446,4 @@ private[yarn] abstract class YarnAllocator(
} }
} }
\ No newline at end of file
...@@ -22,7 +22,7 @@ import scala.collection.{Map, Set} ...@@ -22,7 +22,7 @@ import scala.collection.{Map, Set}
import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.api.records._
import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.{SecurityManager, SparkConf, SparkContext}
import org.apache.spark.scheduler.SplitInfo import org.apache.spark.scheduler.SplitInfo
/** /**
...@@ -45,7 +45,8 @@ trait YarnRMClient { ...@@ -45,7 +45,8 @@ trait YarnRMClient {
sparkConf: SparkConf, sparkConf: SparkConf,
preferredNodeLocations: Map[String, Set[SplitInfo]], preferredNodeLocations: Map[String, Set[SplitInfo]],
uiAddress: String, uiAddress: String,
uiHistoryAddress: String): YarnAllocator uiHistoryAddress: String,
securityMgr: SecurityManager): YarnAllocator
/** /**
* Shuts down the AM. Guaranteed to only be called once. * Shuts down the AM. Guaranteed to only be called once.
......
...@@ -32,10 +32,11 @@ import org.apache.hadoop.security.UserGroupInformation ...@@ -32,10 +32,11 @@ import org.apache.hadoop.security.UserGroupInformation
import org.apache.hadoop.util.StringInterner import org.apache.hadoop.util.StringInterner
import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.ApplicationConstants
import org.apache.hadoop.yarn.api.records.ApplicationAccessType
import org.apache.hadoop.yarn.util.RackResolver import org.apache.hadoop.yarn.util.RackResolver
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.{SecurityManager, SparkConf, SparkContext}
import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.SparkHadoopUtil
import org.apache.spark.util.Utils import org.apache.spark.util.Utils
...@@ -211,4 +212,12 @@ object YarnSparkHadoopUtil { ...@@ -211,4 +212,12 @@ object YarnSparkHadoopUtil {
} }
} }
private[spark] def getApplicationAclsForYarn(securityMgr: SecurityManager):
Map[ApplicationAccessType, String] = {
Map[ApplicationAccessType, String] (
ApplicationAccessType.VIEW_APP -> securityMgr.getViewAcls,
ApplicationAccessType.MODIFY_APP -> securityMgr.getModifyAcls
)
}
} }
...@@ -23,7 +23,10 @@ import com.google.common.io.{ByteStreams, Files} ...@@ -23,7 +23,10 @@ import com.google.common.io.{ByteStreams, Files}
import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.scalatest.{FunSuite, Matchers} import org.scalatest.{FunSuite, Matchers}
import org.apache.spark.{Logging, SparkConf} import org.apache.hadoop.yarn.api.records.ApplicationAccessType
import org.apache.spark.{Logging, SecurityManager, SparkConf}
class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging {
...@@ -74,4 +77,75 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { ...@@ -74,4 +77,75 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging {
yarnConf.get(key) should not be default.get(key) yarnConf.get(key) should not be default.get(key)
} }
test("test getApplicationAclsForYarn acls on") {
// spark acls on, just pick up default user
val sparkConf = new SparkConf()
sparkConf.set("spark.acls.enable", "true")
val securityMgr = new SecurityManager(sparkConf)
val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
viewAcls match {
case Some(vacls) => {
val aclSet = vacls.split(',').map(_.trim).toSet
assert(aclSet.contains(System.getProperty("user.name", "invalid")))
}
case None => {
fail()
}
}
modifyAcls match {
case Some(macls) => {
val aclSet = macls.split(',').map(_.trim).toSet
assert(aclSet.contains(System.getProperty("user.name", "invalid")))
}
case None => {
fail()
}
}
}
test("test getApplicationAclsForYarn acls on and specify users") {
// default spark acls are on and specify acls
val sparkConf = new SparkConf()
sparkConf.set("spark.acls.enable", "true")
sparkConf.set("spark.ui.view.acls", "user1,user2")
sparkConf.set("spark.modify.acls", "user3,user4")
val securityMgr = new SecurityManager(sparkConf)
val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)
val viewAcls = acls.get(ApplicationAccessType.VIEW_APP)
val modifyAcls = acls.get(ApplicationAccessType.MODIFY_APP)
viewAcls match {
case Some(vacls) => {
val aclSet = vacls.split(',').map(_.trim).toSet
assert(aclSet.contains("user1"))
assert(aclSet.contains("user2"))
assert(aclSet.contains(System.getProperty("user.name", "invalid")))
}
case None => {
fail()
}
}
modifyAcls match {
case Some(macls) => {
val aclSet = macls.split(',').map(_.trim).toSet
assert(aclSet.contains("user3"))
assert(aclSet.contains("user4"))
assert(aclSet.contains(System.getProperty("user.name", "invalid")))
}
case None => {
fail()
}
}
}
} }
...@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration ...@@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.ipc.YarnRPC
import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records}
import org.apache.spark.{SparkConf, Logging} import org.apache.spark.{SecurityManager, SparkConf, Logging}
class ExecutorRunnable( class ExecutorRunnable(
...@@ -46,7 +46,8 @@ class ExecutorRunnable( ...@@ -46,7 +46,8 @@ class ExecutorRunnable(
slaveId: String, slaveId: String,
hostname: String, hostname: String,
executorMemory: Int, executorMemory: Int,
executorCores: Int) executorCores: Int,
securityMgr: SecurityManager)
extends Runnable with ExecutorRunnableUtil with Logging { extends Runnable with ExecutorRunnableUtil with Logging {
var rpc: YarnRPC = YarnRPC.create(conf) var rpc: YarnRPC = YarnRPC.create(conf)
...@@ -85,6 +86,8 @@ class ExecutorRunnable( ...@@ -85,6 +86,8 @@ class ExecutorRunnable(
logInfo("Setting up executor with commands: " + commands) logInfo("Setting up executor with commands: " + commands)
ctx.setCommands(commands) ctx.setCommands(commands)
ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr))
// Send the start request to the ContainerManager // Send the start request to the ContainerManager
nmClient.startContainer(container, ctx) nmClient.startContainer(container, ctx)
} }
......
...@@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn ...@@ -20,7 +20,7 @@ package org.apache.spark.deploy.yarn
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.collection.mutable.{ArrayBuffer, HashMap}
import org.apache.spark.SparkConf import org.apache.spark.{SecurityManager, SparkConf}
import org.apache.spark.scheduler.SplitInfo import org.apache.spark.scheduler.SplitInfo
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
...@@ -39,8 +39,9 @@ private[yarn] class YarnAllocationHandler( ...@@ -39,8 +39,9 @@ private[yarn] class YarnAllocationHandler(
amClient: AMRMClient[ContainerRequest], amClient: AMRMClient[ContainerRequest],
appAttemptId: ApplicationAttemptId, appAttemptId: ApplicationAttemptId,
args: ApplicationMasterArguments, args: ApplicationMasterArguments,
preferredNodes: collection.Map[String, collection.Set[SplitInfo]]) preferredNodes: collection.Map[String, collection.Set[SplitInfo]],
extends YarnAllocator(conf, sparkConf, args, preferredNodes) { securityMgr: SecurityManager)
extends YarnAllocator(conf, sparkConf, args, preferredNodes, securityMgr) {
override protected def releaseContainer(container: Container) = { override protected def releaseContainer(container: Container) = {
amClient.releaseAssignedContainer(container.getId()) amClient.releaseAssignedContainer(container.getId())
......
...@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration ...@@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.hadoop.yarn.util.ConverterUtils
import org.apache.hadoop.yarn.webapp.util.WebAppUtils import org.apache.hadoop.yarn.webapp.util.WebAppUtils
import org.apache.spark.{Logging, SparkConf} import org.apache.spark.{Logging, SecurityManager, SparkConf}
import org.apache.spark.scheduler.SplitInfo import org.apache.spark.scheduler.SplitInfo
import org.apache.spark.util.Utils import org.apache.spark.util.Utils
...@@ -46,7 +46,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC ...@@ -46,7 +46,8 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC
sparkConf: SparkConf, sparkConf: SparkConf,
preferredNodeLocations: Map[String, Set[SplitInfo]], preferredNodeLocations: Map[String, Set[SplitInfo]],
uiAddress: String, uiAddress: String,
uiHistoryAddress: String) = { uiHistoryAddress: String,
securityMgr: SecurityManager) = {
amClient = AMRMClient.createAMRMClient() amClient = AMRMClient.createAMRMClient()
amClient.init(conf) amClient.init(conf)
amClient.start() amClient.start()
...@@ -55,7 +56,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC ...@@ -55,7 +56,7 @@ private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMC
logInfo("Registering the ApplicationMaster") logInfo("Registering the ApplicationMaster")
amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress) amClient.registerApplicationMaster(Utils.localHostName(), 0, uiAddress)
new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args, new YarnAllocationHandler(conf, sparkConf, amClient, getAttemptId(), args,
preferredNodeLocations) preferredNodeLocations, securityMgr)
} }
override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") = override def shutdown(status: FinalApplicationStatus, diagnostics: String = "") =
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment