Skip to content

Commit

Permalink
Better import failure messages and importLocalFilesystem resolver set (
Browse files Browse the repository at this point in the history
  • Loading branch information
cjllanwarne authored and aednichols committed Aug 23, 2018
1 parent af6747d commit 139b568
Show file tree
Hide file tree
Showing 16 changed files with 110 additions and 54 deletions.
13 changes: 13 additions & 0 deletions centaur/src/main/resources/standardTestCases/import_passwd.test
Original file line number Diff line number Diff line change
@@ -0,0 +1,13 @@
name: import_passwd
testFormat: workflowfailure

files {
workflow: import_passwd/import_passwd.wdl
}

metadata {
status: Failed
"failures.0.message": "Workflow input processing failed"
"failures.0.causedBy.0.message": "Failed to import workflow /etc/passwd.:\nBad import /etc/passwd: Failed to resolve '/etc/passwd' using resolver: 'http importer (no 'relative-to' origin)' (reason 1 of 1): Relative path"
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,5 @@
import "/etc/passwd" as whoa_there_buddy

workflow foo {
call whoa_there_buddy.dont_do_that
}
Original file line number Diff line number Diff line change
Expand Up @@ -8,5 +8,5 @@ files {
metadata {
status: Failed
"failures.0.message": "Workflow input processing failed"
"failures.0.causedBy.0.message": "Failed to resolve 'https://raw.githubusercontent.com/broadinstitute/cromwell/path_to_url_doesnt_exist' using resolver: 'relative to directory / (without escaping None)' (reason 1 of 1): Import file not found: https://raw.githubusercontent.com/broadinstitute/cromwell/path_to_url_doesnt_exist"
"failures.0.causedBy.0.message": "Failed to resolve 'https://raw.githubusercontent.com/broadinstitute/cromwell/path_to_url_doesnt_exist' using resolver: 'http importer (no 'relative-to' origin)' (reason 1 of 1): Failed to download https://raw.githubusercontent.com/broadinstitute/cromwell/path_to_url_doesnt_exist (reason 1 of 1): 400: Invalid request\n"
}
2 changes: 1 addition & 1 deletion common/src/main/scala/common/validation/Validation.scala
Original file line number Diff line number Diff line change
Expand Up @@ -78,7 +78,7 @@ object Validation {
implicit class ValidationChecked[A](val e: Checked[A]) extends AnyVal {
def unsafe(context: String): A = e.valueOr(errors => throw AggregatedMessageException(context, errors.toList))

def contextualizeErrors(s: String): Checked[A] = e.leftMap { errors =>
def contextualizeErrors(s: => String): Checked[A] = e.leftMap { errors =>
val total = errors.size
errors.zipWithIndex map { case (err, i) => s"Failed to $s (reason ${i + 1} of $total): $err" }
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -40,9 +40,7 @@ class SingleWorkflowRunnerActor(source: WorkflowSourceFilesCollection,
gracefulShutdown: Boolean,
abortJobsOnTerminate: Boolean
)(implicit materializer: ActorMaterializer)
extends CromwellRootActor(gracefulShutdown, abortJobsOnTerminate) with LoggingFSM[RunnerState, SwraData] {

override val serverMode = false
extends CromwellRootActor(gracefulShutdown, abortJobsOnTerminate, false) with LoggingFSM[RunnerState, SwraData] {

import SingleWorkflowRunnerActor._
private val backoff = SimpleExponentialBackoff(1 second, 1 minute, 1.2)
Expand Down
Original file line number Diff line number Diff line change
@@ -1,7 +1,5 @@
package cromwell.engine.workflow.lifecycle.materialization

import java.nio.file.Paths

import akka.actor.{ActorRef, FSM, LoggingFSM, Props, Status}
import akka.pattern.pipe
import cats.Monad
Expand Down Expand Up @@ -29,7 +27,7 @@ import cromwell.core.callcaching._
import cromwell.core.io.AsyncIo
import cromwell.core.labels.{Label, Labels}
import cromwell.core.logging.WorkflowLogging
import cromwell.core.path.{DefaultPath, PathBuilder}
import cromwell.core.path.PathBuilder
import cromwell.engine._
import cromwell.engine.backend.CromwellBackends
import cromwell.engine.language.CromwellLanguages
Expand Down Expand Up @@ -136,7 +134,6 @@ class MaterializeWorkflowDescriptorActor(serviceRegistryActor: ActorRef,
ioActorProxy: ActorRef) extends LoggingFSM[MaterializeWorkflowDescriptorActorState, Unit] with LazyLogging with WorkflowLogging {

import MaterializeWorkflowDescriptorActor._

val tag = self.path.name

val iOExecutionContext = context.system.dispatchers.lookup("akka.dispatchers.io-dispatcher")
Expand Down Expand Up @@ -232,12 +229,10 @@ class MaterializeWorkflowDescriptorActor(serviceRegistryActor: ActorRef,
resolvers: List[ImportResolver]): Checked[(WorkflowSource, List[ImportResolver])] = {
(workflowSource, workflowUrl) match {
case (Some(source), None) => (source, resolvers).validNelCheck
case (None, Some(url)) =>{
case (None, Some(url)) =>
val compoundImportResolver: CheckedAtoB[ImportResolutionRequest, ResolvedImportBundle] = CheckedAtoB.firstSuccess(resolvers.map(_.resolver), s"resolve workflowUrl '$url'")
val wfSourceAndResolvers: Checked[ResolvedImportBundle] = compoundImportResolver.run(ImportResolutionRequest(url, resolvers))

wfSourceAndResolvers map {v => (v.source, v.newResolvers) }
}
wfSourceAndResolvers map { v => (v.source, v.newResolvers) }
case (Some(_), Some(_)) => "Both workflow source and url can't be supplied".invalidNelCheck
case (None, None) => "Either workflow source or url has to be supplied".invalidNelCheck
}
Expand Down Expand Up @@ -268,7 +263,9 @@ class MaterializeWorkflowDescriptorActor(serviceRegistryActor: ActorRef,
errorOrParse(factory).flatMap(_.validateNamespace(sourceFiles, workflowSource, workflowOptions, importLocalFilesystem, workflowIdForLogging, engineIoFunctions, importResolvers))
}

val localFilesystemResolvers = if (importLocalFilesystem) List(DirectoryResolver(DefaultPath(Paths.get("/")))) else List.empty
val localFilesystemResolvers =
if (importLocalFilesystem) DirectoryResolver.localFilesystemResolvers(None)
else List.empty

val zippedResolverCheck: Parse[Option[ImportResolver]] = fromEither[IO](sourceFiles.importsZipFileOption match {
case None => None.validNelCheck
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,7 +52,7 @@ import scala.util.{Failure, Success, Try}
* READ THIS: If you add a "system-level" actor here, make sure to consider what should be its
* position in the shutdown process and modify CromwellShutdown accordingly.
*/
abstract class CromwellRootActor(gracefulShutdown: Boolean, abortJobsOnTerminate: Boolean)(implicit materializer: ActorMaterializer) extends Actor with ActorLogging with GracefulShutdownHelper {
abstract class CromwellRootActor(gracefulShutdown: Boolean, abortJobsOnTerminate: Boolean, val serverMode: Boolean)(implicit materializer: ActorMaterializer) extends Actor with ActorLogging with GracefulShutdownHelper {
import CromwellRootActor._

// Make sure the filesystems are initialized at startup
Expand All @@ -65,8 +65,6 @@ abstract class CromwellRootActor(gracefulShutdown: Boolean, abortJobsOnTerminate
private val workflowHeartbeatConfig = WorkflowHeartbeatConfig(config)
logger.info("Workflow heartbeat configuration:\n{}", workflowHeartbeatConfig)

val serverMode: Boolean

lazy val systemConfig = config.getConfig("system")
lazy val serviceRegistryActor: ActorRef = context.actorOf(ServiceRegistryActor.props(config), "ServiceRegistryActor")
lazy val numberOfWorkflowLogCopyWorkers = systemConfig.as[Option[Int]]("number-of-workflow-log-copy-workers").getOrElse(DefaultNumberOfWorkflowLogCopyWorkers)
Expand Down
4 changes: 1 addition & 3 deletions engine/src/main/scala/cromwell/server/CromwellServer.scala
Original file line number Diff line number Diff line change
Expand Up @@ -24,16 +24,14 @@ object CromwellServer {
}

class CromwellServerActor(cromwellSystem: CromwellSystem, gracefulShutdown: Boolean, abortJobsOnTerminate: Boolean)(override implicit val materializer: ActorMaterializer)
extends CromwellRootActor(gracefulShutdown, abortJobsOnTerminate)
extends CromwellRootActor(gracefulShutdown, abortJobsOnTerminate, serverMode = true)
with CromwellApiService with CromwellInstrumentationActor
with SwaggerService
with ActorLogging {
implicit val actorSystem = context.system
override implicit val ec = context.dispatcher
override def actorRefFactory: ActorContext = context

override val serverMode = true

val webserviceConf = cromwellSystem.conf.getConfig("webservice")
val interface = webserviceConf.getString("interface")
val port = webserviceConf.getInt("port")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,7 @@ import common.transforms.CheckedAtoB
import common.validation.ErrorOr._
import common.validation.Checked._
import common.validation.Validation._
import cromwell.core.path.Path
import cromwell.core.path.{DefaultPathBuilder, Path}
import java.nio.file.{Path => NioPath}

import wom.core.WorkflowSource
Expand All @@ -37,22 +37,42 @@ object ImportResolver {
}
}


object DirectoryResolver {
def apply(directory: Path, allowEscapingDirectory: Boolean): DirectoryResolver = {
def apply(directory: Path, allowEscapingDirectory: Boolean, customName: Option[String]): DirectoryResolver = {
val dontEscapeFrom = if (allowEscapingDirectory) None else Option(directory.toJava.getCanonicalPath)
DirectoryResolver(directory, dontEscapeFrom)
DirectoryResolver(directory, dontEscapeFrom, customName)
}

def localFilesystemResolvers(baseWdl: Option[Path]) = List(
DirectoryResolver(
DefaultPathBuilder.build(Paths.get(".")),
allowEscapingDirectory = true,
customName = None
),
DirectoryResolver(
DefaultPathBuilder.build(Paths.get("/")),
allowEscapingDirectory = false,
customName = Some("entire local filesystem (relative to '/')")
)
) ++ baseWdl.toList.map { rt =>
DirectoryResolver(
DefaultPathBuilder.build(Paths.get(rt.toAbsolutePath.toFile.getParent)),
allowEscapingDirectory = true,
customName = None
)
}
}

case class DirectoryResolver(directory: Path, dontEscapeFrom: Option[String] = None) extends ImportResolver {
lazy val absolutePathToDirectory = directory.toJava.getCanonicalPath
case class DirectoryResolver(directory: Path,
dontEscapeFrom: Option[String] = None,
customName: Option[String]) extends ImportResolver {
lazy val absolutePathToDirectory: String = directory.toJava.getCanonicalPath

override def innerResolver(path: String, currentResolvers: List[ImportResolver]): Checked[ResolvedImportBundle] = {

def updatedResolverSet(oldRootDirectory: Path, newRootDirectory: Path, current: List[ImportResolver]): List[ImportResolver] = {
current map {
case d if d == this => DirectoryResolver(newRootDirectory, dontEscapeFrom)
case d if d == this => DirectoryResolver(newRootDirectory, dontEscapeFrom, customName)
case other => other
}
}
Expand All @@ -63,7 +83,7 @@ object ImportResolver {
if (file.exists) {
File(absolutePathToFile).contentAsString.validNel
} else {
s"Import file not found: $path".invalidNel
s"File not found: $path".invalidNel
}
}
}
Expand Down Expand Up @@ -92,20 +112,35 @@ object ImportResolver {
_ <- checkLocation(abs, path)
} yield abs

override def name: String = s"relative to directory $absolutePathToDirectory (without escaping $dontEscapeFrom)"
override lazy val name: String = (customName, dontEscapeFrom) match {
case (Some(custom), _) => custom
case (None, Some(dontEscapePath)) =>
val dontEscapeFromDirname = Paths.get(dontEscapePath).getFileName.toString
val shortPathToDirectory = absolutePathToDirectory.stripPrefix(dontEscapePath)

val relativePathToDontEscapeFrom = s"[...]/$dontEscapeFromDirname"
val relativePathToDirectory = s"$relativePathToDontEscapeFrom$shortPathToDirectory"

s"relative to directory $relativePathToDirectory (without escaping $relativePathToDontEscapeFrom)"
case (None, None) =>
val shortPathToDirectory = Paths.get(absolutePathToDirectory).toFile.getCanonicalFile.toPath.getFileName.toString
s"relative to directory [...]/$shortPathToDirectory (escaping allowed)"
}
}

def zippedImportResolver(zippedImports: Array[Byte]): ErrorOr[ImportResolver] = {
LanguageFactoryUtil.validateImportsDirectory(zippedImports) map { dir =>
DirectoryResolver(dir, Option(dir.toJava.getCanonicalPath))
DirectoryResolver(dir, Option(dir.toJava.getCanonicalPath), None)
}
}

case class HttpResolver(relativeTo: Option[String] = None, headers: Map[String, String] = Map.empty) extends ImportResolver {
import HttpResolver._

override def name: String = {
s"http importer${relativeTo map { r => s" (relative to $r)" } getOrElse ""}"
override def name: String = relativeTo match {
case Some(relativeToPath) => s"http importer (relative to $relativeToPath)"
case None => "http importer (no 'relative-to' origin)"

}

def newResolverList(newRoot: String): List[ImportResolver] = {
Expand All @@ -122,7 +157,7 @@ object ImportResolver {
else canonicalize(s"${relativeToValue.stripSuffix("/")}/$str").validNelCheck
case None =>
if (str.startsWith("http")) canonicalize(str).validNelCheck
else s"Cannot import '$str' relative to nothing".invalidNelCheck
else "Relative path".invalidNelCheck
}

override def innerResolver(str: String, currentResolvers: List[ImportResolver]): Checked[ResolvedImportBundle] = {
Expand All @@ -147,10 +182,13 @@ object ImportResolver {
}

object HttpResolver {

import common.util.IntrospectableLazy
import common.util.IntrospectableLazy._

val sttpBackend: IntrospectableLazy[SttpBackend[IO, Nothing]] = lazily { AsyncHttpClientCatsBackend[IO]() }
val sttpBackend: IntrospectableLazy[SttpBackend[IO, Nothing]] = lazily {
AsyncHttpClientCatsBackend[IO]()
}

def closeBackendIfNecessary() = if (sttpBackend.exists) sttpBackend.close()

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -65,7 +65,7 @@ class ImportResolverSpec extends FlatSpec with Matchers {

behavior of "directory resolver from root"

val rootDirectoryResolver = DirectoryResolver(DefaultPath(Paths.get("/")))
val rootDirectoryResolver = DirectoryResolver(DefaultPath(Paths.get("/")), customName = None)

it should "resolve a random path" in {
val pathToLookup = rootDirectoryResolver.resolveAndMakeAbsolute("/path/to/file.wdl")
Expand All @@ -74,7 +74,7 @@ class ImportResolverSpec extends FlatSpec with Matchers {

behavior of "unprotected relative directory resolver"

val relativeDirectoryResolver = DirectoryResolver(DefaultPath(Paths.get("/path/to/imports/")))
val relativeDirectoryResolver = DirectoryResolver(DefaultPath(Paths.get("/path/to/imports/")), customName = None)

it should "resolve an absolute path" in {
val pathToLookup = relativeDirectoryResolver.resolveAndMakeAbsolute("/path/to/file.wdl")
Expand All @@ -88,7 +88,7 @@ class ImportResolverSpec extends FlatSpec with Matchers {

behavior of "protected relative directory resolver"

val protectedRelativeDirectoryResolver = DirectoryResolver(DefaultPath(Paths.get("/path/to/imports/")), Some("/path/to/imports/"))
val protectedRelativeDirectoryResolver = DirectoryResolver(DefaultPath(Paths.get("/path/to/imports/")), Some("/path/to/imports/"), customName = None)

it should "resolve a good relative path" in {
val pathToLookup = protectedRelativeDirectoryResolver.resolveAndMakeAbsolute("path/to/file.wdl")
Expand Down
3 changes: 1 addition & 2 deletions server/src/test/scala/cromwell/CromwellTestKitSpec.scala
Original file line number Diff line number Diff line change
Expand Up @@ -232,8 +232,7 @@ object CromwellTestKitSpec {
ServiceRegistryActorSystem.actorOf(ServiceRegistryActor.props(ConfigFactory.load()), "ServiceRegistryActor")
}

class TestCromwellRootActor(config: Config)(implicit materializer: ActorMaterializer) extends CromwellRootActor(false, false) {
override val serverMode = true
class TestCromwellRootActor(config: Config)(implicit materializer: ActorMaterializer) extends CromwellRootActor(false, false, serverMode = true) {
override lazy val serviceRegistryActor = ServiceRegistryActorInstance
override lazy val workflowStore = new InMemoryWorkflowStore
def submitWorkflow(sources: WorkflowSourceFilesWithoutImports): WorkflowId = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -275,7 +275,10 @@ class MaterializeWorkflowDescriptorActorSpec extends CromwellTestKitWordSpec wit
within(Timeout) {
expectMsgPF() {
case MaterializeWorkflowDescriptorFailureResponse(reason) =>
reason.getMessage should startWith("Workflow input processing failed:\nFailed to resolve 'https://raw.githubusercontent.com/broadinstitute/cromwell/develop/my_workflow' using resolver: 'http importer' (reason 1 of 1): Failed to download https://raw.githubusercontent.com/broadinstitute/cromwell/develop/my_workflow (reason 1 of 1): 404: Not Found")
reason.getMessage should startWith(
"""Workflow input processing failed:
|Failed to resolve 'https://raw.githubusercontent.com/broadinstitute/cromwell/develop/my_workflow' using resolver: 'http importer (no 'relative-to' origin)' (reason 1 of 1): Failed to download https://raw.githubusercontent.com/broadinstitute/cromwell/develop/my_workflow (reason 1 of 1): 404: Not Found"""
.stripMargin)
case _: MaterializeWorkflowDescriptorSuccessResponse => fail("This materialization should not have succeeded!")
case unknown =>
fail(s"Unexpected materialization response: $unknown")
Expand Down
17 changes: 3 additions & 14 deletions womtool/src/main/scala/womtool/input/WomGraphMaker.scala
Original file line number Diff line number Diff line change
Expand Up @@ -5,7 +5,7 @@ import java.nio.file.{Files, Paths}
import com.typesafe.config.ConfigFactory
import common.Checked
import common.validation.Validation._
import cromwell.core.path.{DefaultPathBuilder, Path}
import cromwell.core.path.Path
import cromwell.languages.LanguageFactory
import cromwell.languages.util.ImportResolver._
import languages.cwl.CwlV1_0LanguageFactory
Expand All @@ -24,19 +24,8 @@ object WomGraphMaker {
def getBundle(mainFile: Path): Checked[WomBundle] = getBundleAndFactory(mainFile).map(_._1)

private def getBundleAndFactory(mainFile: Path): Checked[(WomBundle, LanguageFactory)] = {
// Resolves for:
// - Where we run from
// - Where the file is
lazy val importResolvers: List[ImportResolver] = List(
DirectoryResolver(
DefaultPathBuilder.build(Paths.get(".")),
allowEscapingDirectory = false),
DirectoryResolver(
DefaultPathBuilder.build(Paths.get(mainFile.toAbsolutePath.toFile.getParent)),
allowEscapingDirectory = true
),
HttpResolver()
)
lazy val importResolvers: List[ImportResolver] =
DirectoryResolver.localFilesystemResolvers(Some(mainFile)) :+ HttpResolver(relativeTo = None)

readFile(mainFile.toAbsolutePath.pathAsString) flatMap { mainFileContents =>
val languageFactory =
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
version 1.0

import "sub_dir/bad_import.wdl" as found_but_invalid

workflow foo {
call found_but_invalid.call_a_bad_thing
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,4 @@
Failed to import 'sub_dir/bad_import.wdl' (reason 1 of 4): Failed to import 'not/a/file.wdl' (reason 1 of 4): Failed to resolve 'not/a/file.wdl' using resolver: 'relative to directory [...]/cromwell (escaping allowed)' (reason 1 of 1): File not found: not/a/file.wdl
Failed to import 'sub_dir/bad_import.wdl' (reason 2 of 4): Failed to import 'not/a/file.wdl' (reason 2 of 4): Failed to resolve 'not/a/file.wdl' using resolver: 'entire local filesystem (relative to '/')' (reason 1 of 1): File not found: not/a/file.wdl
Failed to import 'sub_dir/bad_import.wdl' (reason 3 of 4): Failed to import 'not/a/file.wdl' (reason 3 of 4): Failed to resolve 'not/a/file.wdl' using resolver: 'relative to directory [...]/sub_dir (escaping allowed)' (reason 1 of 1): File not found: not/a/file.wdl
Failed to import 'sub_dir/bad_import.wdl' (reason 4 of 4): Failed to import 'not/a/file.wdl' (reason 4 of 4): Failed to resolve 'not/a/file.wdl' using resolver: 'http importer (no 'relative-to' origin)' (reason 1 of 1): Relative path
Original file line number Diff line number Diff line change
@@ -0,0 +1,7 @@
version 1.0

import "not/a/file.wdl" as not_a_file

workflow call_a_bad_thing {
call not_a_file.not_a_task
}

0 comments on commit 139b568

Please sign in to comment.