From 67070f6e2b84207ef71c8160e33d7ab4d22afc38 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 17 Oct 2023 13:10:06 -0400 Subject: [PATCH 01/34] stash --- .../filesystems/blob/BlobPathBuilder.scala | 6 +- .../TesAsyncBackendJobExecutionActor.scala | 75 ++++++++++++++++--- .../cromwell/backend/impl/tes/TesTask.scala | 43 ++++++----- ...TesAsyncBackendJobExecutionActorSpec.scala | 15 ++++ 4 files changed, 109 insertions(+), 30 deletions(-) create mode 100644 supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index 3aa26eb3c11..9f8adc99634 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -185,6 +185,10 @@ case class BlobPath private[blob](pathString: String, endpoint: EndpointURL, con * @return Path string relative to the container root. */ def pathWithoutContainer : String = pathString - + + def parseTerraWorkspaceIdFromPath: Option[String] = { + if(container.value.startsWith("sc-")) Option(container.value.substring(3)) else None + } + override def getSymlinkSafePath(options: LinkOption*): Path = toAbsolutePath } diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index ad8daca6ec7..a1d5cd9c1ff 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -1,10 +1,5 @@ package cromwell.backend.impl.tes -import common.exception.AggregatedMessageException - -import java.io.FileNotFoundException -import java.nio.file.FileAlreadyExistsException -import cats.syntax.apply._ import akka.http.scaladsl.Http import akka.http.scaladsl.marshallers.sprayjson.SprayJsonSupport._ import akka.http.scaladsl.marshalling.Marshal @@ -13,6 +8,9 @@ import akka.http.scaladsl.model._ import akka.http.scaladsl.unmarshalling.{Unmarshal, Unmarshaller} import akka.stream.ActorMaterializer import akka.util.ByteString +import cats.syntax.apply._ +import common.collections.EnhancedCollections._ +import common.exception.AggregatedMessageException import common.validation.ErrorOr.ErrorOr import common.validation.Validation._ import cromwell.backend.BackendJobLifecycleActor @@ -20,16 +18,19 @@ import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNo import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} import cromwell.core.path.{DefaultPathBuilder, Path} -import cromwell.core.retry.SimpleExponentialBackoff import cromwell.core.retry.Retry._ -import cromwell.filesystems.blob.BlobPath +import cromwell.core.retry.SimpleExponentialBackoff +import cromwell.filesystems.blob.BlobPathBuilder.ValidBlobPath +import cromwell.filesystems.blob.{BlobPath, BlobPathBuilder} import cromwell.filesystems.drs.{DrsPath, DrsResolver} -import wom.values.WomFile import net.ceedubs.ficus.Ficus._ +import wdl.draft2.model.FullyQualifiedName +import wom.values.{WomFile, _} +import java.io.FileNotFoundException +import java.nio.file.FileAlreadyExistsException import scala.concurrent.Future import scala.util.{Failure, Success} - sealed trait TesRunStatus { def isTerminal: Boolean def sysLogs: Seq[String] = Seq.empty[String] @@ -61,6 +62,9 @@ object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" } +case class LocalizedSasTokenParams(wsmEndpoint: String, blobContainer: String, workspaceId: String) + +case class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyncExecutionActorParams) extends BackendJobLifecycleActor with StandardAsyncExecutionActor with TesJobCachingActorHelper { implicit val actorSystem = context.system @@ -90,6 +94,58 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn ) } + override def scriptPreamble: String = { + super.scriptPreamble ++ getLocalizedSasTokenParams(taskInputFiles).map{ + localizedSasParams => + s""" + |blobPath="${localizedSasParams.blobContainer}" + |WORKSPACE_ID="${localizedSasParams.workspaceId}" + |WSM_ENDPOINT="${localizedSasParams.wsmEndpoint}" + |""".stripMargin + }.getOrElse("") + } + def taskInputFiles: List[Input] = { + // input files for the task. NB: Does not contain files from the instantiatedCommand + val inputFiles: Map[FullyQualifiedName, Seq[WomFile]] = + jobDescriptor.fullyQualifiedInputs.safeMapValues(_.collectAsSeq { case w: WomFile => w }) + //++ instantiatedCommand.createdFiles map { f => f.file.value.md5SumShort -> List(f.file) } + + val taskInputs: List[Input] = inputFiles.flatMap { + case (fullyQualifiedName, files) => files.flatMap(_.flattenFiles).zipWithIndex.map { + case (f, index) => + val inputType = f match { + case _: WomUnlistedDirectory => "DIRECTORY" + case _: WomSingleFile => "FILE" + case _: WomGlobFile => "FILE" + } + mapCommandLineWomFile(f).value + Input( + name = Option(fullyQualifiedName + "." + index), + description = Option(workflowDescriptor.callable.name + "." + fullyQualifiedName + "." + index), + url = Option(f.value), + path = mapCommandLineWomFile(f).value, + `type` = Option(inputType), + content = None + ) + } + }.toList + taskInputs + } + def getLocalizedSasTokenParams(inputFiles: List[Input]) : Option[LocalizedSasTokenParams] = { + val blobFiles: List[ValidBlobPath] = inputFiles.map{ + input => BlobPathBuilder.validateBlobPath(input.path) + }.collect{ + case c: BlobPathBuilder.ValidBlobPath => c + } + val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL + if(!shouldLocalizeSas || blobFiles.isEmpty) return None + val templateBlobFile = blobFiles.head + val container = templateBlobFile.container + val maybeWorkspaceId = Option("1234") + val wsmEndpoint = "1234" + maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, workspaceId)) + } + override def mapCommandLineWomFile(womFile: WomFile): WomFile = { womFile.mapFile(value => (getPath(value), asAdHocFile(womFile)) match { @@ -173,7 +229,6 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } override def executeAsync(): Future[ExecutionHandle] = { - // create call exec dir tesJobPaths.callExecutionRoot.createPermissionedDirectories() val taskMessageFuture = createTaskMessage().fold( diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala index a345e87ebf7..6125877c214 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala @@ -9,7 +9,6 @@ import net.ceedubs.ficus.Ficus._ import scala.language.postfixOps import scala.util.Try - import wdl.draft2.model.FullyQualifiedName import wdl4s.parser.MemoryUnit import wom.InstantiatedCommand @@ -17,6 +16,8 @@ import wom.callable.Callable.OutputDefinition import wom.expression.NoIoFunctionSet import wom.values._ +import scala.collection.immutable.Map + final case class WorkflowExecutionIdentityConfig(value: String) {override def toString: String = value.toString} final case class WorkflowExecutionIdentityOption(value: String) {override def toString: String = value} final case class TesTask(jobDescriptor: BackendJobDescriptor, @@ -80,24 +81,7 @@ final case class TesTask(jobDescriptor: BackendJobDescriptor, } lazy val inputs: Seq[Input] = { - val result = (callInputFiles ++ writeFunctionFiles).flatMap { - case (fullyQualifiedName, files) => files.flatMap(_.flattenFiles).zipWithIndex.map { - case (f, index) => - val inputType = f match { - case _: WomUnlistedDirectory => "DIRECTORY" - case _: WomSingleFile => "FILE" - case _: WomGlobFile => "FILE" - } - Input( - name = Option(fullyQualifiedName + "." + index), - description = Option(workflowName + "." + fullyQualifiedName + "." + index), - url = Option(f.value), - path = mapCommandLineWomFile(f).value, - `type` = Option(inputType), - content = None - ) - } - }.toList ++ Seq(commandScript) + val result = TesTask.buildTaskInputs(callInputFiles ++ writeFunctionFiles, workflowName, commandScript, mapCommandLineWomFile) jobLogger.info(s"Calculated TES inputs (found ${result.size}): " + result.mkString(System.lineSeparator(),System.lineSeparator(),System.lineSeparator())) result } @@ -287,6 +271,27 @@ object TesTask { ) } + def buildTaskInputs(taskFiles: Map[FullyQualifiedName, Seq[WomFile]], workflowName: String, commandScript: Input, womMapFn: WomFile => WomFile): Seq[Input] = { + taskFiles.flatMap { + case (fullyQualifiedName, files) => files.flatMap(_.flattenFiles).zipWithIndex.map { + case (f, index) => + val inputType = f match { + case _: WomUnlistedDirectory => "DIRECTORY" + case _: WomSingleFile => "FILE" + case _: WomGlobFile => "FILE" + } + Input( + name = Option(fullyQualifiedName + "." + index), + description = Option(workflowName + "." + fullyQualifiedName + "." + index), + url = Option(f.value), + path = womMapFn(f).value, + `type` = Option(inputType), + content = None + ) + } + }.toList ++ Seq(commandScript) + } + def makeTask(tesTask: TesTask): Task = { Task( id = None, diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala new file mode 100644 index 00000000000..62d4d9fd1d2 --- /dev/null +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -0,0 +1,15 @@ +package cromwell.backend.impl.tes + +import org.scalatest.flatspec.AnyFlatSpec +import org.scalatest.matchers.should.Matchers + + +class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers { + behavior of "TesAsyncBackendJobExecutionActor" + + it should "compile" in { + 1==1 shouldBe true + } + + if should "" +} \ No newline at end of file From 34d63e5c8e5dccbb08b4228f85f71d453f457bc5 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 17 Oct 2023 14:33:30 -0400 Subject: [PATCH 02/34] small refactor --- .../TesAsyncBackendJobExecutionActor.scala | 41 ++++--------------- .../cromwell/backend/impl/tes/TesTask.scala | 6 +-- 2 files changed, 12 insertions(+), 35 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index a1d5cd9c1ff..8bf9f4afb10 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -24,8 +24,7 @@ import cromwell.filesystems.blob.BlobPathBuilder.ValidBlobPath import cromwell.filesystems.blob.{BlobPath, BlobPathBuilder} import cromwell.filesystems.drs.{DrsPath, DrsResolver} import net.ceedubs.ficus.Ficus._ -import wdl.draft2.model.FullyQualifiedName -import wom.values.{WomFile, _} +import wom.values.WomFile import java.io.FileNotFoundException import java.nio.file.FileAlreadyExistsException @@ -95,7 +94,7 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } override def scriptPreamble: String = { - super.scriptPreamble ++ getLocalizedSasTokenParams(taskInputFiles).map{ + super.scriptPreamble ++ getLocalizedSasTokenParams.map{ localizedSasParams => s""" |blobPath="${localizedSasParams.blobContainer}" @@ -104,35 +103,13 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn |""".stripMargin }.getOrElse("") } - def taskInputFiles: List[Input] = { - // input files for the task. NB: Does not contain files from the instantiatedCommand - val inputFiles: Map[FullyQualifiedName, Seq[WomFile]] = - jobDescriptor.fullyQualifiedInputs.safeMapValues(_.collectAsSeq { case w: WomFile => w }) - //++ instantiatedCommand.createdFiles map { f => f.file.value.md5SumShort -> List(f.file) } - - val taskInputs: List[Input] = inputFiles.flatMap { - case (fullyQualifiedName, files) => files.flatMap(_.flattenFiles).zipWithIndex.map { - case (f, index) => - val inputType = f match { - case _: WomUnlistedDirectory => "DIRECTORY" - case _: WomSingleFile => "FILE" - case _: WomGlobFile => "FILE" - } - mapCommandLineWomFile(f).value - Input( - name = Option(fullyQualifiedName + "." + index), - description = Option(workflowDescriptor.callable.name + "." + fullyQualifiedName + "." + index), - url = Option(f.value), - path = mapCommandLineWomFile(f).value, - `type` = Option(inputType), - content = None - ) - } - }.toList - taskInputs - } - def getLocalizedSasTokenParams(inputFiles: List[Input]) : Option[LocalizedSasTokenParams] = { - val blobFiles: List[ValidBlobPath] = inputFiles.map{ + + def getLocalizedSasTokenParams : Option[LocalizedSasTokenParams] = { + val workflowName = workflowDescriptor.callable.name + val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { _.collectAsSeq { case w: WomFile => w }} + val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) + + val blobFiles: List[ValidBlobPath] = taskInputs.map{ input => BlobPathBuilder.validateBlobPath(input.path) }.collect{ case c: BlobPathBuilder.ValidBlobPath => c diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala index 4ebe0498118..d775367ac74 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesTask.scala @@ -81,7 +81,7 @@ final case class TesTask(jobDescriptor: BackendJobDescriptor, } lazy val inputs: Seq[Input] = { - val result = TesTask.buildTaskInputs(callInputFiles ++ writeFunctionFiles, workflowName, commandScript, mapCommandLineWomFile) + val result = TesTask.buildTaskInputs(callInputFiles ++ writeFunctionFiles, workflowName, mapCommandLineWomFile) ++ Seq(commandScript) jobLogger.info(s"Calculated TES inputs (found ${result.size}): " + result.mkString(System.lineSeparator(),System.lineSeparator(),System.lineSeparator())) result } @@ -273,7 +273,7 @@ object TesTask { ) } - def buildTaskInputs(taskFiles: Map[FullyQualifiedName, Seq[WomFile]], workflowName: String, commandScript: Input, womMapFn: WomFile => WomFile): Seq[Input] = { + def buildTaskInputs(taskFiles: Map[FullyQualifiedName, Seq[WomFile]], workflowName: String, womMapFn: WomFile => WomFile): List[Input] = { taskFiles.flatMap { case (fullyQualifiedName, files) => files.flatMap(_.flattenFiles).zipWithIndex.map { case (f, index) => @@ -291,7 +291,7 @@ object TesTask { content = None ) } - }.toList ++ Seq(commandScript) + }.toList } def makeTags(workflowDescriptor: BackendWorkflowDescriptor): Map[String, Option[String]] = { From 0d279eb0b82aa8531e4794cd8582725208a16e31 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 17 Oct 2023 16:27:11 -0400 Subject: [PATCH 03/34] progress --- .../blob/BlobFileSystemManager.scala | 12 ++++++++---- .../filesystems/blob/BlobPathBuilder.scala | 18 ++++++++++++++++-- .../blob/BlobPathBuilderFactory.scala | 1 + .../tes/TesAsyncBackendJobExecutionActor.scala | 15 +++++++++++---- 4 files changed, 36 insertions(+), 10 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index e3de6783d85..969331fd458 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -178,10 +178,7 @@ case class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClient * @return an AzureSasCredential for accessing a blob container */ def generateBlobSasToken(endpoint: EndpointURL, container: BlobContainerName): Try[AzureSasCredential] = { - val wsmAuthToken: Try[String] = overrideWsmAuthToken match { - case Some(t) => Success(t) - case None => AzureCredentials.getAccessToken(None).toTry - } + val wsmAuthToken: Try[String] = getWsmAuth container.workspaceId match { // If this is a Terra workspace, request a token from WSM case Success(workspaceId) => { @@ -205,6 +202,13 @@ case class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClient val wsmResourceClient = wsmClientProvider.getResourceApi(wsmAuth) wsmResourceClient.findContainerResourceId(workspaceId, container) } + + def getWsmAuth: Try[String] = { + overrideWsmAuthToken match { + case Some(t) => Success(t) + case None => AzureCredentials.getAccessToken(None).toTry + } + } } case class NativeBlobSasTokenGenerator(subscription: Option[SubscriptionId] = None) extends BlobSasTokenGenerator { diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index 9f8adc99634..c70b3b8711c 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -7,6 +7,7 @@ import cromwell.filesystems.blob.BlobPathBuilder._ import java.net.{MalformedURLException, URI} import java.nio.file.{Files, LinkOption} +import java.util.UUID import scala.jdk.CollectionConverters._ import scala.language.postfixOps import scala.util.{Failure, Success, Try} @@ -186,9 +187,22 @@ case class BlobPath private[blob](pathString: String, endpoint: EndpointURL, con */ def pathWithoutContainer : String = pathString - def parseTerraWorkspaceIdFromPath: Option[String] = { - if(container.value.startsWith("sc-")) Option(container.value.substring(3)) else None + def parseTerraWorkspaceIdFromPath: Try[UUID] = { + if(container.value.startsWith("sc-")) Try(UUID.fromString(container.value.substring(3))) else Failure(new Exception("Could not parse workspace ID from storage container")) + } + + def containerWSMResourceId: Try[UUID] = { + + val wsmGenerator: Option[WSMBlobSasTokenGenerator] = fsm.blobTokenGenerator match { + case wsmGenerator: WSMBlobSasTokenGenerator => Option(wsmGenerator) + case _: Any => None + } + val workspaceId: Try[UUID] = parseTerraWorkspaceIdFromPath + val wsmAuth: Try[String] = wsmGenerator.get.getWsmAuth + + Try(wsmGenerator.get.getContainerResourceId(workspaceId.get, container, wsmAuth.get)).flatten } override def getSymlinkSafePath(options: LinkOption*): Path = toAbsolutePath + } diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala index 47245552dc2..c6921338799 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala @@ -29,6 +29,7 @@ final case class EndpointURL(value: String) { } } final case class WorkspaceId(value: UUID) {override def toString: String = value.toString} + final case class ContainerResourceId(value: UUID) {override def toString: String = value.toString} final case class WorkspaceManagerURL(value: String) {override def toString: String = value} diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 8bf9f4afb10..d323929aad1 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -29,7 +29,7 @@ import wom.values.WomFile import java.io.FileNotFoundException import java.nio.file.FileAlreadyExistsException import scala.concurrent.Future -import scala.util.{Failure, Success} +import scala.util.{Failure, Success, Try} sealed trait TesRunStatus { def isTerminal: Boolean def sysLogs: Seq[String] = Seq.empty[String] @@ -74,7 +74,6 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn override type StandardAsyncRunState = TesRunStatus def statusEquivalentTo(thiz: StandardAsyncRunState)(that: StandardAsyncRunState): Boolean = thiz == that - override lazy val pollBackOff: SimpleExponentialBackoff = tesConfiguration.pollBackoff override lazy val executeOrRecoverBackOff: SimpleExponentialBackoff = tesConfiguration.executeOrRecoverBackoff @@ -105,6 +104,7 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } def getLocalizedSasTokenParams : Option[LocalizedSasTokenParams] = { + val workflowName = workflowDescriptor.callable.name val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { _.collectAsSeq { case w: WomFile => w }} val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) @@ -114,13 +114,20 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn }.collect{ case c: BlobPathBuilder.ValidBlobPath => c } + val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL if(!shouldLocalizeSas || blobFiles.isEmpty) return None val templateBlobFile = blobFiles.head + val initialPath: Try[Path] = getPath(templateBlobFile.path) + val blobPath: Option[BlobPath] = initialPath.get match { + case blob: BlobPath => Option(blob) + case _: Any => None + } + val container = templateBlobFile.container - val maybeWorkspaceId = Option("1234") + val maybeWorkspaceId = blobPath val wsmEndpoint = "1234" - maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, workspaceId)) + maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, blobPath.get.containerWSMResourceId.get.toString)) } override def mapCommandLineWomFile(womFile: WomFile): WomFile = { From 06531d7a313da81f1f36502b838d452ff36501ad Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Thu, 19 Oct 2023 15:03:41 -0400 Subject: [PATCH 04/34] blobular --- .../filesystems/blob/BlobPathBuilder.scala | 4 +- .../TesAsyncBackendJobExecutionActor.scala | 76 ++++++++++-------- ...TesAsyncBackendJobExecutionActorSpec.scala | 77 ++++++++++++++++++- 3 files changed, 119 insertions(+), 38 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index c70b3b8711c..344b98f0337 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -15,7 +15,9 @@ import scala.util.{Failure, Success, Try} object BlobPathBuilder { private val blobHostnameSuffix = ".blob.core.windows.net" sealed trait BlobPathValidation - case class ValidBlobPath(path: String, container: BlobContainerName, endpoint: EndpointURL) extends BlobPathValidation + case class ValidBlobPath(path: String, container: BlobContainerName, endpoint: EndpointURL) extends BlobPathValidation { + def toUrl: String = endpoint.value + "/" + container.value + path + } case class UnparsableBlobPath(errorMessage: Throwable) extends BlobPathValidation def invalidBlobHostMessage(endpoint: EndpointURL) = s"Malformed Blob URL for this builder: The endpoint $endpoint doesn't contain the expected host string '{SA}.blob.core.windows.net/'" diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index d323929aad1..5654ffcc4fc 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -15,6 +15,7 @@ import common.validation.ErrorOr.ErrorOr import common.validation.Validation._ import cromwell.backend.BackendJobLifecycleActor import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNonRetryableExecutionHandle, PendingExecutionHandle} +import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{generateLocalizedSasScriptPreammble, getLocalizedSasTokenParams} import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} import cromwell.core.path.{DefaultPathBuilder, Path} @@ -59,9 +60,43 @@ case object Cancelled extends TesRunStatus { object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" + def generateLocalizedSasScriptPreammble(sasParams: LocalizedSasTokenParams) : String = { + s""" + |WSM_ENDPOINT="${sasParams.wsmEndpoint}" + |WORKSPACE_ID="${sasParams.workspaceId}" + |CONTAINER_RESOURCE_ID="${sasParams.containerResourceId}" + |""".stripMargin + } + def getLocalizedSasTokenParams(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[LocalizedSasTokenParams] = { + val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL + if (!shouldLocalizeSas || taskInputs.isEmpty) return None + + val templateInput = taskInputs.head + val templateValidatedBlob = BlobPathBuilder.validateBlobPath(templateInput.url.getOrElse("NotValid")) + if(templateValidatedBlob.equals("")) return None + + val blobFiles: List[ValidBlobPath] = taskInputs.map { + input => BlobPathBuilder.validateBlobPath(input.url.getOrElse("NotValid")) + }.collect { + case c: BlobPathBuilder.ValidBlobPath => c + } + + val templateBlobFile = blobFiles.head + val url = templateBlobFile.toUrl + val initialPath: Try[Path] = pathGetter(url) + val blobPath: Option[BlobPath] = initialPath.get match { + case blob: BlobPath => Option(blob) + case _: Any => None + } + + val container = templateBlobFile.container + val maybeWorkspaceId = blobPath + val wsmEndpoint = "1234" + maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, blobPath.get.containerWSMResourceId.get.toString)) + } } -case class LocalizedSasTokenParams(wsmEndpoint: String, blobContainer: String, workspaceId: String) +case class LocalizedSasTokenParams(wsmEndpoint: String, workspaceId: String, containerResourceId: String) case class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyncExecutionActorParams) @@ -93,41 +128,14 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } override def scriptPreamble: String = { - super.scriptPreamble ++ getLocalizedSasTokenParams.map{ - localizedSasParams => - s""" - |blobPath="${localizedSasParams.blobContainer}" - |WORKSPACE_ID="${localizedSasParams.workspaceId}" - |WSM_ENDPOINT="${localizedSasParams.wsmEndpoint}" - |""".stripMargin - }.getOrElse("") - } - - def getLocalizedSasTokenParams : Option[LocalizedSasTokenParams] = { - val workflowName = workflowDescriptor.callable.name - val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { _.collectAsSeq { case w: WomFile => w }} - val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) - - val blobFiles: List[ValidBlobPath] = taskInputs.map{ - input => BlobPathBuilder.validateBlobPath(input.path) - }.collect{ - case c: BlobPathBuilder.ValidBlobPath => c + val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { + _.collectAsSeq { case w: WomFile => w } } - - val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL - if(!shouldLocalizeSas || blobFiles.isEmpty) return None - val templateBlobFile = blobFiles.head - val initialPath: Try[Path] = getPath(templateBlobFile.path) - val blobPath: Option[BlobPath] = initialPath.get match { - case blob: BlobPath => Option(blob) - case _: Any => None - } - - val container = templateBlobFile.container - val maybeWorkspaceId = blobPath - val wsmEndpoint = "1234" - maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, blobPath.get.containerWSMResourceId.get.toString)) + val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) + super.scriptPreamble ++ getLocalizedSasTokenParams(taskInputs, getPath).map{ + sasParams => generateLocalizedSasScriptPreammble(sasParams) + }.getOrElse("") } override def mapCommandLineWomFile(womFile: WomFile): WomFile = { diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 62d4d9fd1d2..4a2d9b1d5b6 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -1,15 +1,86 @@ package cromwell.backend.impl.tes +import com.fasterxml.jackson.databind.ext.NioPathSerializer +import cromwell.core.path.{NioPath, Path} +import cromwell.filesystems.blob.{BlobPath, BlobPathBuilderFactory, EndpointURL} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import scala.tools.nsc.io.Path + class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers { behavior of "TesAsyncBackendJobExecutionActor" - it should "compile" in { - 1==1 shouldBe true + val fullyQualifiedName = "this.name.is.more.than.qualified" + val workflowName = "mockWorkflow" + val someBlobUrl = "https://lz813a3d637adefec2c6e88f.blob.core.windows.net/sc-d8143fd8-aa07-446d-9ba0-af72203f1794/nyxp6c/tes-internal/configuration/supported-vm-sizes?sv=sasToken" + val someNotBlobUrl = "https://www.google.com/path/to/exile" + var index = 0 + + val blobInput_0 = Input( + name = Option(fullyQualifiedName + "." + index), + description = Option(workflowName + "." + fullyQualifiedName + "." + index), + url = Option(someBlobUrl), + path = someBlobUrl, + `type` = Option("FILE"), + content = None + ) + index = index+1 + + val blobInput_1 = Input( + name = Option(fullyQualifiedName + "." + index), + description = Option(workflowName + "." + fullyQualifiedName + "." + index), + url = Option(someBlobUrl), + path = someBlobUrl, + `type` = Option("FILE"), + content = None + ) + index = index+1 + + val notBlobInput_1 = Input( + name = Option(fullyQualifiedName + "." + index), + description = Option(workflowName + "." + fullyQualifiedName + "." + index), + url = Option(someNotBlobUrl + index), + path = someNotBlobUrl + index, + `type` = Option("FILE"), + content = None + ) + index = index+1 + + val notBlobInput_2 = Input( + name = Option(fullyQualifiedName + "." + index), + description = Option(workflowName + "." + fullyQualifiedName + "." + index), + url = Option(someNotBlobUrl + index), + path = someNotBlobUrl + index, + `type` = Option("FILE"), + content = None + ) + + def pathGetter(pathString: String): Path = { + /* + val factory: BlobPathBuilderFactory = BlobPathBuilderFactory() + val nioPath: java.nio.file.Path = NioPathSerializer() + val endpoint: EndpointURL = EndpointURL("www.some.blob.endpoint/url") + if(pathString.startsWith(someBlobUrl)) BlobPath("someBlobPath",endpoint,) + */ + } + + it should "not generate sas params when no blob paths are provided" in { + val inputs = List[Input] + //TesAsyncBackendJobExecutionActor.getLocalizedSasTokenParams(inputs,) } - if should "" + it should "generate proper script preamble" in { + val mockEndpoint = "www.workspacemanager.com" + val mockWorkspaceId = "1111-2222-3333-4444" + val mockContainerId = "5678-who-do-we-appreciate" + val mockSasParams: LocalizedSasTokenParams = LocalizedSasTokenParams(mockEndpoint, mockWorkspaceId, mockContainerId) + TesAsyncBackendJobExecutionActor.generateLocalizedSasScriptPreammble(mockSasParams) shouldBe + s""" + |WSM_ENDPOINT="${mockEndpoint}" + |WORKSPACE_ID="${mockWorkspaceId}" + |CONTAINER_RESOURCE_ID="${mockContainerId}" + |""".stripMargin + } } \ No newline at end of file From aa1bc529b815f045e94ddf9f8b32c89e0f9db1ae Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Thu, 19 Oct 2023 15:53:40 -0400 Subject: [PATCH 05/34] got WSM endpoint --- .../filesystems/blob/BlobPathBuilder.scala | 17 +++++++++++++++-- .../WorkspaceManagerApiClientProvider.scala | 5 +++++ .../tes/TesAsyncBackendJobExecutionActor.scala | 12 +++++++++++- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index 344b98f0337..b613bb43fe9 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -194,9 +194,8 @@ case class BlobPath private[blob](pathString: String, endpoint: EndpointURL, con } def containerWSMResourceId: Try[UUID] = { - val wsmGenerator: Option[WSMBlobSasTokenGenerator] = fsm.blobTokenGenerator match { - case wsmGenerator: WSMBlobSasTokenGenerator => Option(wsmGenerator) + case wsmGenerator: WSMBlobSasTokenGenerator => Option(wsmGenerator) case _: Any => None } val workspaceId: Try[UUID] = parseTerraWorkspaceIdFromPath @@ -205,6 +204,20 @@ case class BlobPath private[blob](pathString: String, endpoint: EndpointURL, con Try(wsmGenerator.get.getContainerResourceId(workspaceId.get, container, wsmAuth.get)).flatten } + def wsmEndpoint: Try[String] = { + val wsmGenerator: Option[WSMBlobSasTokenGenerator] = fsm.blobTokenGenerator match { + case wsmGenerator: WSMBlobSasTokenGenerator => Option(wsmGenerator) + case _: Any => None + } + val maybeEndpoint = wsmGenerator.map{generator => + generator.wsmClientProvider.getBaseWorkspaceManagerUrl + } + maybeEndpoint match { + case endpoint: Some[String] => Try(endpoint.value) + case _ => Failure(new NoSuchElementException("Could not determine WSM API endpoint.")) + } + } + override def getSymlinkSafePath(options: LinkOption*): Path = toAbsolutePath } diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala index 276738c98b6..4d09cafc96e 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala @@ -20,6 +20,7 @@ import scala.util.Try trait WorkspaceManagerApiClientProvider { def getControlledAzureResourceApi(token: String): WsmControlledAzureResourceApi def getResourceApi(token: String): WsmResourceApi + def getBaseWorkspaceManagerUrl: String } class HttpWorkspaceManagerClientProvider(baseWorkspaceManagerUrl: WorkspaceManagerURL) extends WorkspaceManagerApiClientProvider { @@ -40,6 +41,10 @@ class HttpWorkspaceManagerClientProvider(baseWorkspaceManagerUrl: WorkspaceManag apiClient.setAccessToken(token) WsmControlledAzureResourceApi(new ControlledAzureResourceApi(apiClient)) } + + def getBaseWorkspaceManagerUrl: String = { + baseWorkspaceManagerUrl.value + } } case class WsmResourceApi(resourcesApi : ResourceApi) { diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 5654ffcc4fc..ae2517fc8da 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -29,6 +29,7 @@ import wom.values.WomFile import java.io.FileNotFoundException import java.nio.file.FileAlreadyExistsException +import java.util.UUID import scala.concurrent.Future import scala.util.{Failure, Success, Try} sealed trait TesRunStatus { @@ -67,6 +68,12 @@ object TesAsyncBackendJobExecutionActor { |CONTAINER_RESOURCE_ID="${sasParams.containerResourceId}" |""".stripMargin } + + /* Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a + * fresh SaS token for itself. In order to be able to do this, we must provide the task execution script with a + * WSM endpoint, WorkspaceID, and container resource ID. The task VM will use the user assigned managed identity that + * it is running as in order to authenticate. + */ def getLocalizedSasTokenParams(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[LocalizedSasTokenParams] = { val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL if (!shouldLocalizeSas || taskInputs.isEmpty) return None @@ -89,9 +96,12 @@ object TesAsyncBackendJobExecutionActor { case _: Any => None } + val uuid: Try[UUID] = blobPath.get.containerWSMResourceId + if(!uuid.isSuccess) return None val container = templateBlobFile.container val maybeWorkspaceId = blobPath - val wsmEndpoint = "1234" + val tryWsm = blobPath.get.wsmEndpoint + val wsmEndpoint = tryWsm.getOrElse("invalid") maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, blobPath.get.containerWSMResourceId.get.toString)) } } From 8d3a2e707b78a3875eb636b2195a374f54a0b45d Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Thu, 19 Oct 2023 17:04:25 -0400 Subject: [PATCH 06/34] tidy --- .../filesystems/blob/BlobPathBuilder.scala | 32 +++++------- .../TesAsyncBackendJobExecutionActor.scala | 50 +++++++++---------- 2 files changed, 38 insertions(+), 44 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index b613bb43fe9..98fb7eb7db7 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -193,29 +193,23 @@ case class BlobPath private[blob](pathString: String, endpoint: EndpointURL, con if(container.value.startsWith("sc-")) Try(UUID.fromString(container.value.substring(3))) else Failure(new Exception("Could not parse workspace ID from storage container")) } - def containerWSMResourceId: Try[UUID] = { - val wsmGenerator: Option[WSMBlobSasTokenGenerator] = fsm.blobTokenGenerator match { - case wsmGenerator: WSMBlobSasTokenGenerator => Option(wsmGenerator) - case _: Any => None + private def getWSMTokenGenerator: Try[WSMBlobSasTokenGenerator] = { + fsm.blobTokenGenerator match { + case wsmGenerator: WSMBlobSasTokenGenerator => Try(wsmGenerator) + case _: Any => Failure(new NoSuchElementException("This blob file does not have an associated WSMBlobSasTokenGenerator")) } - val workspaceId: Try[UUID] = parseTerraWorkspaceIdFromPath - val wsmAuth: Try[String] = wsmGenerator.get.getWsmAuth - - Try(wsmGenerator.get.getContainerResourceId(workspaceId.get, container, wsmAuth.get)).flatten + } + def containerWSMResourceId: Try[UUID] = { + for { + generator <- getWSMTokenGenerator + workspaceId <- parseTerraWorkspaceIdFromPath + wsmAuth <- generator.getWsmAuth + resourceId <- generator.getContainerResourceId(workspaceId, container, wsmAuth) + } yield resourceId } def wsmEndpoint: Try[String] = { - val wsmGenerator: Option[WSMBlobSasTokenGenerator] = fsm.blobTokenGenerator match { - case wsmGenerator: WSMBlobSasTokenGenerator => Option(wsmGenerator) - case _: Any => None - } - val maybeEndpoint = wsmGenerator.map{generator => - generator.wsmClientProvider.getBaseWorkspaceManagerUrl - } - maybeEndpoint match { - case endpoint: Some[String] => Try(endpoint.value) - case _ => Failure(new NoSuchElementException("Could not determine WSM API endpoint.")) - } + getWSMTokenGenerator.map(generator => generator.wsmClientProvider.getBaseWorkspaceManagerUrl) } override def getSymlinkSafePath(options: LinkOption*): Path = toAbsolutePath diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index ae2517fc8da..e5edcd20439 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -70,39 +70,39 @@ object TesAsyncBackendJobExecutionActor { } /* Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a - * fresh SaS token for itself. In order to be able to do this, we must provide the task execution script with a - * WSM endpoint, WorkspaceID, and container resource ID. The task VM will use the user assigned managed identity that - * it is running as in order to authenticate. + * fresh sas token for itself. In order to be able to do this, we provide it with a precomputed endpoint it can use. + * This endpoint will contain the WSM root, WorkspaceID, and container resource ID. + * The task VM will use the user assigned managed identity that it is running as in order to authenticate. */ - def getLocalizedSasTokenParams(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[LocalizedSasTokenParams] = { + def getLocalizedSasTokenParams(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[String] = { val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL - if (!shouldLocalizeSas || taskInputs.isEmpty) return None + if (!shouldLocalizeSas) return None - val templateInput = taskInputs.head - val templateValidatedBlob = BlobPathBuilder.validateBlobPath(templateInput.url.getOrElse("NotValid")) - if(templateValidatedBlob.equals("")) return None + val blobFiles = taskInputs.collect{ //Collect all inputs with URLs defined as (in)valid blob paths + case input if input.url.isDefined => BlobPathBuilder.validateBlobPath(input.url.get) + }.collect{ //Collect only the valid blob paths + case valid: BlobPathBuilder.ValidBlobPath => valid + } - val blobFiles: List[ValidBlobPath] = taskInputs.map { - input => BlobPathBuilder.validateBlobPath(input.url.getOrElse("NotValid")) - }.collect { - case c: BlobPathBuilder.ValidBlobPath => c + if(blobFiles.isEmpty) return None + //We use the first blob file in the list as a template for determining the localized sas params + val blobPath: Try[BlobPath] = pathGetter(blobFiles.head.toUrl).getOrElse(None) match { + case blob: BlobPath => Try(blob) + case _: Any => Failure(new UnsupportedOperationException("Could not convert path into Blob path")) } - val templateBlobFile = blobFiles.head - val url = templateBlobFile.toUrl - val initialPath: Try[Path] = pathGetter(url) - val blobPath: Option[BlobPath] = initialPath.get match { - case blob: BlobPath => Option(blob) + val sasTokenEndpoint = for { + blob <- blobPath + wsmEndpoint <- blob.wsmEndpoint + workspaceId <- blob.parseTerraWorkspaceIdFromPath + containerResourceId <- blob.containerWSMResourceId + endpoint = s"$wsmEndpoint/$workspaceId/resources/controlled/azure/storageContainer/$containerResourceId/getSasToken" + } yield endpoint + + sasTokenEndpoint match { + case good: Success[String] => Some(good.value) case _: Any => None } - - val uuid: Try[UUID] = blobPath.get.containerWSMResourceId - if(!uuid.isSuccess) return None - val container = templateBlobFile.container - val maybeWorkspaceId = blobPath - val tryWsm = blobPath.get.wsmEndpoint - val wsmEndpoint = tryWsm.getOrElse("invalid") - maybeWorkspaceId.map(workspaceId => LocalizedSasTokenParams(wsmEndpoint, container.value, blobPath.get.containerWSMResourceId.get.toString)) } } From 678f05c11af5a4c716091580e591a5e7c74b6218 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Fri, 20 Oct 2023 17:15:33 -0400 Subject: [PATCH 07/34] mostly there --- .../TesAsyncBackendJobExecutionActor.scala | 41 ++++++++++++------- 1 file changed, 27 insertions(+), 14 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index e5edcd20439..1587a972ae0 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -15,13 +15,12 @@ import common.validation.ErrorOr.ErrorOr import common.validation.Validation._ import cromwell.backend.BackendJobLifecycleActor import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNonRetryableExecutionHandle, PendingExecutionHandle} -import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{generateLocalizedSasScriptPreammble, getLocalizedSasTokenParams} +import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{generateLocalizedSasScriptPreammble, determineWSMSasEndpointFromInputs} import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} import cromwell.core.path.{DefaultPathBuilder, Path} import cromwell.core.retry.Retry._ import cromwell.core.retry.SimpleExponentialBackoff -import cromwell.filesystems.blob.BlobPathBuilder.ValidBlobPath import cromwell.filesystems.blob.{BlobPath, BlobPathBuilder} import cromwell.filesystems.drs.{DrsPath, DrsResolver} import net.ceedubs.ficus.Ficus._ @@ -29,7 +28,6 @@ import wom.values.WomFile import java.io.FileNotFoundException import java.nio.file.FileAlreadyExistsException -import java.util.UUID import scala.concurrent.Future import scala.util.{Failure, Success, Try} sealed trait TesRunStatus { @@ -61,20 +59,36 @@ case object Cancelled extends TesRunStatus { object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" - def generateLocalizedSasScriptPreammble(sasParams: LocalizedSasTokenParams) : String = { + def generateLocalizedSasScriptPreammble(getSasWsmEndpoint: String) : String = { + // BEARER_TOKEN: https://learn.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#get-a-token-using-http + // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. + // NB: For easier debugging/logging, we echo the first 4 characters of the acquired sas token. If something goes wrong with the curl, these will be "null" s""" - |WSM_ENDPOINT="${sasParams.wsmEndpoint}" - |WORKSPACE_ID="${sasParams.workspaceId}" - |CONTAINER_RESOURCE_ID="${sasParams.containerResourceId}" + |apt-get install jq + |apt-get install curl + |BEARER_TOKEN=$$(curl 'http://169.254.169.254/metadata/identity/oauth2/token?api-version=2018-02-01&resource=https%3A%2F%2Fmanagement.azure.com%2F' -H Metadata:true -s | jq .access_token) + |echo "Acquired bearer token:" + |echo $${BEARER_TOKEN:0:4} + |GET_SAS_ENDPOINT=$getSasWsmEndpoint + |sas_response_json=$$(curl -s \\ + | --retry 3 \\ + | --retry-delay 2 \\ + | -X POST "$$GET_SAS_ENDPOINT" \\ + | -H "Content-Type: application/json" \\ + | -H "accept: */*" \\ + | -H "Authorization: Bearer $${BEARER_TOKEN}") + |export AZURE_STORAGE_SAS_TOKEN=$$(echo "$${sas_response_json}" | jq -r '.token') + |echo Acquired sas token: + |echo $${AZURE_STORAGE_SAS_TOKEN:0:4} |""".stripMargin } /* Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a * fresh sas token for itself. In order to be able to do this, we provide it with a precomputed endpoint it can use. - * This endpoint will contain the WSM root, WorkspaceID, and container resource ID. * The task VM will use the user assigned managed identity that it is running as in order to authenticate. + * We only return a value if //TODO and if at least one blob storage file is provided as a task input. */ - def getLocalizedSasTokenParams(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[String] = { + def determineWSMSasEndpointFromInputs(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[String] = { val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL if (!shouldLocalizeSas) return None @@ -86,9 +100,8 @@ object TesAsyncBackendJobExecutionActor { if(blobFiles.isEmpty) return None //We use the first blob file in the list as a template for determining the localized sas params - val blobPath: Try[BlobPath] = pathGetter(blobFiles.head.toUrl).getOrElse(None) match { - case blob: BlobPath => Try(blob) - case _: Any => Failure(new UnsupportedOperationException("Could not convert path into Blob path")) + val blobPath: Try[BlobPath] = pathGetter(blobFiles.head.toUrl).collect{ + case blob: BlobPath => blob } val sasTokenEndpoint = for { @@ -143,8 +156,8 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn _.collectAsSeq { case w: WomFile => w } } val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) - super.scriptPreamble ++ getLocalizedSasTokenParams(taskInputs, getPath).map{ - sasParams => generateLocalizedSasScriptPreammble(sasParams) + super.scriptPreamble ++ determineWSMSasEndpointFromInputs(taskInputs, getPath).map{ + endpoint => generateLocalizedSasScriptPreammble(endpoint) }.getOrElse("") } From 9991e5aec03224918b929c08a7cdaa8b8da0b2ff Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 23 Oct 2023 11:40:36 -0400 Subject: [PATCH 08/34] working --- .../TesAsyncBackendJobExecutionActor.scala | 35 ++++++++++++------- ...TesAsyncBackendJobExecutionActorSpec.scala | 2 +- 2 files changed, 24 insertions(+), 13 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 1587a972ae0..dda10d6c898 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -15,7 +15,7 @@ import common.validation.ErrorOr.ErrorOr import common.validation.Validation._ import cromwell.backend.BackendJobLifecycleActor import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNonRetryableExecutionHandle, PendingExecutionHandle} -import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{generateLocalizedSasScriptPreammble, determineWSMSasEndpointFromInputs} +import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{generateLocalizedSasScriptPreamble, determineWSMSasEndpointFromInputs} import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} import cromwell.core.path.{DefaultPathBuilder, Path} @@ -59,16 +59,25 @@ case object Cancelled extends TesRunStatus { object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" - def generateLocalizedSasScriptPreammble(getSasWsmEndpoint: String) : String = { + def generateLocalizedSasScriptPreamble(getSasWsmEndpoint: String) : String = { // BEARER_TOKEN: https://learn.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#get-a-token-using-http // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. // NB: For easier debugging/logging, we echo the first 4 characters of the acquired sas token. If something goes wrong with the curl, these will be "null" s""" - |apt-get install jq - |apt-get install curl + |### BEGIN ACQUIRE LOCAL SAS TOKEN ### + |# Install dependencies + |apt-get -y update + |apt-get -y install curl + |apt-get -y install jq + | + |# Acquire bearer token, relying on the User Assigned Managed Identity of this VM. |BEARER_TOKEN=$$(curl 'http://169.254.169.254/metadata/identity/oauth2/token?api-version=2018-02-01&resource=https%3A%2F%2Fmanagement.azure.com%2F' -H Metadata:true -s | jq .access_token) - |echo "Acquired bearer token:" - |echo $${BEARER_TOKEN:0:4} + | + |# Remove the leading and trailing quotes + |BEARER_TOKEN="$${BEARER_TOKEN#\\"}" + |BEARER_TOKEN="$${BEARER_TOKEN%\\"}" + | + |# Use the precomputed endpoint from cromwell + WSM to acquire a sas token |GET_SAS_ENDPOINT=$getSasWsmEndpoint |sas_response_json=$$(curl -s \\ | --retry 3 \\ @@ -77,9 +86,13 @@ object TesAsyncBackendJobExecutionActor { | -H "Content-Type: application/json" \\ | -H "accept: */*" \\ | -H "Authorization: Bearer $${BEARER_TOKEN}") + | + |# Store token as environment variable |export AZURE_STORAGE_SAS_TOKEN=$$(echo "$${sas_response_json}" | jq -r '.token') - |echo Acquired sas token: - |echo $${AZURE_STORAGE_SAS_TOKEN:0:4} + | + |# Echo the first 3 characters for logging/debugging purposes. We expect them to be sv= if everything went well. + |echo Acquired sas token: "$${AZURE_STORAGE_SAS_TOKEN:0:3}****" + |### END ACQUIRE LOCAL SAS TOKEN ### |""".stripMargin } @@ -109,7 +122,7 @@ object TesAsyncBackendJobExecutionActor { wsmEndpoint <- blob.wsmEndpoint workspaceId <- blob.parseTerraWorkspaceIdFromPath containerResourceId <- blob.containerWSMResourceId - endpoint = s"$wsmEndpoint/$workspaceId/resources/controlled/azure/storageContainer/$containerResourceId/getSasToken" + endpoint = s"$wsmEndpoint/api/workspaces/v1/$workspaceId/resources/controlled/azure/storageContainer/$containerResourceId/getSasToken" } yield endpoint sasTokenEndpoint match { @@ -119,8 +132,6 @@ object TesAsyncBackendJobExecutionActor { } } -case class LocalizedSasTokenParams(wsmEndpoint: String, workspaceId: String, containerResourceId: String) - case class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyncExecutionActorParams) extends BackendJobLifecycleActor with StandardAsyncExecutionActor with TesJobCachingActorHelper { @@ -157,7 +168,7 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) super.scriptPreamble ++ determineWSMSasEndpointFromInputs(taskInputs, getPath).map{ - endpoint => generateLocalizedSasScriptPreammble(endpoint) + endpoint => generateLocalizedSasScriptPreamble(endpoint) }.getOrElse("") } diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 4a2d9b1d5b6..71a9a606a03 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -76,7 +76,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers { val mockWorkspaceId = "1111-2222-3333-4444" val mockContainerId = "5678-who-do-we-appreciate" val mockSasParams: LocalizedSasTokenParams = LocalizedSasTokenParams(mockEndpoint, mockWorkspaceId, mockContainerId) - TesAsyncBackendJobExecutionActor.generateLocalizedSasScriptPreammble(mockSasParams) shouldBe + TesAsyncBackendJobExecutionActor.generateLocalizedSasScriptPreamble(mockSasParams) shouldBe s""" |WSM_ENDPOINT="${mockEndpoint}" |WORKSPACE_ID="${mockWorkspaceId}" From 8cad15b2cc530c714cd1e5fabe5ecd6774a48de7 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 23 Oct 2023 15:09:50 -0400 Subject: [PATCH 09/34] unit tests --- .../TesAsyncBackendJobExecutionActor.scala | 27 ++++--- ...TesAsyncBackendJobExecutionActorSpec.scala | 72 ++++++++++++------- 2 files changed, 64 insertions(+), 35 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index dda10d6c898..9c01295403a 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -15,7 +15,7 @@ import common.validation.ErrorOr.ErrorOr import common.validation.Validation._ import cromwell.backend.BackendJobLifecycleActor import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNonRetryableExecutionHandle, PendingExecutionHandle} -import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{generateLocalizedSasScriptPreamble, determineWSMSasEndpointFromInputs} +import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{determineWSMSasEndpointFromInputs, generateLocalizedSasScriptPreamble, maybeConvertToBlob} import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} import cromwell.core.path.{DefaultPathBuilder, Path} @@ -59,7 +59,7 @@ case object Cancelled extends TesRunStatus { object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" - def generateLocalizedSasScriptPreamble(getSasWsmEndpoint: String) : String = { + private def generateLocalizedSasScriptPreamble(getSasWsmEndpoint: String) : String = { // BEARER_TOKEN: https://learn.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#get-a-token-using-http // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. // NB: For easier debugging/logging, we echo the first 4 characters of the acquired sas token. If something goes wrong with the curl, these will be "null" @@ -96,12 +96,23 @@ object TesAsyncBackendJobExecutionActor { |""".stripMargin } - /* Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a + private def maybeConvertToBlob(pathToTest: Try[Path]): Try[BlobPath] = { + pathToTest.collect { case blob: BlobPath => blob } + } + /** + * Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a * fresh sas token for itself. In order to be able to do this, we provide it with a precomputed endpoint it can use. * The task VM will use the user assigned managed identity that it is running as in order to authenticate. - * We only return a value if //TODO and if at least one blob storage file is provided as a task input. + * We only return a value if at least one of the inputs is a BlobPath and //TODO flag specified in WDL. + * @param taskInputs The inputs to this particular TesTask. If any are blob files, the first will be used to + * determine the storage container to retrieve the sas token for. + * @param pathGetter A function to convert string filepath into a cromwell Path object. + * @param blobConverter A function to convert a Path into a Blob path, if possible. Use the default: this is a parameter + * only because the usual means of identifying if a Path is a BlobPath doesn't work with mocked types. */ - def determineWSMSasEndpointFromInputs(taskInputs: List[Input], pathGetter: String => Try[Path]): Option[String] = { + def determineWSMSasEndpointFromInputs(taskInputs: List[Input], + pathGetter: String => Try[Path], + blobConverter: Try[Path] => Try[BlobPath] = maybeConvertToBlob): Option[String] = { val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL if (!shouldLocalizeSas) return None @@ -112,10 +123,8 @@ object TesAsyncBackendJobExecutionActor { } if(blobFiles.isEmpty) return None - //We use the first blob file in the list as a template for determining the localized sas params - val blobPath: Try[BlobPath] = pathGetter(blobFiles.head.toUrl).collect{ - case blob: BlobPath => blob - } + // We use the first blob file in the list as a template for determining the localized sas params + val blobPath: Try[BlobPath] = blobConverter(pathGetter(blobFiles.head.toUrl)) val sasTokenEndpoint = for { blob <- blobPath diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 71a9a606a03..e59bc1f963e 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -1,20 +1,21 @@ package cromwell.backend.impl.tes -import com.fasterxml.jackson.databind.ext.NioPathSerializer -import cromwell.core.path.{NioPath, Path} -import cromwell.filesystems.blob.{BlobPath, BlobPathBuilderFactory, EndpointURL} +import common.mock.MockSugar +import cromwell.core.path.Path +import cromwell.filesystems.blob.{BlobContainerName, BlobPath} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import scala.tools.nsc.io.Path +import java.util.UUID +import scala.util.{Failure, Success, Try} -class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers { +class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers with MockSugar { behavior of "TesAsyncBackendJobExecutionActor" val fullyQualifiedName = "this.name.is.more.than.qualified" val workflowName = "mockWorkflow" - val someBlobUrl = "https://lz813a3d637adefec2c6e88f.blob.core.windows.net/sc-d8143fd8-aa07-446d-9ba0-af72203f1794/nyxp6c/tes-internal/configuration/supported-vm-sizes?sv=sasToken" + val someBlobUrl = "https://lz813a3d637adefec2c6e88f.blob.core.windows.net/sc-d8143fd8-aa07-446d-9ba0-af72203f1794/nyxp6c/tes-internal/configuration/supported-vm-sizes" val someNotBlobUrl = "https://www.google.com/path/to/exile" var index = 0 @@ -57,30 +58,49 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers { content = None ) - def pathGetter(pathString: String): Path = { - /* - val factory: BlobPathBuilderFactory = BlobPathBuilderFactory() - val nioPath: java.nio.file.Path = NioPathSerializer() - val endpoint: EndpointURL = EndpointURL("www.some.blob.endpoint/url") - if(pathString.startsWith(someBlobUrl)) BlobPath("someBlobPath",endpoint,) - */ + // Mock blob path functionality. + val mockWsmEndpoint = "https://wsm.mock.com/endpoint" + val mockWorkspaceId = "e58ed763-928c-4155-0000-fdbaaadc15f3" + val mockContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" + + val mockBlobPath: BlobPath = mock[BlobPath] + mockBlobPath.container returns BlobContainerName("1234") + mockBlobPath.wsmEndpoint returns Try(mockWsmEndpoint) + mockBlobPath.parseTerraWorkspaceIdFromPath returns Try(UUID.fromString(mockWorkspaceId)) + mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) + mockBlobPath.md5 returns "blobmd5" + + val mockPath: Path = mock[Path] + mockPath.md5 returns "nonBlobmd5" + def mockPathGetter(pathString: String): Try[Path] = { + val foundBlobPath: Success[BlobPath] = Success(mockBlobPath) + val foundNonBlobPath: Success[Path] = Success(mockPath) + if (pathString.equals(blobInput_0.url.get) || pathString.equals(blobInput_1.url.get)) return foundBlobPath + foundNonBlobPath + } + + def mockBlobConverter(pathToConvert: Try[Path]): Try[BlobPath] = { + val mockMd5 = mockBlobPath.md5 + val pathMd5 = pathToConvert.get.md5 + if (mockMd5.equals(pathMd5)) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) } - it should "not generate sas params when no blob paths are provided" in { - val inputs = List[Input] - //TesAsyncBackendJobExecutionActor.getLocalizedSasTokenParams(inputs,) + it should "not return sas endpoint when no blob paths are provided" in { + val emptyInputs: List[Input] = List() + val bloblessInputs: List[Input] = List(notBlobInput_1, notBlobInput_2) + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(emptyInputs, mockPathGetter, mockBlobConverter).isEmpty shouldBe true + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(bloblessInputs, mockPathGetter, mockBlobConverter).isEmpty shouldBe true + } + + it should "return a sas endpoint based on inputs when blob paths are provided" in { + val expected = s"$mockWsmEndpoint/api/workspaces/v1/$mockWorkspaceId/resources/controlled/azure/storageContainer/$mockContainerResourceId/getSasToken" + val blobInputs: List[Input] = List(blobInput_0, blobInput_1) + val mixedInputs: List[Input] = List(notBlobInput_1, blobInput_0, blobInput_1) + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, mockPathGetter, mockBlobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, mockPathGetter, mockBlobConverter).get shouldEqual expected } it should "generate proper script preamble" in { - val mockEndpoint = "www.workspacemanager.com" - val mockWorkspaceId = "1111-2222-3333-4444" - val mockContainerId = "5678-who-do-we-appreciate" - val mockSasParams: LocalizedSasTokenParams = LocalizedSasTokenParams(mockEndpoint, mockWorkspaceId, mockContainerId) - TesAsyncBackendJobExecutionActor.generateLocalizedSasScriptPreamble(mockSasParams) shouldBe - s""" - |WSM_ENDPOINT="${mockEndpoint}" - |WORKSPACE_ID="${mockWorkspaceId}" - |CONTAINER_RESOURCE_ID="${mockContainerId}" - |""".stripMargin + } } \ No newline at end of file From 408e0089d9213a41cdb251a2e998a83f2d1eccce Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 23 Oct 2023 15:50:34 -0400 Subject: [PATCH 10/34] tidy --- .../TesAsyncBackendJobExecutionActor.scala | 30 +++++++++---------- ...TesAsyncBackendJobExecutionActorSpec.scala | 16 ++++------ 2 files changed, 21 insertions(+), 25 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 9c01295403a..b6bb5b1642a 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -15,7 +15,7 @@ import common.validation.ErrorOr.ErrorOr import common.validation.Validation._ import cromwell.backend.BackendJobLifecycleActor import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNonRetryableExecutionHandle, PendingExecutionHandle} -import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{determineWSMSasEndpointFromInputs, generateLocalizedSasScriptPreamble, maybeConvertToBlob} +import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{determineWSMSasEndpointFromInputs, generateLocalizedSasScriptPreamble} import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} import cromwell.core.path.{DefaultPathBuilder, Path} @@ -62,7 +62,6 @@ object TesAsyncBackendJobExecutionActor { private def generateLocalizedSasScriptPreamble(getSasWsmEndpoint: String) : String = { // BEARER_TOKEN: https://learn.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#get-a-token-using-http // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. - // NB: For easier debugging/logging, we echo the first 4 characters of the acquired sas token. If something goes wrong with the curl, these will be "null" s""" |### BEGIN ACQUIRE LOCAL SAS TOKEN ### |# Install dependencies @@ -78,11 +77,10 @@ object TesAsyncBackendJobExecutionActor { |BEARER_TOKEN="$${BEARER_TOKEN%\\"}" | |# Use the precomputed endpoint from cromwell + WSM to acquire a sas token - |GET_SAS_ENDPOINT=$getSasWsmEndpoint |sas_response_json=$$(curl -s \\ | --retry 3 \\ | --retry-delay 2 \\ - | -X POST "$$GET_SAS_ENDPOINT" \\ + | -X POST "$getSasWsmEndpoint" \\ | -H "Content-Type: application/json" \\ | -H "accept: */*" \\ | -H "Authorization: Bearer $${BEARER_TOKEN}") @@ -90,7 +88,7 @@ object TesAsyncBackendJobExecutionActor { |# Store token as environment variable |export AZURE_STORAGE_SAS_TOKEN=$$(echo "$${sas_response_json}" | jq -r '.token') | - |# Echo the first 3 characters for logging/debugging purposes. We expect them to be sv= if everything went well. + |# Echo the first characters for logging/debugging purposes. "null" indicates something went wrong. |echo Acquired sas token: "$${AZURE_STORAGE_SAS_TOKEN:0:3}****" |### END ACQUIRE LOCAL SAS TOKEN ### |""".stripMargin @@ -99,6 +97,7 @@ object TesAsyncBackendJobExecutionActor { private def maybeConvertToBlob(pathToTest: Try[Path]): Try[BlobPath] = { pathToTest.collect { case blob: BlobPath => blob } } + /** * Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a * fresh sas token for itself. In order to be able to do this, we provide it with a precomputed endpoint it can use. @@ -107,8 +106,9 @@ object TesAsyncBackendJobExecutionActor { * @param taskInputs The inputs to this particular TesTask. If any are blob files, the first will be used to * determine the storage container to retrieve the sas token for. * @param pathGetter A function to convert string filepath into a cromwell Path object. - * @param blobConverter A function to convert a Path into a Blob path, if possible. Use the default: this is a parameter - * only because the usual means of identifying if a Path is a BlobPath doesn't work with mocked types. + * @param blobConverter A function to convert a Path into a Blob path, if possible. Provided for testing purposes. + * @return A URL endpoint that, when called with proper authentication, will return a sas token. + * Returns 'None' if one should not be used for this task. */ def determineWSMSasEndpointFromInputs(taskInputs: List[Input], pathGetter: String => Try[Path], @@ -116,18 +116,17 @@ object TesAsyncBackendJobExecutionActor { val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL if (!shouldLocalizeSas) return None - val blobFiles = taskInputs.collect{ //Collect all inputs with URLs defined as (in)valid blob paths + // Collect all of the inputs that are valid blob paths + val blobFiles = taskInputs.collect{ case input if input.url.isDefined => BlobPathBuilder.validateBlobPath(input.url.get) - }.collect{ //Collect only the valid blob paths + }.collect{ case valid: BlobPathBuilder.ValidBlobPath => valid } - if(blobFiles.isEmpty) return None - // We use the first blob file in the list as a template for determining the localized sas params - val blobPath: Try[BlobPath] = blobConverter(pathGetter(blobFiles.head.toUrl)) + // We use the first blob file in the list as a template for determining the localized sas params val sasTokenEndpoint = for { - blob <- blobPath + blob <- blobConverter(pathGetter(blobFiles.head.toUrl)) wsmEndpoint <- blob.wsmEndpoint workspaceId <- blob.parseTerraWorkspaceIdFromPath containerResourceId <- blob.containerWSMResourceId @@ -141,7 +140,6 @@ object TesAsyncBackendJobExecutionActor { } } -case class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyncExecutionActorParams) extends BackendJobLifecycleActor with StandardAsyncExecutionActor with TesJobCachingActorHelper { implicit val actorSystem = context.system @@ -176,9 +174,11 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn _.collectAsSeq { case w: WomFile => w } } val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) - super.scriptPreamble ++ determineWSMSasEndpointFromInputs(taskInputs, getPath).map{ + val tesTaskPreamble = determineWSMSasEndpointFromInputs(taskInputs, getPath).map { endpoint => generateLocalizedSasScriptPreamble(endpoint) }.getOrElse("") + + super.scriptPreamble ++ tesTaskPreamble } override def mapCommandLineWomFile(womFile: WomFile): WomFile = { diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index e59bc1f963e..681a1a3fbb7 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -68,10 +68,9 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockBlobPath.wsmEndpoint returns Try(mockWsmEndpoint) mockBlobPath.parseTerraWorkspaceIdFromPath returns Try(UUID.fromString(mockWorkspaceId)) mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) - mockBlobPath.md5 returns "blobmd5" + mockBlobPath.md5 returns "BLOB_MD5" val mockPath: Path = mock[Path] - mockPath.md5 returns "nonBlobmd5" def mockPathGetter(pathString: String): Try[Path] = { val foundBlobPath: Success[BlobPath] = Success(mockBlobPath) val foundNonBlobPath: Success[Path] = Success(mockPath) @@ -80,9 +79,8 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit } def mockBlobConverter(pathToConvert: Try[Path]): Try[BlobPath] = { - val mockMd5 = mockBlobPath.md5 - val pathMd5 = pathToConvert.get.md5 - if (mockMd5.equals(pathMd5)) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) + //using a stubbed md5 rather than matching on type because type matching of mocked types at runtime causes problems + if (pathToConvert.get.md5.equals("BLOB_MD5")) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) } it should "not return sas endpoint when no blob paths are provided" in { @@ -94,13 +92,11 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit it should "return a sas endpoint based on inputs when blob paths are provided" in { val expected = s"$mockWsmEndpoint/api/workspaces/v1/$mockWorkspaceId/resources/controlled/azure/storageContainer/$mockContainerResourceId/getSasToken" + val blobInput: List[Input] = List(blobInput_0) val blobInputs: List[Input] = List(blobInput_0, blobInput_1) val mixedInputs: List[Input] = List(notBlobInput_1, blobInput_0, blobInput_1) + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInput, mockPathGetter, mockBlobConverter).get shouldEqual expected TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, mockPathGetter, mockBlobConverter).get shouldEqual expected TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, mockPathGetter, mockBlobConverter).get shouldEqual expected } - - it should "generate proper script preamble" in { - - } -} \ No newline at end of file +} From 05cab44b9b2608bd6acee5f0bb6cf13ab5cc7025 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 23 Oct 2023 15:54:34 -0400 Subject: [PATCH 11/34] 4 --- .../backend/impl/tes/TesAsyncBackendJobExecutionActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index b6bb5b1642a..d94f01a14c8 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -89,7 +89,7 @@ object TesAsyncBackendJobExecutionActor { |export AZURE_STORAGE_SAS_TOKEN=$$(echo "$${sas_response_json}" | jq -r '.token') | |# Echo the first characters for logging/debugging purposes. "null" indicates something went wrong. - |echo Acquired sas token: "$${AZURE_STORAGE_SAS_TOKEN:0:3}****" + |echo Acquired sas token: "$${AZURE_STORAGE_SAS_TOKEN:0:4}****" |### END ACQUIRE LOCAL SAS TOKEN ### |""".stripMargin } From e04f175e57f8b72b543dca3061b6172805f42463 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 24 Oct 2023 13:13:11 -0400 Subject: [PATCH 12/34] fix mock --- .../filesystems/blob/BlobPathBuilderFactory.scala | 1 - .../tes/TesAsyncBackendJobExecutionActorSpec.scala | 11 ++++++----- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala index c6921338799..47245552dc2 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilderFactory.scala @@ -29,7 +29,6 @@ final case class EndpointURL(value: String) { } } final case class WorkspaceId(value: UUID) {override def toString: String = value.toString} - final case class ContainerResourceId(value: UUID) {override def toString: String = value.toString} final case class WorkspaceManagerURL(value: String) {override def toString: String = value} diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 681a1a3fbb7..bac368b4d11 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -1,11 +1,11 @@ package cromwell.backend.impl.tes import common.mock.MockSugar -import cromwell.core.path.Path import cromwell.filesystems.blob.{BlobContainerName, BlobPath} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.nio.file.Paths import java.util.UUID import scala.util.{Failure, Success, Try} @@ -68,17 +68,18 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockBlobPath.wsmEndpoint returns Try(mockWsmEndpoint) mockBlobPath.parseTerraWorkspaceIdFromPath returns Try(UUID.fromString(mockWorkspaceId)) mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) + mockBlobPath.nioPath returns Paths.get("dummy/path") mockBlobPath.md5 returns "BLOB_MD5" - val mockPath: Path = mock[Path] - def mockPathGetter(pathString: String): Try[Path] = { + val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] + def mockPathGetter(pathString: String): Try[cromwell.core.path.Path] = { val foundBlobPath: Success[BlobPath] = Success(mockBlobPath) - val foundNonBlobPath: Success[Path] = Success(mockPath) + val foundNonBlobPath: Success[cromwell.core.path.Path] = Success(mockPath) if (pathString.equals(blobInput_0.url.get) || pathString.equals(blobInput_1.url.get)) return foundBlobPath foundNonBlobPath } - def mockBlobConverter(pathToConvert: Try[Path]): Try[BlobPath] = { + def mockBlobConverter(pathToConvert: Try[cromwell.core.path.Path]): Try[BlobPath] = { //using a stubbed md5 rather than matching on type because type matching of mocked types at runtime causes problems if (pathToConvert.get.md5.equals("BLOB_MD5")) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) } From 05d399150753c52d3434564eaf358a0b1d498113 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 24 Oct 2023 16:41:31 -0400 Subject: [PATCH 13/34] configureable --- .../TesAsyncBackendJobExecutionActor.scala | 24 ++++++------ .../impl/tes/TesRuntimeAttributes.scala | 39 +++++++++++++++++-- .../impl/tes/TesInitializationActorSpec.scala | 4 +- .../impl/tes/TesRuntimeAttributesSpec.scala | 12 ++++++ .../backend/impl/tes/TesTaskSpec.scala | 1 + 5 files changed, 64 insertions(+), 16 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index d94f01a14c8..7d0545374d3 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -59,7 +59,7 @@ case object Cancelled extends TesRunStatus { object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" - private def generateLocalizedSasScriptPreamble(getSasWsmEndpoint: String) : String = { + private def generateLocalizedSasScriptPreamble(environmentVariableName: String, getSasWsmEndpoint: String) : String = { // BEARER_TOKEN: https://learn.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#get-a-token-using-http // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. s""" @@ -86,10 +86,10 @@ object TesAsyncBackendJobExecutionActor { | -H "Authorization: Bearer $${BEARER_TOKEN}") | |# Store token as environment variable - |export AZURE_STORAGE_SAS_TOKEN=$$(echo "$${sas_response_json}" | jq -r '.token') + |export $environmentVariableName=$$(echo "$${sas_response_json}" | jq -r '.token') | |# Echo the first characters for logging/debugging purposes. "null" indicates something went wrong. - |echo Acquired sas token: "$${AZURE_STORAGE_SAS_TOKEN:0:4}****" + |echo Acquired sas token: "$${$environmentVariableName:0:4}****" |### END ACQUIRE LOCAL SAS TOKEN ### |""".stripMargin } @@ -169,15 +169,17 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } override def scriptPreamble: String = { - val workflowName = workflowDescriptor.callable.name - val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { - _.collectAsSeq { case w: WomFile => w } - } - val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) - val tesTaskPreamble = determineWSMSasEndpointFromInputs(taskInputs, getPath).map { - endpoint => generateLocalizedSasScriptPreamble(endpoint) + val tesTaskPreamble: String = runtimeAttributes.localizedSasEnvVar.map{enviornmentVariableName => + val workflowName = workflowDescriptor.callable.name + val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { + _.collectAsSeq { case w: WomFile => w } + } + val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) + val preamble = determineWSMSasEndpointFromInputs(taskInputs, getPath).map{ endpoint => + generateLocalizedSasScriptPreamble(enviornmentVariableName, endpoint) + }.getOrElse("") + preamble }.getOrElse("") - super.scriptPreamble ++ tesTaskPreamble } diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala index c5b3c4df66d..a406a572bb0 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala @@ -1,5 +1,6 @@ package cromwell.backend.impl.tes +import cats.data.Validated import cats.syntax.validated._ import com.typesafe.config.Config import common.validation.ErrorOr.ErrorOr @@ -15,6 +16,8 @@ import wom.format.MemorySize import wom.types.{WomIntegerType, WomStringType} import wom.values._ +import java.util.regex.Pattern + case class TesRuntimeAttributes(continueOnReturnCode: ContinueOnReturnCode, dockerImage: String, dockerWorkingDir: Option[String], @@ -23,13 +26,14 @@ case class TesRuntimeAttributes(continueOnReturnCode: ContinueOnReturnCode, memory: Option[MemorySize], disk: Option[MemorySize], preemptible: Boolean, + localizedSasEnvVar: Option[String], backendParameters: Map[String, Option[String]]) object TesRuntimeAttributes { - val DockerWorkingDirKey = "dockerWorkingDir" val DiskSizeKey = "disk" val PreemptibleKey = "preemptible" + val LocalizedSasKey = "sasEnvironmentVariable" private def cpuValidation(runtimeConfig: Option[Config]): OptionalRuntimeAttributesValidation[Int Refined Positive] = CpuValidation.optional @@ -47,8 +51,8 @@ object TesRuntimeAttributes { private val dockerValidation: RuntimeAttributesValidation[String] = DockerValidation.instance private val dockerWorkingDirValidation: OptionalRuntimeAttributesValidation[String] = DockerWorkingDirValidation.optional - private def preemptibleValidation(runtimeConfig: Option[Config]) = PreemptibleValidation.default(runtimeConfig) + private def localizedSasValidation: OptionalRuntimeAttributesValidation[String] = LocalizedSasValidation.optional def runtimeAttributesBuilder(backendRuntimeConfig: Option[Config]): StandardValidatedRuntimeAttributesBuilder = // !! NOTE !! If new validated attributes are added to TesRuntimeAttributes, be sure to include @@ -62,6 +66,7 @@ object TesRuntimeAttributes { dockerValidation, dockerWorkingDirValidation, preemptibleValidation(backendRuntimeConfig), + localizedSasValidation ) def makeBackendParameters(runtimeAttributes: Map[String, WomValue], @@ -124,8 +129,10 @@ object TesRuntimeAttributes { RuntimeAttributesValidation.extract(failOnStderrValidation(backendRuntimeConfig), validatedRuntimeAttributes) val continueOnReturnCode: ContinueOnReturnCode = RuntimeAttributesValidation.extract(continueOnReturnCodeValidation(backendRuntimeConfig), validatedRuntimeAttributes) - val preemptible: Boolean = + val preemptible: Boolean = { RuntimeAttributesValidation.extract(preemptibleValidation(backendRuntimeConfig), validatedRuntimeAttributes) + } + val localizedSas: Option[String] = RuntimeAttributesValidation.extractOption(localizedSasValidation.key, validatedRuntimeAttributes) // !! NOTE !! If new validated attributes are added to TesRuntimeAttributes, be sure to include // their validations here so that they will be handled correctly with backendParameters. @@ -139,7 +146,8 @@ object TesRuntimeAttributes { diskSizeCompatValidation(backendRuntimeConfig), failOnStderrValidation(backendRuntimeConfig), continueOnReturnCodeValidation(backendRuntimeConfig), - preemptibleValidation(backendRuntimeConfig) + preemptibleValidation(backendRuntimeConfig), + localizedSasValidation ) // BT-458 any strings included in runtime attributes that aren't otherwise used should be @@ -156,6 +164,7 @@ object TesRuntimeAttributes { memory, disk, preemptible, + localizedSas, backendParameters ) } @@ -218,3 +227,25 @@ class PreemptibleValidation extends BooleanRuntimeAttributesValidation(TesRuntim override protected def missingValueMessage: String = s"Expecting $key runtime attribute to be an Integer, Boolean, or a String with values of 'true' or 'false'" } + +object LocalizedSasValidation { + lazy val instance: RuntimeAttributesValidation[String] = new LocalizedSasValidation + lazy val optional: OptionalRuntimeAttributesValidation[String] = instance.optional +} + +class LocalizedSasValidation extends StringRuntimeAttributesValidation(TesRuntimeAttributes.LocalizedSasKey) { + private def isValidBashVariableName(str: String): Boolean = { + // require string be only letters, numbers, and underscores + val pattern = Pattern.compile("^[a-zA-Z0-9_]+$", Pattern.CASE_INSENSITIVE) + val matcher = pattern.matcher(str) + matcher.find + } + + override protected def invalidValueMessage(value: WomValue): String = { + s"Invalid Runtime Attribute value for ${TesRuntimeAttributes.LocalizedSasKey}. Value must be a string containing only letters, numbers, and underscores." + } + + override protected def validateValue: PartialFunction[WomValue, ErrorOr[String]] = { + case WomString(value) => if(isValidBashVariableName(value)) value.validNel else Validated.invalidNel(invalidValueMessage(WomString(value))) + } +} diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala index 731dd3c6c70..a1b2e4e1cd9 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala @@ -63,6 +63,7 @@ class TesInitializationActorSpec extends TestKitSuite | # The keys below have been commented out as they are optional runtime attributes. | # dockerWorkingDir | # docker + | # sasEnvironmentVariable |} |""".stripMargin @@ -107,6 +108,7 @@ class TesInitializationActorSpec extends TestKitSuite } def nonStringErrorMessage(key: String) = s"Workflow option $key must be a string" + val bothRequiredErrorMessage = s"Workflow options ${TesWorkflowOptionKeys.WorkflowExecutionIdentity} and ${TesWorkflowOptionKeys.DataAccessIdentity} are both required if one is provided" "fail when WorkflowExecutionIdentity is not a string and DataAccessIdentity is missing" in { @@ -120,7 +122,7 @@ class TesInitializationActorSpec extends TestKitSuite case InitializationFailed(failure) => val expectedMsg = nonStringErrorMessage(TesWorkflowOptionKeys.WorkflowExecutionIdentity) if (!(failure.getMessage.contains(expectedMsg) && - failure.getMessage.contains(bothRequiredErrorMessage))) { + failure.getMessage.contains(bothRequiredErrorMessage))) { fail(s"Exception message did not contain both '$expectedMsg' and '$bothRequiredErrorMessage'. Was '$failure'") } } diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala index e1984fb65dc..d5e30730af9 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala @@ -25,6 +25,7 @@ class TesRuntimeAttributesSpec extends AnyWordSpecLike with CromwellTimeoutSpec None, None, false, + None, Map.empty ) @@ -71,6 +72,17 @@ class TesRuntimeAttributesSpec extends AnyWordSpecLike with CromwellTimeoutSpec assertSuccess(runtimeAttributes, expectedRuntimeAttributes) } + "validate a valid sasEnvironmentVariable entry" in { + val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), "sasEnvironmentVariable" -> WomString("THIS_IS_VALID")) + val expectedRuntimeAttributes = expectedDefaultsPlusUbuntuDocker.copy(localizedSasEnvVar = Some("THIS_IS_VALID")) + assertSuccess(runtimeAttributes, expectedRuntimeAttributes) + } + + "fail to validate an invalid sasEnvironmentVariable entry" in { + val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), "sasEnvironmentVariable" -> WomString("THIS IS INVALID")) + assertFailure(runtimeAttributes, "Value must be a string containing only letters, numbers, and underscores.") + } + "convert a positive integer preemptible entry to true boolean" in { val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), "preemptible" -> WomInteger(3)) val expectedRuntimeAttributes = expectedDefaultsPlusUbuntuDocker.copy(preemptible = true) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesTaskSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesTaskSpec.scala index b7887b29944..a5fd3a3a7e2 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesTaskSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesTaskSpec.scala @@ -31,6 +31,7 @@ class TesTaskSpec None, None, false, + None, Map.empty ) val internalPathPrefix = Option("mock/path/to/tes/task") From 045835b9f17f5a874c7d29d2b6e6debb3b5743d4 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Wed, 25 Oct 2023 14:13:35 -0400 Subject: [PATCH 14/34] better mock path --- .../backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index bac368b4d11..e523080964d 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -68,7 +68,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockBlobPath.wsmEndpoint returns Try(mockWsmEndpoint) mockBlobPath.parseTerraWorkspaceIdFromPath returns Try(UUID.fromString(mockWorkspaceId)) mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) - mockBlobPath.nioPath returns Paths.get("dummy/path") + mockBlobPath.nioPath returns Paths.get(".") mockBlobPath.md5 returns "BLOB_MD5" val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] From f37ee7ad5c5945e6446ef72a4b9da9d799c4bae8 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Wed, 25 Oct 2023 15:19:38 -0400 Subject: [PATCH 15/34] mocking has escalated to bullying --- .../impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index e523080964d..9e165d7756d 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -1,15 +1,13 @@ package cromwell.backend.impl.tes import common.mock.MockSugar +import cromwell.core.path import cromwell.filesystems.blob.{BlobContainerName, BlobPath} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers - -import java.nio.file.Paths import java.util.UUID import scala.util.{Failure, Success, Try} - class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers with MockSugar { behavior of "TesAsyncBackendJobExecutionActor" @@ -64,11 +62,13 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val mockContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" val mockBlobPath: BlobPath = mock[BlobPath] + val mockNioPath: path.NioPath = mock[path.NioPath] + mockBlobPath.container returns BlobContainerName("1234") mockBlobPath.wsmEndpoint returns Try(mockWsmEndpoint) mockBlobPath.parseTerraWorkspaceIdFromPath returns Try(UUID.fromString(mockWorkspaceId)) mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) - mockBlobPath.nioPath returns Paths.get(".") + mockBlobPath.nioPath returns mockNioPath mockBlobPath.md5 returns "BLOB_MD5" val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] From 75ca3eacb59300ad0fb136649b43f8d412f83559 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Wed, 25 Oct 2023 16:13:18 -0400 Subject: [PATCH 16/34] better logging --- .../backend/impl/tes/TesAsyncBackendJobExecutionActor.scala | 4 +++- .../impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 1 + 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 7d0545374d3..7a1866e330b 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -70,6 +70,7 @@ object TesAsyncBackendJobExecutionActor { |apt-get -y install jq | |# Acquire bearer token, relying on the User Assigned Managed Identity of this VM. + |echo Acquiring Bearer Token using User Assigned Managed Identity... |BEARER_TOKEN=$$(curl 'http://169.254.169.254/metadata/identity/oauth2/token?api-version=2018-02-01&resource=https%3A%2F%2Fmanagement.azure.com%2F' -H Metadata:true -s | jq .access_token) | |# Remove the leading and trailing quotes @@ -77,6 +78,7 @@ object TesAsyncBackendJobExecutionActor { |BEARER_TOKEN="$${BEARER_TOKEN%\\"}" | |# Use the precomputed endpoint from cromwell + WSM to acquire a sas token + |echo Requesting sas token from WSM... |sas_response_json=$$(curl -s \\ | --retry 3 \\ | --retry-delay 2 \\ @@ -89,7 +91,7 @@ object TesAsyncBackendJobExecutionActor { |export $environmentVariableName=$$(echo "$${sas_response_json}" | jq -r '.token') | |# Echo the first characters for logging/debugging purposes. "null" indicates something went wrong. - |echo Acquired sas token: "$${$environmentVariableName:0:4}****" + |echo Saving sas token: $${$environmentVariableName:0:4}**** to environment variable $environmentVariableName... |### END ACQUIRE LOCAL SAS TOKEN ### |""".stripMargin } diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 9e165d7756d..5b73ce59ac9 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -70,6 +70,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) mockBlobPath.nioPath returns mockNioPath mockBlobPath.md5 returns "BLOB_MD5" + mockBlobPath.toAbsolutePath returns mockBlobPath val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] def mockPathGetter(pathString: String): Try[cromwell.core.path.Path] = { From 5b6852cabb2e399955742a7a74d8334dc0f6f783 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 31 Oct 2023 14:59:26 -0400 Subject: [PATCH 17/34] most PR feedback --- .../blob/BlobFileSystemManager.scala | 28 ++++++++- .../filesystems/blob/BlobPathBuilder.scala | 24 +------ .../TesAsyncBackendJobExecutionActor.scala | 63 ++++++++++--------- .../impl/tes/TesRuntimeAttributes.scala | 2 +- ...TesAsyncBackendJobExecutionActorSpec.scala | 33 +++++----- 5 files changed, 83 insertions(+), 67 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index 969331fd458..2bf2aedca47 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -165,6 +165,8 @@ object BlobSasTokenGenerator { } +case class WSMTerraCoordinates(wsmEndpoint: String, workspaceId: UUID, containerResourceId: UUID) + case class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvider, overrideWsmAuthToken: Option[String]) extends BlobSasTokenGenerator { @@ -203,12 +205,36 @@ case class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClient wsmResourceClient.findContainerResourceId(workspaceId, container) } - def getWsmAuth: Try[String] = { + private def getWsmAuth: Try[String] = { overrideWsmAuthToken match { case Some(t) => Success(t) case None => AzureCredentials.getAccessToken(None).toTry } } + private def parseTerraWorkspaceIdFromPath(blobPath: BlobPath): Try[UUID] = { + if (blobPath.container.value.startsWith("sc-")) Try(UUID.fromString(blobPath.container.value.substring(3))) + else Failure(new Exception("Could not parse workspace ID from storage container. Are you sure this is a file in a Terra Workspace?")) + } + + /** + * If the provided blob path looks like it comes from a terra workspace, return an end point that, when called with GET + * and proper authentication, will return a sas token capable of accessing the container the blob path points to. + * @param blobPath A blob path of a file living in a blob container that WSM knows about (likely a workspace container). + * + * NOTE: This function makes two synchronous REST requests. + */ + def getWSMSasFetchEndpoint(blobPath: BlobPath): Try[String] = { + val wsmEndpoint = wsmClientProvider.getBaseWorkspaceManagerUrl + val terraInfo: Try[WSMTerraCoordinates] = for { + workspaceId <- parseTerraWorkspaceIdFromPath(blobPath) + auth <- getWsmAuth + containerResourceId <- getContainerResourceId(workspaceId, blobPath.container, auth) + coordinates = WSMTerraCoordinates(wsmEndpoint, workspaceId, containerResourceId) + } yield coordinates + terraInfo.map{terraCoordinates => + s"${terraCoordinates.wsmEndpoint}/api/workspaces/v1/${terraCoordinates.workspaceId.toString}/resources/controlled/azure/storageContainer/${terraCoordinates.containerResourceId.toString}/getSasToken" + } + } } case class NativeBlobSasTokenGenerator(subscription: Option[SubscriptionId] = None) extends BlobSasTokenGenerator { diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index 98fb7eb7db7..2051ca18162 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -7,7 +7,6 @@ import cromwell.filesystems.blob.BlobPathBuilder._ import java.net.{MalformedURLException, URI} import java.nio.file.{Files, LinkOption} -import java.util.UUID import scala.jdk.CollectionConverters._ import scala.language.postfixOps import scala.util.{Failure, Success, Try} @@ -189,28 +188,7 @@ case class BlobPath private[blob](pathString: String, endpoint: EndpointURL, con */ def pathWithoutContainer : String = pathString - def parseTerraWorkspaceIdFromPath: Try[UUID] = { - if(container.value.startsWith("sc-")) Try(UUID.fromString(container.value.substring(3))) else Failure(new Exception("Could not parse workspace ID from storage container")) - } - - private def getWSMTokenGenerator: Try[WSMBlobSasTokenGenerator] = { - fsm.blobTokenGenerator match { - case wsmGenerator: WSMBlobSasTokenGenerator => Try(wsmGenerator) - case _: Any => Failure(new NoSuchElementException("This blob file does not have an associated WSMBlobSasTokenGenerator")) - } - } - def containerWSMResourceId: Try[UUID] = { - for { - generator <- getWSMTokenGenerator - workspaceId <- parseTerraWorkspaceIdFromPath - wsmAuth <- generator.getWsmAuth - resourceId <- generator.getContainerResourceId(workspaceId, container, wsmAuth) - } yield resourceId - } - - def wsmEndpoint: Try[String] = { - getWSMTokenGenerator.map(generator => generator.wsmClientProvider.getBaseWorkspaceManagerUrl) - } + def getFilesystemManager: BlobFileSystemManager = fsm override def getSymlinkSafePath(options: LinkOption*): Path = toAbsolutePath diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 7a1866e330b..7d823101e02 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -18,10 +18,12 @@ import cromwell.backend.async.{AbortedExecutionHandle, ExecutionHandle, FailedNo import cromwell.backend.impl.tes.TesAsyncBackendJobExecutionActor.{determineWSMSasEndpointFromInputs, generateLocalizedSasScriptPreamble} import cromwell.backend.impl.tes.TesResponseJsonFormatter._ import cromwell.backend.standard.{StandardAsyncExecutionActor, StandardAsyncExecutionActorParams, StandardAsyncJob} +import cromwell.core.logging.JobLogger import cromwell.core.path.{DefaultPathBuilder, Path} import cromwell.core.retry.Retry._ import cromwell.core.retry.SimpleExponentialBackoff -import cromwell.filesystems.blob.{BlobPath, BlobPathBuilder} +import cromwell.filesystems.blob.BlobPathBuilder.ValidBlobPath +import cromwell.filesystems.blob.{BlobContainerName, BlobPath, BlobPathBuilder, WSMBlobSasTokenGenerator} import cromwell.filesystems.drs.{DrsPath, DrsResolver} import net.ceedubs.ficus.Ficus._ import wom.values.WomFile @@ -114,31 +116,35 @@ object TesAsyncBackendJobExecutionActor { */ def determineWSMSasEndpointFromInputs(taskInputs: List[Input], pathGetter: String => Try[Path], - blobConverter: Try[Path] => Try[BlobPath] = maybeConvertToBlob): Option[String] = { - val shouldLocalizeSas = true //TODO: Make this a Workflow Option or come from the WDL - if (!shouldLocalizeSas) return None - + logger: JobLogger, + blobConverter: Try[Path] => Try[BlobPath] = maybeConvertToBlob): Try[String] = { // Collect all of the inputs that are valid blob paths val blobFiles = taskInputs.collect{ - case input if input.url.isDefined => BlobPathBuilder.validateBlobPath(input.url.get) + case Input(_, _, Some(url), _, _, _) => BlobPathBuilder.validateBlobPath(url) }.collect{ case valid: BlobPathBuilder.ValidBlobPath => valid } - if(blobFiles.isEmpty) return None - // We use the first blob file in the list as a template for determining the localized sas params - val sasTokenEndpoint = for { - blob <- blobConverter(pathGetter(blobFiles.head.toUrl)) - wsmEndpoint <- blob.wsmEndpoint - workspaceId <- blob.parseTerraWorkspaceIdFromPath - containerResourceId <- blob.containerWSMResourceId - endpoint = s"$wsmEndpoint/api/workspaces/v1/$workspaceId/resources/controlled/azure/storageContainer/$containerResourceId/getSasToken" - } yield endpoint + // Log if not all input files live in the same container. + // We'll do our best anyway, but will still only be able to retrieve a token for a single container. + if(blobFiles.forall(_.container == blobFiles.headOption.map(file => file.container).getOrElse(BlobContainerName("no_container")))) { + logger.warn(s"While parsing blob inputs, found more than one container. Can only generate an environment sas token for a single blob container at once.") + } - sasTokenEndpoint match { - case good: Success[String] => Some(good.value) - case _: Any => None + // We use the first blob file in the list as a template for determining the localized sas params + val headBlob: Try[ValidBlobPath] = blobFiles.headOption match { + case Some(validBlob) => Try(validBlob) + case _ => Failure(new NoSuchElementException("No valid blob file for determining WSM end point found in task inputs.")) } + + for { + blobFile <- headBlob + blob <- blobConverter(pathGetter(blobFile.toUrl)) + endpoint <- blob.getFilesystemManager.blobTokenGenerator match { + case wsmGenerator: WSMBlobSasTokenGenerator => wsmGenerator.getWSMSasFetchEndpoint(blob) + case _ => Failure(new NoSuchElementException("This blob file does not have an associated WSMBlobSasTokenGenerator")) + } + } yield endpoint } } @@ -171,17 +177,18 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn } override def scriptPreamble: String = { - val tesTaskPreamble: String = runtimeAttributes.localizedSasEnvVar.map{enviornmentVariableName => - val workflowName = workflowDescriptor.callable.name - val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { + val tesTaskPreamble: String = runtimeAttributes.localizedSasEnvVar match { + case Some(environmentVariableName) => + val workflowName = workflowDescriptor.callable.name + val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { _.collectAsSeq { case w: WomFile => w } - } - val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) - val preamble = determineWSMSasEndpointFromInputs(taskInputs, getPath).map{ endpoint => - generateLocalizedSasScriptPreamble(enviornmentVariableName, endpoint) - }.getOrElse("") - preamble - }.getOrElse("") + } + val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) + determineWSMSasEndpointFromInputs(taskInputs, getPath, jobLogger).map { endpoint => + generateLocalizedSasScriptPreamble(environmentVariableName, endpoint) + }.getOrElse("") + case _ => "" + } super.scriptPreamble ++ tesTaskPreamble } diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala index a406a572bb0..48ade7b234a 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesRuntimeAttributes.scala @@ -33,7 +33,7 @@ object TesRuntimeAttributes { val DockerWorkingDirKey = "dockerWorkingDir" val DiskSizeKey = "disk" val PreemptibleKey = "preemptible" - val LocalizedSasKey = "sasEnvironmentVariable" + val LocalizedSasKey = "azureSasEnvironmentVariable" private def cpuValidation(runtimeConfig: Option[Config]): OptionalRuntimeAttributesValidation[Int Refined Positive] = CpuValidation.optional diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 5b73ce59ac9..dc5db08cdb3 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -1,11 +1,12 @@ package cromwell.backend.impl.tes import common.mock.MockSugar +import cromwell.core.logging.JobLogger import cromwell.core.path -import cromwell.filesystems.blob.{BlobContainerName, BlobPath} +import cromwell.filesystems.blob.{BlobFileSystemManager, BlobPath, WSMBlobSasTokenGenerator} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers -import java.util.UUID + import scala.util.{Failure, Success, Try} class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers with MockSugar { @@ -61,16 +62,20 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val mockWorkspaceId = "e58ed763-928c-4155-0000-fdbaaadc15f3" val mockContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" + val mockLogger: JobLogger = mock[JobLogger] val mockBlobPath: BlobPath = mock[BlobPath] + val mockTokenGenerator: WSMBlobSasTokenGenerator = mock[WSMBlobSasTokenGenerator] + val mockFsm: BlobFileSystemManager = mock[BlobFileSystemManager] + + mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath) returns Try(s"$mockWsmEndpoint/api/workspaces/v1/$mockWorkspaceId/resources/controlled/azure/storageContainer/$mockContainerResourceId/getSasToken") + mockFsm.blobTokenGenerator returns mockTokenGenerator + val mockNioPath: path.NioPath = mock[path.NioPath] - mockBlobPath.container returns BlobContainerName("1234") - mockBlobPath.wsmEndpoint returns Try(mockWsmEndpoint) - mockBlobPath.parseTerraWorkspaceIdFromPath returns Try(UUID.fromString(mockWorkspaceId)) - mockBlobPath.containerWSMResourceId returns Try(UUID.fromString(mockContainerResourceId)) - mockBlobPath.nioPath returns mockNioPath - mockBlobPath.md5 returns "BLOB_MD5" + mockBlobPath.getFilesystemManager returns mockFsm mockBlobPath.toAbsolutePath returns mockBlobPath + mockBlobPath.md5 returns "MOCK_MD5" + val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] def mockPathGetter(pathString: String): Try[cromwell.core.path.Path] = { @@ -82,14 +87,14 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit def mockBlobConverter(pathToConvert: Try[cromwell.core.path.Path]): Try[BlobPath] = { //using a stubbed md5 rather than matching on type because type matching of mocked types at runtime causes problems - if (pathToConvert.get.md5.equals("BLOB_MD5")) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) + if (pathToConvert.get.md5.equals("MOCK_MD5")) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) } it should "not return sas endpoint when no blob paths are provided" in { val emptyInputs: List[Input] = List() val bloblessInputs: List[Input] = List(notBlobInput_1, notBlobInput_2) - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(emptyInputs, mockPathGetter, mockBlobConverter).isEmpty shouldBe true - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(bloblessInputs, mockPathGetter, mockBlobConverter).isEmpty shouldBe true + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(emptyInputs, mockPathGetter, mockLogger, mockBlobConverter).isFailure shouldBe true + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(bloblessInputs, mockPathGetter, mockLogger, mockBlobConverter).isFailure shouldBe true } it should "return a sas endpoint based on inputs when blob paths are provided" in { @@ -97,8 +102,8 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val blobInput: List[Input] = List(blobInput_0) val blobInputs: List[Input] = List(blobInput_0, blobInput_1) val mixedInputs: List[Input] = List(notBlobInput_1, blobInput_0, blobInput_1) - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInput, mockPathGetter, mockBlobConverter).get shouldEqual expected - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, mockPathGetter, mockBlobConverter).get shouldEqual expected - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, mockPathGetter, mockBlobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInput, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected } } From f1e9ea76bb0510fb50e34ad0a8a0b2da256f9909 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 31 Oct 2023 16:02:10 -0400 Subject: [PATCH 18/34] improve bash script and fix test --- .../TesAsyncBackendJobExecutionActor.scala | 36 +++++++++++++++++-- .../impl/tes/TesRuntimeAttributesSpec.scala | 4 +-- 2 files changed, 35 insertions(+), 5 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 7d823101e02..cdb729a3d2a 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -67,9 +67,39 @@ object TesAsyncBackendJobExecutionActor { s""" |### BEGIN ACQUIRE LOCAL SAS TOKEN ### |# Install dependencies - |apt-get -y update - |apt-get -y install curl - |apt-get -y install jq + | + |# Function to check if a command exists + |command_exists() { + | command -v "$$1" > /dev/null 2>&1 + |} + | + |# Check if curl exists; install if not + |if ! command_exists curl; then + | if command_exists apt-get; then + | apt-get -y update && apt-get -y install curl + | if [ $$? -ne 0 ]; then + | echo "Error: Failed to install curl via apt-get." + | exit 1 + | fi + | else + | echo "Error: apt-get is not available, and curl is not installed." + | exit 1 + | fi + |fi + | + |# Check if jq exists; install if not + |if ! command_exists jq; then + | if command_exists apt-get; then + | apt-get -y update && apt-get -y install jq + | if [ $$? -ne 0 ]; then + | echo "Error: Failed to install jq via apt-get." + | exit 1 + | fi + | else + | echo "Error: apt-get is not available, and jq is not installed." + | exit 1 + | fi + |fi | |# Acquire bearer token, relying on the User Assigned Managed Identity of this VM. |echo Acquiring Bearer Token using User Assigned Managed Identity... diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala index d5e30730af9..4197f5da3d3 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala @@ -73,13 +73,13 @@ class TesRuntimeAttributesSpec extends AnyWordSpecLike with CromwellTimeoutSpec } "validate a valid sasEnvironmentVariable entry" in { - val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), "sasEnvironmentVariable" -> WomString("THIS_IS_VALID")) + val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), TesRuntimeAttributes.LocalizedSasKey -> WomString("THIS_IS_VALID")) val expectedRuntimeAttributes = expectedDefaultsPlusUbuntuDocker.copy(localizedSasEnvVar = Some("THIS_IS_VALID")) assertSuccess(runtimeAttributes, expectedRuntimeAttributes) } "fail to validate an invalid sasEnvironmentVariable entry" in { - val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), "sasEnvironmentVariable" -> WomString("THIS IS INVALID")) + val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), TesRuntimeAttributes.LocalizedSasKey -> WomString("THIS IS INVALID")) assertFailure(runtimeAttributes, "Value must be a string containing only letters, numbers, and underscores.") } From 9c9a29329976beeadf660c1eb6ef46dbb79f1b27 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 31 Oct 2023 17:12:02 -0400 Subject: [PATCH 19/34] fix stub --- .../impl/tes/TesAsyncBackendJobExecutionActor.scala | 4 +--- .../impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 7 ++----- 2 files changed, 3 insertions(+), 8 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index cdb729a3d2a..c9a29c5259c 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -66,9 +66,7 @@ object TesAsyncBackendJobExecutionActor { // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. s""" |### BEGIN ACQUIRE LOCAL SAS TOKEN ### - |# Install dependencies - | - |# Function to check if a command exists + |# Function to check if a command exists on this machine |command_exists() { | command -v "$$1" > /dev/null 2>&1 |} diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index dc5db08cdb3..5e34117e42c 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -2,7 +2,6 @@ package cromwell.backend.impl.tes import common.mock.MockSugar import cromwell.core.logging.JobLogger -import cromwell.core.path import cromwell.filesystems.blob.{BlobFileSystemManager, BlobPath, WSMBlobSasTokenGenerator} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers @@ -70,14 +69,12 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath) returns Try(s"$mockWsmEndpoint/api/workspaces/v1/$mockWorkspaceId/resources/controlled/azure/storageContainer/$mockContainerResourceId/getSasToken") mockFsm.blobTokenGenerator returns mockTokenGenerator - val mockNioPath: path.NioPath = mock[path.NioPath] + val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] mockBlobPath.getFilesystemManager returns mockFsm - mockBlobPath.toAbsolutePath returns mockBlobPath + mockBlobPath.toAbsolutePath returns mockPath mockBlobPath.md5 returns "MOCK_MD5" - - val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] def mockPathGetter(pathString: String): Try[cromwell.core.path.Path] = { val foundBlobPath: Success[BlobPath] = Success(mockBlobPath) val foundNonBlobPath: Success[cromwell.core.path.Path] = Success(mockPath) From b1699d547752b847ea487fc8f6dc863c048f397b Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Wed, 1 Nov 2023 14:19:34 -0400 Subject: [PATCH 20/34] stubbing --- .../impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 5e34117e42c..e90fdc08a35 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -2,10 +2,12 @@ package cromwell.backend.impl.tes import common.mock.MockSugar import cromwell.core.logging.JobLogger +import cromwell.core.path import cromwell.filesystems.blob.{BlobFileSystemManager, BlobPath, WSMBlobSasTokenGenerator} import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers +import java.nio.file.Path import scala.util.{Failure, Success, Try} class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers with MockSugar { @@ -70,8 +72,12 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockFsm.blobTokenGenerator returns mockTokenGenerator val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] + val mockNioPath: path.NioPath = mock[path.NioPath] + val mockJavaPath: Path = mock[java.nio.file.Path] + mockNioPath.toAbsolutePath returns mockJavaPath mockBlobPath.getFilesystemManager returns mockFsm + mockBlobPath.nioPath returns mockNioPath mockBlobPath.toAbsolutePath returns mockPath mockBlobPath.md5 returns "MOCK_MD5" From 31533450b97606d229ca864c4f36c6209722bded Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Thu, 2 Nov 2023 11:05:54 -0400 Subject: [PATCH 21/34] stubtry --- .../backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index e90fdc08a35..b400f2a965a 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -72,6 +72,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockFsm.blobTokenGenerator returns mockTokenGenerator val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] + mockPath.normalize() returns mockPath val mockNioPath: path.NioPath = mock[path.NioPath] val mockJavaPath: Path = mock[java.nio.file.Path] From f958150e5a20122b9f7f42fb3b02473c754c363b Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Thu, 2 Nov 2023 15:35:23 -0400 Subject: [PATCH 22/34] caching --- .../backend/impl/tes/TesAsyncBackendJobExecutionActor.scala | 2 +- .../impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 6 ++++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index c9a29c5259c..97c5bbd4b62 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -156,7 +156,7 @@ object TesAsyncBackendJobExecutionActor { // Log if not all input files live in the same container. // We'll do our best anyway, but will still only be able to retrieve a token for a single container. if(blobFiles.forall(_.container == blobFiles.headOption.map(file => file.container).getOrElse(BlobContainerName("no_container")))) { - logger.warn(s"While parsing blob inputs, found more than one container. Can only generate an environment sas token for a single blob container at once.") + logger.info(s"While parsing blob inputs, found more than one container. Can only generate an environment sas token for a single blob container at once.") } // We use the first blob file in the list as a template for determining the localized sas params diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index b400f2a965a..98843f43bdd 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -72,11 +72,13 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockFsm.blobTokenGenerator returns mockTokenGenerator val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] - mockPath.normalize() returns mockPath + mockPath.normalize returns mockPath + val mockNioPath: path.NioPath = mock[path.NioPath] val mockJavaPath: Path = mock[java.nio.file.Path] - mockNioPath.toAbsolutePath returns mockJavaPath + mockNioPath.normalize returns mockNioPath + mockBlobPath.getFilesystemManager returns mockFsm mockBlobPath.nioPath returns mockNioPath mockBlobPath.toAbsolutePath returns mockPath From 4146ac11c501e219eeb87d56e2dac24906265bbd Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Thu, 2 Nov 2023 15:40:24 -0400 Subject: [PATCH 23/34] caching --- .../blob/BlobFileSystemManager.scala | 23 +++++++++++++++---- 1 file changed, 18 insertions(+), 5 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index 2bf2aedca47..43fb1eecdd1 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -15,6 +15,7 @@ import java.nio.file.spi.FileSystemProvider import java.time.temporal.ChronoUnit import java.time.{Duration, OffsetDateTime} import java.util.UUID +import scala.collection.mutable import scala.jdk.CollectionConverters._ import scala.util.{Failure, Success, Try} @@ -160,14 +161,14 @@ object BlobSasTokenGenerator { */ def createBlobTokenGenerator(workspaceManagerClient: WorkspaceManagerApiClientProvider, overrideWsmAuthToken: Option[String]): BlobSasTokenGenerator = { - WSMBlobSasTokenGenerator(workspaceManagerClient, overrideWsmAuthToken) + new WSMBlobSasTokenGenerator(workspaceManagerClient, overrideWsmAuthToken) } } case class WSMTerraCoordinates(wsmEndpoint: String, workspaceId: UUID, containerResourceId: UUID) -case class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvider, +class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvider, overrideWsmAuthToken: Option[String]) extends BlobSasTokenGenerator { /** @@ -200,9 +201,21 @@ case class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClient } } - def getContainerResourceId(workspaceId: UUID, container: BlobContainerName, wsmAuth : String): Try[UUID] = { - val wsmResourceClient = wsmClientProvider.getResourceApi(wsmAuth) - wsmResourceClient.findContainerResourceId(workspaceId, container) + private val cachedContainerResourceIds = new mutable.HashMap[BlobContainerName, UUID]() + + private def getContainerResourceId(workspaceId: UUID, container: BlobContainerName, wsmAuth : String): Try[UUID] = { + cachedContainerResourceIds.get(container) match { + case Some(id) => Try(id) + case _ => { + val wsmResourceClient = wsmClientProvider.getResourceApi(wsmAuth) + val resourceId = wsmResourceClient.findContainerResourceId(workspaceId, container) + resourceId.map(id => cachedContainerResourceIds.put(container, id)) //NB: Modifying cache state here. + cachedContainerResourceIds.get(container) match { + case Some(uuid) => Try(uuid) + case _ => Failure(new NoSuchElementException("Could not retrieve container resource ID from WSM")) + } + } + } } private def getWsmAuth: Try[String] = { From f694f65bfb12045fb596f4eb19631c4adc84bce6 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Fri, 3 Nov 2023 13:16:51 -0400 Subject: [PATCH 24/34] parens --- .../backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 98843f43bdd..b570f218aa2 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -77,7 +77,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val mockNioPath: path.NioPath = mock[path.NioPath] val mockJavaPath: Path = mock[java.nio.file.Path] mockNioPath.toAbsolutePath returns mockJavaPath - mockNioPath.normalize returns mockNioPath + mockNioPath.normalize() returns mockNioPath mockBlobPath.getFilesystemManager returns mockFsm mockBlobPath.nioPath returns mockNioPath From 8e05ca0a0c5c510c698d4293c0975ffc25f23747 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Fri, 3 Nov 2023 16:55:03 -0400 Subject: [PATCH 25/34] some but not all Friday Feedback --- .../blob/BlobFileSystemManager.scala | 41 +++++++++++-------- ...TesAsyncBackendJobExecutionActorSpec.scala | 2 +- 2 files changed, 25 insertions(+), 18 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index 43fb1eecdd1..9be0ece9d55 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -188,7 +188,7 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi (for { wsmAuth <- wsmAuthToken wsmAzureResourceClient = wsmClientProvider.getControlledAzureResourceApi(wsmAuth) - resourceId <- getContainerResourceId(workspaceId, container, wsmAuth) + resourceId <- getContainerResourceId(workspaceId, container, Some(wsmAuth)) sasToken <- wsmAzureResourceClient.createAzureStorageContainerSasToken(workspaceId, resourceId) } yield sasToken).recoverWith { // If the storage account was still not found in WSM, this may be a public filesystem @@ -203,19 +203,25 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi private val cachedContainerResourceIds = new mutable.HashMap[BlobContainerName, UUID]() - private def getContainerResourceId(workspaceId: UUID, container: BlobContainerName, wsmAuth : String): Try[UUID] = { - cachedContainerResourceIds.get(container) match { - case Some(id) => Try(id) - case _ => { - val wsmResourceClient = wsmClientProvider.getResourceApi(wsmAuth) - val resourceId = wsmResourceClient.findContainerResourceId(workspaceId, container) - resourceId.map(id => cachedContainerResourceIds.put(container, id)) //NB: Modifying cache state here. - cachedContainerResourceIds.get(container) match { - case Some(uuid) => Try(uuid) - case _ => Failure(new NoSuchElementException("Could not retrieve container resource ID from WSM")) - } - } - } + // Optionally provide wsmAuth to avoid acquiring it twice in generateBlobSasToken. + // In the case that the resourceId is not cached and no auth is provided, this function will acquire a new auth as necessary. + private def getContainerResourceId(workspaceId: UUID, container: BlobContainerName, precomputedWsmAuth: Option[String]): Try[UUID] = { + cachedContainerResourceIds.get(container) match { + case Some(id) => Try(id) //cache hit + case _ => { //cache miss + val auth: Try[String] = precomputedWsmAuth.map(auth => Try(auth)).getOrElse(getWsmAuth) + val resourceId = for { + wsmAuth <- auth + wsmResourceApi = wsmClientProvider.getResourceApi(wsmAuth) + resourceId <- wsmResourceApi.findContainerResourceId(workspaceId, container) + } yield resourceId + resourceId.map(id => cachedContainerResourceIds.put(container, id)) //NB: Modifying cache state here. + cachedContainerResourceIds.get(container) match { + case Some(uuid) => Try(uuid) + case _ => Failure(new NoSuchElementException("Could not retrieve container resource ID from WSM")) + } + } + } } private def getWsmAuth: Try[String] = { @@ -224,6 +230,7 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi case None => AzureCredentials.getAccessToken(None).toTry } } + private def parseTerraWorkspaceIdFromPath(blobPath: BlobPath): Try[UUID] = { if (blobPath.container.value.startsWith("sc-")) Try(UUID.fromString(blobPath.container.value.substring(3))) else Failure(new Exception("Could not parse workspace ID from storage container. Are you sure this is a file in a Terra Workspace?")) @@ -234,14 +241,14 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi * and proper authentication, will return a sas token capable of accessing the container the blob path points to. * @param blobPath A blob path of a file living in a blob container that WSM knows about (likely a workspace container). * - * NOTE: This function makes two synchronous REST requests. + * NOTE: If requesting the sas endpoint of a container that *isn't* what this token generator was constructed for, this + * function may make two REST requests. Otherwise, it's safe to assume that the relevant data is already cached locally. */ def getWSMSasFetchEndpoint(blobPath: BlobPath): Try[String] = { val wsmEndpoint = wsmClientProvider.getBaseWorkspaceManagerUrl val terraInfo: Try[WSMTerraCoordinates] = for { workspaceId <- parseTerraWorkspaceIdFromPath(blobPath) - auth <- getWsmAuth - containerResourceId <- getContainerResourceId(workspaceId, blobPath.container, auth) + containerResourceId <- getContainerResourceId(workspaceId, blobPath.container, None) coordinates = WSMTerraCoordinates(wsmEndpoint, workspaceId, containerResourceId) } yield coordinates terraInfo.map{terraCoordinates => diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index b570f218aa2..3043f14110b 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -72,7 +72,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit mockFsm.blobTokenGenerator returns mockTokenGenerator val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] - mockPath.normalize returns mockPath + mockPath.normalize() returns mockPath val mockNioPath: path.NioPath = mock[path.NioPath] val mockJavaPath: Path = mock[java.nio.file.Path] From a62307afde335edc65b4070c8cc43997da360443 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 6 Nov 2023 16:23:30 -0500 Subject: [PATCH 26/34] PR feedback --- .../StandardAsyncExecutionActor.scala | 7 +- .../blob/BlobFileSystemManager.scala | 7 +- .../filesystems/blob/BlobPathBuilder.scala | 4 +- .../WorkspaceManagerApiClientProvider.scala | 5 +- ...inesApiAsyncBackendJobExecutionActor.scala | 4 +- .../TesAsyncBackendJobExecutionActor.scala | 78 +++++++++-------- ...TesAsyncBackendJobExecutionActorSpec.scala | 84 +++++++++++++------ .../impl/tes/TesInitializationActorSpec.scala | 2 +- .../impl/tes/TesRuntimeAttributesSpec.scala | 4 +- 9 files changed, 113 insertions(+), 82 deletions(-) diff --git a/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala b/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala index c98e429c63d..12cbf7d6cef 100644 --- a/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala +++ b/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala @@ -1,7 +1,6 @@ package cromwell.backend.standard import java.io.IOException - import akka.actor.{Actor, ActorLogging, ActorRef} import akka.event.LoggingReceive import cats.implicits._ @@ -329,7 +328,7 @@ trait StandardAsyncExecutionActor } /** Any custom code that should be run within commandScriptContents before the instantiated command. */ - def scriptPreamble: String = "" + def scriptPreamble: ErrorOr[String] = "".valid def cwd: Path = commandDirectory def rcPath: Path = cwd./(jobPaths.returnCodeFilename) @@ -430,7 +429,7 @@ trait StandardAsyncExecutionActor // The `tee` trickery below is to be able to redirect to known filenames for CWL while also streaming // stdout and stderr for PAPI to periodically upload to cloud storage. // https://stackoverflow.com/questions/692000/how-do-i-write-stderr-to-a-file-while-using-tee-with-a-pipe - (errorOrDirectoryOutputs, errorOrGlobFiles).mapN((directoryOutputs, globFiles) => + (errorOrDirectoryOutputs, errorOrGlobFiles, scriptPreamble).mapN((directoryOutputs, globFiles, preamble) => s"""|#!$jobShell |DOCKER_OUTPUT_DIR_LINK |cd ${cwd.pathAsString} @@ -464,7 +463,7 @@ trait StandardAsyncExecutionActor |) |mv $rcTmpPath $rcPath |""".stripMargin - .replace("SCRIPT_PREAMBLE", scriptPreamble) + .replace("SCRIPT_PREAMBLE", preamble) .replace("ENVIRONMENT_VARIABLES", environmentVariables) .replace("INSTANTIATED_COMMAND", commandString) .replace("SCRIPT_EPILOGUE", scriptEpilogue) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index 9be0ece9d55..07fdd4c59cc 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -237,12 +237,11 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi } /** - * If the provided blob path looks like it comes from a terra workspace, return an end point that, when called with GET - * and proper authentication, will return a sas token capable of accessing the container the blob path points to. + * Return a REST endpoint that will reply with a sas token for the blob storage container associated with the provided blob path. * @param blobPath A blob path of a file living in a blob container that WSM knows about (likely a workspace container). * - * NOTE: If requesting the sas endpoint of a container that *isn't* what this token generator was constructed for, this - * function may make two REST requests. Otherwise, it's safe to assume that the relevant data is already cached locally. + * NOTE: If a blobPath is provided for a file in a container other than what this token generator was constructed for, + * this function will make two REST requests. Otherwise, the relevant data is already cached locally. */ def getWSMSasFetchEndpoint(blobPath: BlobPath): Try[String] = { val wsmEndpoint = wsmClientProvider.getBaseWorkspaceManagerUrl diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala index 2051ca18162..3acb99857e0 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobPathBuilder.scala @@ -14,9 +14,7 @@ import scala.util.{Failure, Success, Try} object BlobPathBuilder { private val blobHostnameSuffix = ".blob.core.windows.net" sealed trait BlobPathValidation - case class ValidBlobPath(path: String, container: BlobContainerName, endpoint: EndpointURL) extends BlobPathValidation { - def toUrl: String = endpoint.value + "/" + container.value + path - } + case class ValidBlobPath(path: String, container: BlobContainerName, endpoint: EndpointURL) extends BlobPathValidation case class UnparsableBlobPath(errorMessage: Throwable) extends BlobPathValidation def invalidBlobHostMessage(endpoint: EndpointURL) = s"Malformed Blob URL for this builder: The endpoint $endpoint doesn't contain the expected host string '{SA}.blob.core.windows.net/'" diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala index 4d09cafc96e..490d0fcc704 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/WorkspaceManagerApiClientProvider.scala @@ -41,10 +41,7 @@ class HttpWorkspaceManagerClientProvider(baseWorkspaceManagerUrl: WorkspaceManag apiClient.setAccessToken(token) WsmControlledAzureResourceApi(new ControlledAzureResourceApi(apiClient)) } - - def getBaseWorkspaceManagerUrl: String = { - baseWorkspaceManagerUrl.value - } + def getBaseWorkspaceManagerUrl: String = baseWorkspaceManagerUrl.value } case class WsmResourceApi(resourcesApi : ResourceApi) { diff --git a/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiAsyncBackendJobExecutionActor.scala b/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiAsyncBackendJobExecutionActor.scala index 745e11bee35..942838f8125 100644 --- a/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiAsyncBackendJobExecutionActor.scala @@ -380,12 +380,12 @@ class PipelinesApiAsyncBackendJobExecutionActor(override val standardParams: Sta private lazy val isDockerImageCacheUsageRequested = runtimeAttributes.useDockerImageCache.getOrElse(useDockerImageCache(jobDescriptor.workflowDescriptor)) - override def scriptPreamble: String = { + override def scriptPreamble: ErrorOr[String] = { if (monitoringOutput.isDefined) { s"""|touch $DockerMonitoringLogPath |chmod u+x $DockerMonitoringScriptPath |$DockerMonitoringScriptPath > $DockerMonitoringLogPath &""".stripMargin - } else "" + }.valid else "".valid } override def globParentDirectory(womGlobFile: WomGlobFile): Path = { diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 97c5bbd4b62..338369584e8 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -8,7 +8,7 @@ import akka.http.scaladsl.model._ import akka.http.scaladsl.unmarshalling.{Unmarshal, Unmarshaller} import akka.stream.ActorMaterializer import akka.util.ByteString -import cats.syntax.apply._ +import cats.implicits._ import common.collections.EnhancedCollections._ import common.exception.AggregatedMessageException import common.validation.ErrorOr.ErrorOr @@ -22,8 +22,7 @@ import cromwell.core.logging.JobLogger import cromwell.core.path.{DefaultPathBuilder, Path} import cromwell.core.retry.Retry._ import cromwell.core.retry.SimpleExponentialBackoff -import cromwell.filesystems.blob.BlobPathBuilder.ValidBlobPath -import cromwell.filesystems.blob.{BlobContainerName, BlobPath, BlobPathBuilder, WSMBlobSasTokenGenerator} +import cromwell.filesystems.blob.{BlobPath, WSMBlobSasTokenGenerator} import cromwell.filesystems.drs.{DrsPath, DrsResolver} import net.ceedubs.ficus.Ficus._ import wom.values.WomFile @@ -61,7 +60,8 @@ case object Cancelled extends TesRunStatus { object TesAsyncBackendJobExecutionActor { val JobIdKey = "tes_job_id" - private def generateLocalizedSasScriptPreamble(environmentVariableName: String, getSasWsmEndpoint: String) : String = { + + def generateLocalizedSasScriptPreamble(environmentVariableName: String, getSasWsmEndpoint: String) : String = { // BEARER_TOKEN: https://learn.microsoft.com/en-us/azure/active-directory/managed-identities-azure-resources/how-to-use-vm-token#get-a-token-using-http // NB: Scala string interpolation and bash variable substitution use similar syntax. $$ is an escaped $, much like \\ is an escaped \. s""" @@ -131,10 +131,10 @@ object TesAsyncBackendJobExecutionActor { } /** - * Under certain situations (and only on Terra), we want the VM running a TES task to have the ability to acquire a - * fresh sas token for itself. In order to be able to do this, we provide it with a precomputed endpoint it can use. + * Computes an endpoint that can be used to retrieve a sas token for a particular blob storage container. + * This assumes that some of the task inputs are blob files, all blob files are in the same container, and we can get a sas + * token for this container from WSM. * The task VM will use the user assigned managed identity that it is running as in order to authenticate. - * We only return a value if at least one of the inputs is a BlobPath and //TODO flag specified in WDL. * @param taskInputs The inputs to this particular TesTask. If any are blob files, the first will be used to * determine the storage container to retrieve the sas token for. * @param pathGetter A function to convert string filepath into a cromwell Path object. @@ -147,32 +147,22 @@ object TesAsyncBackendJobExecutionActor { logger: JobLogger, blobConverter: Try[Path] => Try[BlobPath] = maybeConvertToBlob): Try[String] = { // Collect all of the inputs that are valid blob paths - val blobFiles = taskInputs.collect{ - case Input(_, _, Some(url), _, _, _) => BlobPathBuilder.validateBlobPath(url) - }.collect{ - case valid: BlobPathBuilder.ValidBlobPath => valid - } + val blobFiles = taskInputs + .collect{ case Input(_, _, Some(url), _, _, _) => blobConverter(pathGetter(url)) } + .collect{ case Success(blob) => blob } // Log if not all input files live in the same container. - // We'll do our best anyway, but will still only be able to retrieve a token for a single container. - if(blobFiles.forall(_.container == blobFiles.headOption.map(file => file.container).getOrElse(BlobContainerName("no_container")))) { + if (blobFiles.map(_.container).distinct.size > 1) { logger.info(s"While parsing blob inputs, found more than one container. Can only generate an environment sas token for a single blob container at once.") } - // We use the first blob file in the list as a template for determining the localized sas params - val headBlob: Try[ValidBlobPath] = blobFiles.headOption match { - case Some(validBlob) => Try(validBlob) - case _ => Failure(new NoSuchElementException("No valid blob file for determining WSM end point found in task inputs.")) - } - - for { - blobFile <- headBlob - blob <- blobConverter(pathGetter(blobFile.toUrl)) - endpoint <- blob.getFilesystemManager.blobTokenGenerator match { - case wsmGenerator: WSMBlobSasTokenGenerator => wsmGenerator.getWSMSasFetchEndpoint(blob) - case _ => Failure(new NoSuchElementException("This blob file does not have an associated WSMBlobSasTokenGenerator")) + // We use the first blob file in the list to determine the correct blob container. + blobFiles.headOption.map{blobPath => + blobPath.getFilesystemManager.blobTokenGenerator match { + case wsmGenerator: WSMBlobSasTokenGenerator => wsmGenerator.getWSMSasFetchEndpoint(blobPath) + case _ => Failure(new UnsupportedOperationException("Blob file does not have an associated WSMBlobSasTokenGenerator")) } - } yield endpoint + }.getOrElse(Failure(new NoSuchElementException("Could not infer blob storage container from task inputs: No valid blob files provided."))) } } @@ -204,20 +194,36 @@ class TesAsyncBackendJobExecutionActor(override val standardParams: StandardAsyn ) } - override def scriptPreamble: String = { - val tesTaskPreamble: String = runtimeAttributes.localizedSasEnvVar match { - case Some(environmentVariableName) => + /** + * This script preamble is bash code that is executed at the start of a task inside the user's container. + * It is executed directly before the user's instantiated command is, which gives cromwell a chance to adjust the + * container environment before the actual task runs. See commandScriptContents in StandardAsyncExecutionActor for more context. + * + * For TES tasks, we sometimes want to acquire and save an azure sas token to an environment variable. + * If the user provides a value for runtimeAttributes.localizedSasEnvVar, we will add the relevant bash code to the preamble + * that acquires/exports the sas token to an environment variable. Once there, it will be visible to the user's task code. + * + * If runtimeAttributes.localizedSasEnvVar is provided in the WDL (and determineWSMSasEndpointFromInputs is successful), + * we will export the sas token to an environment variable named to be the value of runtimeAttributes.localizedSasEnvVar. + * Otherwise, we won't alter the preamble. + * + * See determineWSMSasEndpointFromInputs to see how we use taskInputs to infer *which* container to get a sas token for. + * + * @return Bash code to run at the start of a task. + */ + override def scriptPreamble: ErrorOr[String] = { + runtimeAttributes.localizedSasEnvVar match { + case Some(environmentVariableName) => { // Case: user wants a sas token. Return the computed preamble or die trying. val workflowName = workflowDescriptor.callable.name val callInputFiles = jobDescriptor.fullyQualifiedInputs.safeMapValues { - _.collectAsSeq { case w: WomFile => w } + _.collectAsSeq { case w: WomFile => w } } val taskInputs: List[Input] = TesTask.buildTaskInputs(callInputFiles, workflowName, mapCommandLineWomFile) - determineWSMSasEndpointFromInputs(taskInputs, getPath, jobLogger).map { endpoint => - generateLocalizedSasScriptPreamble(environmentVariableName, endpoint) - }.getOrElse("") - case _ => "" + val computedEndpoint = determineWSMSasEndpointFromInputs(taskInputs, getPath, jobLogger) + computedEndpoint.map(endpoint => generateLocalizedSasScriptPreamble(environmentVariableName, endpoint)) + }.toErrorOr + case _ => "".valid // Case: user doesn't want a sas token. Empty preamble is the correct preamble. } - super.scriptPreamble ++ tesTaskPreamble } override def mapCommandLineWomFile(womFile: WomFile): WomFile = { diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 3043f14110b..cf957088ca0 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -59,34 +59,37 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit ) // Mock blob path functionality. - val mockWsmEndpoint = "https://wsm.mock.com/endpoint" - val mockWorkspaceId = "e58ed763-928c-4155-0000-fdbaaadc15f3" - val mockContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" - - val mockLogger: JobLogger = mock[JobLogger] - val mockBlobPath: BlobPath = mock[BlobPath] - val mockTokenGenerator: WSMBlobSasTokenGenerator = mock[WSMBlobSasTokenGenerator] - val mockFsm: BlobFileSystemManager = mock[BlobFileSystemManager] - - mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath) returns Try(s"$mockWsmEndpoint/api/workspaces/v1/$mockWorkspaceId/resources/controlled/azure/storageContainer/$mockContainerResourceId/getSasToken") - mockFsm.blobTokenGenerator returns mockTokenGenerator - - val mockPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] - mockPath.normalize() returns mockPath - - val mockNioPath: path.NioPath = mock[path.NioPath] - val mockJavaPath: Path = mock[java.nio.file.Path] - mockNioPath.toAbsolutePath returns mockJavaPath - mockNioPath.normalize() returns mockNioPath - - mockBlobPath.getFilesystemManager returns mockFsm - mockBlobPath.nioPath returns mockNioPath - mockBlobPath.toAbsolutePath returns mockPath - mockBlobPath.md5 returns "MOCK_MD5" + val testWsmEndpoint = "https://wsm.mock.com/endpoint" + val testWorkspaceId = "e58ed763-928c-4155-0000-fdbaaadc15f3" + val testContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" + + def generateMockBlobPath: BlobPath = { + val mockCromwellPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] + mockCromwellPath.normalize() returns mockCromwellPath + val mockNioPath: path.NioPath = mock[path.NioPath] + val mockJavaPath: Path = mock[java.nio.file.Path] + mockNioPath.toAbsolutePath returns mockJavaPath + mockNioPath.normalize() returns mockNioPath + + val mockBlobPath = mock[BlobPath] + mockBlobPath.nioPath returns mockNioPath + mockBlobPath.toAbsolutePath returns mockCromwellPath + mockBlobPath.md5 returns "MOCK_MD5" + + val mockTokenGenerator: WSMBlobSasTokenGenerator = mock[WSMBlobSasTokenGenerator] + val mockFsm: BlobFileSystemManager = mock[BlobFileSystemManager] + mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath) returns Try(s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken") + mockFsm.blobTokenGenerator returns mockTokenGenerator + mockBlobPath.getFilesystemManager returns mockFsm + + mockBlobPath + } def mockPathGetter(pathString: String): Try[cromwell.core.path.Path] = { + val mockBlobPath = generateMockBlobPath + val mockCromwellPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] val foundBlobPath: Success[BlobPath] = Success(mockBlobPath) - val foundNonBlobPath: Success[cromwell.core.path.Path] = Success(mockPath) + val foundNonBlobPath: Success[cromwell.core.path.Path] = Success(mockCromwellPath) if (pathString.equals(blobInput_0.url.get) || pathString.equals(blobInput_1.url.get)) return foundBlobPath foundNonBlobPath } @@ -97,6 +100,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit } it should "not return sas endpoint when no blob paths are provided" in { + val mockLogger: JobLogger = mock[JobLogger] val emptyInputs: List[Input] = List() val bloblessInputs: List[Input] = List(notBlobInput_1, notBlobInput_2) TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(emptyInputs, mockPathGetter, mockLogger, mockBlobConverter).isFailure shouldBe true @@ -104,7 +108,8 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit } it should "return a sas endpoint based on inputs when blob paths are provided" in { - val expected = s"$mockWsmEndpoint/api/workspaces/v1/$mockWorkspaceId/resources/controlled/azure/storageContainer/$mockContainerResourceId/getSasToken" + val mockLogger: JobLogger = mock[JobLogger] + val expected = s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken" val blobInput: List[Input] = List(blobInput_0) val blobInputs: List[Input] = List(blobInput_0, blobInput_1) val mixedInputs: List[Input] = List(notBlobInput_1, blobInput_0, blobInput_1) @@ -112,4 +117,31 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected } + + it should "contain expected strings in the bash script" in { + + val mockEnvironmentVariableNameFromWom = "mock_env_var_for_storing_sas_token" + val expectedEndpoint = s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken" + + val beginSubstring = "### BEGIN ACQUIRE LOCAL SAS TOKEN ###" + val endSubstring = "### END ACQUIRE LOCAL SAS TOKEN ###" + val curlCommandSubstring = + s""" + |sas_response_json=$$(curl -s \\ + | --retry 3 \\ + | --retry-delay 2 \\ + | -X POST "$expectedEndpoint" \\ + | -H "Content-Type: application/json" \\ + | -H "accept: */*" \\ + | -H "Authorization: Bearer $${BEARER_TOKEN}") + |""".stripMargin + val exportCommandSubstring = s"""export $mockEnvironmentVariableNameFromWom=$$(echo "$${sas_response_json}" | jq -r '.token')""" + + val generatedBashScript = TesAsyncBackendJobExecutionActor.generateLocalizedSasScriptPreamble(mockEnvironmentVariableNameFromWom, expectedEndpoint) + + generatedBashScript should include (beginSubstring) + generatedBashScript should include (endSubstring) + generatedBashScript should include (curlCommandSubstring) + generatedBashScript should include (exportCommandSubstring) + } } diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala index a1b2e4e1cd9..a081f26c910 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesInitializationActorSpec.scala @@ -63,7 +63,7 @@ class TesInitializationActorSpec extends TestKitSuite | # The keys below have been commented out as they are optional runtime attributes. | # dockerWorkingDir | # docker - | # sasEnvironmentVariable + | # azureSasEnvironmentVariable |} |""".stripMargin diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala index 4197f5da3d3..830e0cbe70c 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesRuntimeAttributesSpec.scala @@ -72,13 +72,13 @@ class TesRuntimeAttributesSpec extends AnyWordSpecLike with CromwellTimeoutSpec assertSuccess(runtimeAttributes, expectedRuntimeAttributes) } - "validate a valid sasEnvironmentVariable entry" in { + "validate a valid azureSasEnvironmentVariable entry" in { val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), TesRuntimeAttributes.LocalizedSasKey -> WomString("THIS_IS_VALID")) val expectedRuntimeAttributes = expectedDefaultsPlusUbuntuDocker.copy(localizedSasEnvVar = Some("THIS_IS_VALID")) assertSuccess(runtimeAttributes, expectedRuntimeAttributes) } - "fail to validate an invalid sasEnvironmentVariable entry" in { + "fail to validate an invalid azureSasEnvironmentVariable entry" in { val runtimeAttributes = Map("docker" -> WomString("ubuntu:latest"), TesRuntimeAttributes.LocalizedSasKey -> WomString("THIS IS INVALID")) assertFailure(runtimeAttributes, "Value must be a string containing only letters, numbers, and underscores.") } From e7cd74cc59ca08ede059808bb32eeca5837fbd1a Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 7 Nov 2023 10:00:03 -0500 Subject: [PATCH 27/34] missed file --- .../actors/GcpBatchAsyncBackendJobExecutionActor.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/supportedBackends/google/batch/src/main/scala/cromwell/backend/google/batch/actors/GcpBatchAsyncBackendJobExecutionActor.scala b/supportedBackends/google/batch/src/main/scala/cromwell/backend/google/batch/actors/GcpBatchAsyncBackendJobExecutionActor.scala index ae228ad503b..766a8f2552f 100644 --- a/supportedBackends/google/batch/src/main/scala/cromwell/backend/google/batch/actors/GcpBatchAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/google/batch/src/main/scala/cromwell/backend/google/batch/actors/GcpBatchAsyncBackendJobExecutionActor.scala @@ -663,12 +663,12 @@ class GcpBatchAsyncBackendJobExecutionActor(override val standardParams: Standar private val DockerMonitoringLogPath: Path = GcpBatchWorkingDisk.MountPoint.resolve(gcpBatchCallPaths.batchMonitoringLogFilename) private val DockerMonitoringScriptPath: Path = GcpBatchWorkingDisk.MountPoint.resolve(gcpBatchCallPaths.batchMonitoringScriptFilename) - override def scriptPreamble: String = { + override def scriptPreamble: ErrorOr[String] = { if (monitoringOutput.isDefined) { s"""|touch $DockerMonitoringLogPath |chmod u+x $DockerMonitoringScriptPath - |$DockerMonitoringScriptPath > $DockerMonitoringLogPath &""".stripMargin - } else "" + |$DockerMonitoringScriptPath > $DockerMonitoringLogPath &""".stripMargin.valid + } else "".valid } private[actors] def generateInputs(jobDescriptor: BackendJobDescriptor): Set[GcpBatchInput] = { From 510aa31ecd7216275cb76c5efa1ce845a532053b Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 7 Nov 2023 14:31:59 -0500 Subject: [PATCH 28/34] more sensible mocks --- .../blob/BlobFileSystemManager.scala | 7 +-- .../TesAsyncBackendJobExecutionActor.scala | 4 +- ...TesAsyncBackendJobExecutionActorSpec.scala | 46 +++++++++++-------- 3 files changed, 34 insertions(+), 23 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index 07fdd4c59cc..79be0b0c0eb 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -239,11 +239,12 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi /** * Return a REST endpoint that will reply with a sas token for the blob storage container associated with the provided blob path. * @param blobPath A blob path of a file living in a blob container that WSM knows about (likely a workspace container). - * + * @param tokenDuration How long will the token last after being generated. Default is 8 hours. Sas tokens won't last longer than 24h. * NOTE: If a blobPath is provided for a file in a container other than what this token generator was constructed for, * this function will make two REST requests. Otherwise, the relevant data is already cached locally. */ - def getWSMSasFetchEndpoint(blobPath: BlobPath): Try[String] = { + def getWSMSasFetchEndpoint(blobPath: BlobPath, tokenDuration: Option[Duration] = None): Try[String] = { + val lifetimeSeconds: Int = tokenDuration.map(d => d.toSeconds.intValue).getOrElse(8*60*60) val wsmEndpoint = wsmClientProvider.getBaseWorkspaceManagerUrl val terraInfo: Try[WSMTerraCoordinates] = for { workspaceId <- parseTerraWorkspaceIdFromPath(blobPath) @@ -251,7 +252,7 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi coordinates = WSMTerraCoordinates(wsmEndpoint, workspaceId, containerResourceId) } yield coordinates terraInfo.map{terraCoordinates => - s"${terraCoordinates.wsmEndpoint}/api/workspaces/v1/${terraCoordinates.workspaceId.toString}/resources/controlled/azure/storageContainer/${terraCoordinates.containerResourceId.toString}/getSasToken" + s"${terraCoordinates.wsmEndpoint}/api/workspaces/v1/${terraCoordinates.workspaceId.toString}/resources/controlled/azure/storageContainer/${terraCoordinates.containerResourceId.toString}/getSasToken?sasExpirationDuration=$lifetimeSeconds" } } } diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 338369584e8..1a935c1a64a 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -29,6 +29,8 @@ import wom.values.WomFile import java.io.FileNotFoundException import java.nio.file.FileAlreadyExistsException +import java.time.Duration +import java.time.temporal.ChronoUnit import scala.concurrent.Future import scala.util.{Failure, Success, Try} sealed trait TesRunStatus { @@ -159,7 +161,7 @@ object TesAsyncBackendJobExecutionActor { // We use the first blob file in the list to determine the correct blob container. blobFiles.headOption.map{blobPath => blobPath.getFilesystemManager.blobTokenGenerator match { - case wsmGenerator: WSMBlobSasTokenGenerator => wsmGenerator.getWSMSasFetchEndpoint(blobPath) + case wsmGenerator: WSMBlobSasTokenGenerator => wsmGenerator.getWSMSasFetchEndpoint(blobPath, Some(Duration.of(24, ChronoUnit.HOURS))) case _ => Failure(new UnsupportedOperationException("Blob file does not have an associated WSMBlobSasTokenGenerator")) } }.getOrElse(Failure(new NoSuchElementException("Could not infer blob storage container from task inputs: No valid blob files provided."))) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index cf957088ca0..f7de7f318a4 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -8,7 +8,9 @@ import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers import java.nio.file.Path -import scala.util.{Failure, Success, Try} +import java.time.Duration +import java.time.temporal.ChronoUnit +import scala.util.{Failure, Try} class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers with MockSugar { behavior of "TesAsyncBackendJobExecutionActor" @@ -63,6 +65,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val testWorkspaceId = "e58ed763-928c-4155-0000-fdbaaadc15f3" val testContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" + //path to a blob file def generateMockBlobPath: BlobPath = { val mockCromwellPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] mockCromwellPath.normalize() returns mockCromwellPath @@ -74,48 +77,53 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val mockBlobPath = mock[BlobPath] mockBlobPath.nioPath returns mockNioPath mockBlobPath.toAbsolutePath returns mockCromwellPath - mockBlobPath.md5 returns "MOCK_MD5" + mockBlobPath.md5 returns "BLOB_MD5" val mockTokenGenerator: WSMBlobSasTokenGenerator = mock[WSMBlobSasTokenGenerator] val mockFsm: BlobFileSystemManager = mock[BlobFileSystemManager] - mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath) returns Try(s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken") + val expectedTokenDuration: Duration = Duration.of(24, ChronoUnit.HOURS) + mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath, Some(expectedTokenDuration)) returns Try(s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken?sasExpirationDuration=${expectedTokenDuration.getSeconds.toInt}") mockFsm.blobTokenGenerator returns mockTokenGenerator mockBlobPath.getFilesystemManager returns mockFsm mockBlobPath } - def mockPathGetter(pathString: String): Try[cromwell.core.path.Path] = { - val mockBlobPath = generateMockBlobPath - val mockCromwellPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] - val foundBlobPath: Success[BlobPath] = Success(mockBlobPath) - val foundNonBlobPath: Success[cromwell.core.path.Path] = Success(mockCromwellPath) - if (pathString.equals(blobInput_0.url.get) || pathString.equals(blobInput_1.url.get)) return foundBlobPath - foundNonBlobPath + //Path to a file that isn't a blob file + def generateMockDefaultPath: cromwell.core.path.Path = { + val mockDefaultPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] + mockDefaultPath.pathAsString returns someNotBlobUrl + mockDefaultPath.md5 returns "DEFAULT_MD5" + mockDefaultPath + } + def pathGetter(pathString: String): Try[cromwell.core.path.Path] = { + if(pathString.contains(someBlobUrl)) Try(generateMockBlobPath) else Try(generateMockDefaultPath) } - def mockBlobConverter(pathToConvert: Try[cromwell.core.path.Path]): Try[BlobPath] = { - //using a stubbed md5 rather than matching on type because type matching of mocked types at runtime causes problems - if (pathToConvert.get.md5.equals("MOCK_MD5")) pathToConvert.asInstanceOf[Try[BlobPath]] else Failure(new Exception("failed")) + def blobConverter(pathToConvert: Try[cromwell.core.path.Path]): Try[BlobPath] = { + // Cromwell matches on (sub)type at runtime to determine if BlobPath or not. + // That doesn't work with mocks, so we use stubbed md5 for testing. + if(pathToConvert.get.md5.equals("BLOB_MD5")) Try(generateMockBlobPath) else Failure(new Exception("failed")) } it should "not return sas endpoint when no blob paths are provided" in { val mockLogger: JobLogger = mock[JobLogger] val emptyInputs: List[Input] = List() val bloblessInputs: List[Input] = List(notBlobInput_1, notBlobInput_2) - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(emptyInputs, mockPathGetter, mockLogger, mockBlobConverter).isFailure shouldBe true - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(bloblessInputs, mockPathGetter, mockLogger, mockBlobConverter).isFailure shouldBe true + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(emptyInputs, pathGetter, mockLogger, blobConverter).isFailure shouldBe true + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(bloblessInputs, pathGetter, mockLogger, blobConverter).isFailure shouldBe true } it should "return a sas endpoint based on inputs when blob paths are provided" in { val mockLogger: JobLogger = mock[JobLogger] - val expected = s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken" + val expectedTokenLifetimeSeconds = 24 * 60 * 60 //assert that cromwell asks for 24h token duration. + val expected = s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken?sasExpirationDuration=${expectedTokenLifetimeSeconds}" val blobInput: List[Input] = List(blobInput_0) val blobInputs: List[Input] = List(blobInput_0, blobInput_1) val mixedInputs: List[Input] = List(notBlobInput_1, blobInput_0, blobInput_1) - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInput, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected - TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, mockPathGetter, mockLogger, mockBlobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInput, pathGetter, mockLogger, blobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(blobInputs, pathGetter, mockLogger, blobConverter).get shouldEqual expected + TesAsyncBackendJobExecutionActor.determineWSMSasEndpointFromInputs(mixedInputs, pathGetter, mockLogger, blobConverter).get shouldEqual expected } it should "contain expected strings in the bash script" in { From b5e3c37587279f2914f806be6dfbdeb13d0b397c Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 7 Nov 2023 15:40:22 -0500 Subject: [PATCH 29/34] mock --- .../StandardAsyncExecutionActor.scala | 4 ++- ...TesAsyncBackendJobExecutionActorSpec.scala | 34 ++++++++----------- 2 files changed, 18 insertions(+), 20 deletions(-) diff --git a/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala b/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala index 12cbf7d6cef..59eb7f08269 100644 --- a/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala +++ b/backend/src/main/scala/cromwell/backend/standard/StandardAsyncExecutionActor.scala @@ -426,10 +426,12 @@ trait StandardAsyncExecutionActor |find . -type d -exec sh -c '[ -z "$$(ls -A '"'"'{}'"'"')" ] && touch '"'"'{}'"'"'/.file' \\; |)""".stripMargin) + val errorOrPreamble: ErrorOr[String] = scriptPreamble + // The `tee` trickery below is to be able to redirect to known filenames for CWL while also streaming // stdout and stderr for PAPI to periodically upload to cloud storage. // https://stackoverflow.com/questions/692000/how-do-i-write-stderr-to-a-file-while-using-tee-with-a-pipe - (errorOrDirectoryOutputs, errorOrGlobFiles, scriptPreamble).mapN((directoryOutputs, globFiles, preamble) => + (errorOrDirectoryOutputs, errorOrGlobFiles, errorOrPreamble).mapN((directoryOutputs, globFiles, preamble) => s"""|#!$jobShell |DOCKER_OUTPUT_DIR_LINK |cd ${cwd.pathAsString} diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index f7de7f318a4..eafd059f817 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -2,12 +2,10 @@ package cromwell.backend.impl.tes import common.mock.MockSugar import cromwell.core.logging.JobLogger -import cromwell.core.path import cromwell.filesystems.blob.{BlobFileSystemManager, BlobPath, WSMBlobSasTokenGenerator} +import org.mockito.ArgumentMatchers.any import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers - -import java.nio.file.Path import java.time.Duration import java.time.temporal.ChronoUnit import scala.util.{Failure, Try} @@ -65,30 +63,28 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit val testWorkspaceId = "e58ed763-928c-4155-0000-fdbaaadc15f3" val testContainerResourceId = "e58ed763-928c-4155-1111-fdbaaadc15f3" + def generateMockWsmTokenGenerator: WSMBlobSasTokenGenerator = { + val mockTokenGenerator = mock[WSMBlobSasTokenGenerator] + val expectedTokenDuration: Duration = Duration.of(24, ChronoUnit.HOURS) + mockTokenGenerator.getWSMSasFetchEndpoint(any[BlobPath], any[Option[Duration]]) returns Try(s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken?sasExpirationDuration=${expectedTokenDuration.getSeconds.toInt}") + mockTokenGenerator + } + def generateMockFsm: BlobFileSystemManager = { + val mockFsm: BlobFileSystemManager = mock[BlobFileSystemManager] + val mockGenerator: WSMBlobSasTokenGenerator = generateMockWsmTokenGenerator + mockFsm.blobTokenGenerator returns mockGenerator + mockFsm + } //path to a blob file def generateMockBlobPath: BlobPath = { - val mockCromwellPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] - mockCromwellPath.normalize() returns mockCromwellPath - val mockNioPath: path.NioPath = mock[path.NioPath] - val mockJavaPath: Path = mock[java.nio.file.Path] - mockNioPath.toAbsolutePath returns mockJavaPath - mockNioPath.normalize() returns mockNioPath - val mockBlobPath = mock[BlobPath] - mockBlobPath.nioPath returns mockNioPath - mockBlobPath.toAbsolutePath returns mockCromwellPath mockBlobPath.md5 returns "BLOB_MD5" - - val mockTokenGenerator: WSMBlobSasTokenGenerator = mock[WSMBlobSasTokenGenerator] - val mockFsm: BlobFileSystemManager = mock[BlobFileSystemManager] - val expectedTokenDuration: Duration = Duration.of(24, ChronoUnit.HOURS) - mockTokenGenerator.getWSMSasFetchEndpoint(mockBlobPath, Some(expectedTokenDuration)) returns Try(s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken?sasExpirationDuration=${expectedTokenDuration.getSeconds.toInt}") - mockFsm.blobTokenGenerator returns mockTokenGenerator + val mockFsm = generateMockFsm mockBlobPath.getFilesystemManager returns mockFsm - mockBlobPath } + //Path to a file that isn't a blob file def generateMockDefaultPath: cromwell.core.path.Path = { val mockDefaultPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] From 7f9044e831a6435596b8baed01984026a3ca1ef2 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 7 Nov 2023 16:20:59 -0500 Subject: [PATCH 30/34] mock --- .../impl/tes/TesAsyncBackendJobExecutionActorSpec.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index eafd059f817..8f2f3a17945 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -2,10 +2,12 @@ package cromwell.backend.impl.tes import common.mock.MockSugar import cromwell.core.logging.JobLogger +import cromwell.core.path.NioPath import cromwell.filesystems.blob.{BlobFileSystemManager, BlobPath, WSMBlobSasTokenGenerator} import org.mockito.ArgumentMatchers.any import org.scalatest.flatspec.AnyFlatSpec import org.scalatest.matchers.should.Matchers + import java.time.Duration import java.time.temporal.ChronoUnit import scala.util.{Failure, Try} @@ -79,12 +81,15 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit def generateMockBlobPath: BlobPath = { val mockBlobPath = mock[BlobPath] mockBlobPath.md5 returns "BLOB_MD5" + val mockFsm = generateMockFsm mockBlobPath.getFilesystemManager returns mockFsm + + val mockNioPath: NioPath = mock[NioPath] + mockBlobPath.nioPath returns mockNioPath mockBlobPath } - //Path to a file that isn't a blob file def generateMockDefaultPath: cromwell.core.path.Path = { val mockDefaultPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] From e227bb1b4079a8b5c7ede48bde3110ff696e34a6 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Tue, 7 Nov 2023 16:47:04 -0500 Subject: [PATCH 31/34] ACTUALLY FIXED IT --- .../tes/TesAsyncBackendJobExecutionActorSpec.scala | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala index 8f2f3a17945..a28fce3d445 100644 --- a/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala +++ b/supportedBackends/tes/src/test/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActorSpec.scala @@ -80,7 +80,7 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit //path to a blob file def generateMockBlobPath: BlobPath = { val mockBlobPath = mock[BlobPath] - mockBlobPath.md5 returns "BLOB_MD5" + mockBlobPath.pathAsString returns someBlobUrl val mockFsm = generateMockFsm mockBlobPath.getFilesystemManager returns mockFsm @@ -94,17 +94,17 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit def generateMockDefaultPath: cromwell.core.path.Path = { val mockDefaultPath: cromwell.core.path.Path = mock[cromwell.core.path.Path] mockDefaultPath.pathAsString returns someNotBlobUrl - mockDefaultPath.md5 returns "DEFAULT_MD5" mockDefaultPath } def pathGetter(pathString: String): Try[cromwell.core.path.Path] = { - if(pathString.contains(someBlobUrl)) Try(generateMockBlobPath) else Try(generateMockDefaultPath) + val mockBlob: BlobPath = generateMockBlobPath + val mockDefault: cromwell.core.path.Path = generateMockDefaultPath + if(pathString.contains(someBlobUrl)) Try(mockBlob) else Try(mockDefault) } def blobConverter(pathToConvert: Try[cromwell.core.path.Path]): Try[BlobPath] = { - // Cromwell matches on (sub)type at runtime to determine if BlobPath or not. - // That doesn't work with mocks, so we use stubbed md5 for testing. - if(pathToConvert.get.md5.equals("BLOB_MD5")) Try(generateMockBlobPath) else Failure(new Exception("failed")) + val mockBlob: BlobPath = generateMockBlobPath + if(pathToConvert.get.pathAsString.contains(someBlobUrl)) Try(mockBlob) else Failure(new Exception("failed")) } it should "not return sas endpoint when no blob paths are provided" in { @@ -128,7 +128,6 @@ class TesAsyncBackendJobExecutionActorSpec extends AnyFlatSpec with Matchers wit } it should "contain expected strings in the bash script" in { - val mockEnvironmentVariableNameFromWom = "mock_env_var_for_storing_sas_token" val expectedEndpoint = s"$testWsmEndpoint/api/workspaces/v1/$testWorkspaceId/resources/controlled/azure/storageContainer/$testContainerResourceId/getSasToken" From 52939977bbf423248f173d2ef3269f3b6c55df27 Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 13 Nov 2023 09:12:19 -0500 Subject: [PATCH 32/34] Update supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala Co-authored-by: Adam Nichols --- .../backend/impl/tes/TesAsyncBackendJobExecutionActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala index 1a935c1a64a..100ed6137e9 100644 --- a/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala +++ b/supportedBackends/tes/src/main/scala/cromwell/backend/impl/tes/TesAsyncBackendJobExecutionActor.scala @@ -155,7 +155,7 @@ object TesAsyncBackendJobExecutionActor { // Log if not all input files live in the same container. if (blobFiles.map(_.container).distinct.size > 1) { - logger.info(s"While parsing blob inputs, found more than one container. Can only generate an environment sas token for a single blob container at once.") + logger.info(s"While parsing blob inputs, found more than one container. Generating SAS token based on first file in the list.") } // We use the first blob file in the list to determine the correct blob container. From 60f999bf83901c54045c255c5907a2791c1f97ce Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 13 Nov 2023 09:12:25 -0500 Subject: [PATCH 33/34] Update filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala Co-authored-by: Adam Nichols --- .../scala/cromwell/filesystems/blob/BlobFileSystemManager.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index 79be0b0c0eb..ff02991a84f 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -188,7 +188,7 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi (for { wsmAuth <- wsmAuthToken wsmAzureResourceClient = wsmClientProvider.getControlledAzureResourceApi(wsmAuth) - resourceId <- getContainerResourceId(workspaceId, container, Some(wsmAuth)) + resourceId <- getContainerResourceId(workspaceId, container, Option(wsmAuth)) sasToken <- wsmAzureResourceClient.createAzureStorageContainerSasToken(workspaceId, resourceId) } yield sasToken).recoverWith { // If the storage account was still not found in WSM, this may be a public filesystem From c519f58f18dcbe3e43697112817c75381a573f4c Mon Sep 17 00:00:00 2001 From: Tom Wiseman Date: Mon, 13 Nov 2023 11:24:05 -0500 Subject: [PATCH 34/34] expiry --- .../cromwell/filesystems/blob/BlobFileSystemManager.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala index ff02991a84f..8f03dbe7e33 100644 --- a/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala +++ b/filesystems/blob/src/main/scala/cromwell/filesystems/blob/BlobFileSystemManager.scala @@ -244,15 +244,15 @@ class WSMBlobSasTokenGenerator(wsmClientProvider: WorkspaceManagerApiClientProvi * this function will make two REST requests. Otherwise, the relevant data is already cached locally. */ def getWSMSasFetchEndpoint(blobPath: BlobPath, tokenDuration: Option[Duration] = None): Try[String] = { - val lifetimeSeconds: Int = tokenDuration.map(d => d.toSeconds.intValue).getOrElse(8*60*60) val wsmEndpoint = wsmClientProvider.getBaseWorkspaceManagerUrl + val lifetimeQueryParameters: String = tokenDuration.map(d => s"?sasExpirationDuration=${d.toSeconds.intValue}").getOrElse("") val terraInfo: Try[WSMTerraCoordinates] = for { workspaceId <- parseTerraWorkspaceIdFromPath(blobPath) containerResourceId <- getContainerResourceId(workspaceId, blobPath.container, None) coordinates = WSMTerraCoordinates(wsmEndpoint, workspaceId, containerResourceId) } yield coordinates terraInfo.map{terraCoordinates => - s"${terraCoordinates.wsmEndpoint}/api/workspaces/v1/${terraCoordinates.workspaceId.toString}/resources/controlled/azure/storageContainer/${terraCoordinates.containerResourceId.toString}/getSasToken?sasExpirationDuration=$lifetimeSeconds" + s"${terraCoordinates.wsmEndpoint}/api/workspaces/v1/${terraCoordinates.workspaceId.toString}/resources/controlled/azure/storageContainer/${terraCoordinates.containerResourceId.toString}/getSasToken${lifetimeQueryParameters}" } } }