From 1b2faaf249fbe07464be2f421c70b11f12d9e93f Mon Sep 17 00:00:00 2001 From: Khalid Shakir Date: Sat, 1 Jun 2019 12:30:42 -0400 Subject: [PATCH 01/28] Google Auth Updates - Replaced to-be-deprecated Credential (no 's') with Adapter around Credentials - Removed dupe credentials adapting from PipelinesApiFactoryInterface - Move service specific scopes (KMS, Genomics) out of GoogleAuthMode - Changed credential creation methods to take scala collections --- .../src/main/scala/centaur/test/Test.scala | 6 +- .../gcp/auth/GoogleAuthMode.scala | 98 ++++--------------- .../drs/DrsPathBuilderFactory.scala | 9 +- .../filesystems/gcs/GcsPathBuilder.scala | 3 +- .../cromwell/filesystems/gcs/GoogleUtil.scala | 11 ++- .../WorkbenchHealthMonitorServiceActor.scala | 7 +- .../PipelinesApiInitializationActor.scala | 51 ++++++++-- .../api/PipelinesApiFactoryInterface.scala | 15 +-- .../pipelines/v2alpha1/GenomicsFactory.scala | 26 +++-- 9 files changed, 98 insertions(+), 128 deletions(-) diff --git a/centaur/src/main/scala/centaur/test/Test.scala b/centaur/src/main/scala/centaur/test/Test.scala index 89a528c1e97..e384539ae46 100644 --- a/centaur/src/main/scala/centaur/test/Test.scala +++ b/centaur/src/main/scala/centaur/test/Test.scala @@ -12,7 +12,8 @@ import centaur.test.metadata.WorkflowFlatMetadata import centaur.test.metadata.WorkflowFlatMetadata._ import centaur.test.submit.SubmitHttpResponse import centaur.test.workflow.Workflow -import com.google.api.services.genomics.Genomics +import com.google.api.services.genomics.{Genomics, GenomicsScopes} +import com.google.api.services.storage.StorageScopes import com.google.auth.Credentials import com.google.auth.http.HttpCredentialsAdapter import com.google.auth.oauth2.ServiceAccountCredentials @@ -86,9 +87,10 @@ object Operations { lazy val googleConf: Config = CentaurConfig.conf.getConfig("google") lazy val authName: String = googleConf.getString("auth") lazy val genomicsEndpointUrl: String = googleConf.getString("genomics.endpoint-url") + lazy val genomicsAndStorageScopes = List(StorageScopes.CLOUD_PLATFORM_READ_ONLY, GenomicsScopes.GENOMICS) lazy val credentials: Credentials = configuration.auth(authName) .unsafe - .pipelinesApiCredentials(GoogleAuthMode.NoOptionLookup) + .credentials(genomicsAndStorageScopes) lazy val credentialsProjectOption: Option[String] = { Option(credentials) collect { case serviceAccountCredentials: ServiceAccountCredentials => serviceAccountCredentials.getProjectId diff --git a/cloudSupport/src/main/scala/cromwell/cloudsupport/gcp/auth/GoogleAuthMode.scala b/cloudSupport/src/main/scala/cromwell/cloudsupport/gcp/auth/GoogleAuthMode.scala index 018dff2f364..74f56dbfc68 100644 --- a/cloudSupport/src/main/scala/cromwell/cloudsupport/gcp/auth/GoogleAuthMode.scala +++ b/cloudSupport/src/main/scala/cromwell/cloudsupport/gcp/auth/GoogleAuthMode.scala @@ -4,15 +4,9 @@ import java.io.{ByteArrayInputStream, FileNotFoundException, InputStream} import java.net.HttpURLConnection._ import better.files.File -import com.google.api.client.googleapis.auth.oauth2.GoogleCredential import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport -import com.google.api.client.googleapis.testing.auth.oauth2.MockGoogleCredential import com.google.api.client.http.HttpResponseException import com.google.api.client.json.jackson2.JacksonFactory -import com.google.api.services.cloudkms.v1.CloudKMS -import com.google.api.services.compute.ComputeScopes -import com.google.api.services.genomics.v2alpha1.GenomicsScopes -import com.google.api.services.storage.StorageScopes import com.google.auth.Credentials import com.google.auth.http.HttpTransportFactory import com.google.auth.oauth2.{GoogleCredentials, OAuth2Credentials, ServiceAccountCredentials, UserCredentials} @@ -55,13 +49,6 @@ object GoogleAuthMode { val DockerCredentialsEncryptionKeyNameKey = "docker_credentials_key_name" val DockerCredentialsTokenKey = "docker_credentials_token" - private val PipelinesApiScopes = List( - StorageScopes.DEVSTORAGE_FULL_CONTROL, - StorageScopes.DEVSTORAGE_READ_WRITE, - GenomicsScopes.GENOMICS, - ComputeScopes.COMPUTE - ) - def checkReadable(file: File) = { if (!file.isReadable) throw new FileNotFoundException(s"File $file does not exist or is not readable") } @@ -85,24 +72,6 @@ object GoogleAuthMode { } } - def encryptKms(keyName: String, credential: GoogleCredential, plainText: String) = { - import com.google.api.services.cloudkms.v1.CloudKMSScopes - - // Depending on the environment that provides the default credentials (e.g. Compute Engine, App - // Engine), the credentials may require us to specify the scopes we need explicitly. - // Check for this case, and inject the scope if required. - val scopedCredential = if (credential.createScopedRequired) credential.createScoped(CloudKMSScopes.all) else credential - - val kms = new CloudKMS.Builder(httpTransport, jsonFactory, scopedCredential) - .setApplicationName("cromwell") - .build() - - import com.google.api.services.cloudkms.v1.model.EncryptRequest - val request = new EncryptRequest().encodePlaintext(plainText.toCharArray.map(_.toByte)) - val response = kms.projects.locations.keyRings.cryptoKeys.encrypt(keyName, request).execute - response.getCiphertext - } - /** Used for both checking that the credential is valid and creating a fresh credential. */ private def refreshCredentials(credentials: Credentials): Unit = { credentials.refresh() @@ -115,49 +84,33 @@ sealed trait GoogleAuthMode { def name: String - // Create a Credential object from the google.api.client.auth library (https://github.com/google/google-api-java-client) - def credentials(options: OptionLookup, scopes: java.util.Collection[String]): OAuth2Credentials - /** - * Create a credential object suitable for use with Pipelines API. - * - * @param options A lookup for external credential information. - * @return Credentials with scopes compatible with the Genomics API compute and storage. + * Creates OAuth credentials with the specified scopes. */ - def pipelinesApiCredentials(options: OptionLookup): OAuth2Credentials = { - credentials(options, PipelinesApiScopes.asJavaCollection) - } + def credentials(options: OptionLookup, scopes: Iterable[String]): OAuth2Credentials /** * Alias for credentials(GoogleAuthMode.NoOptionLookup, scopes). * Only valid for credentials that are NOT externally provided, such as ApplicationDefault. */ def credentials(scopes: Iterable[String]): OAuth2Credentials = { - credentials(GoogleAuthMode.NoOptionLookup, scopes.asJavaCollection) - } - - /** - * Alias for credentials(GoogleAuthMode.NoOptionLookup, scopes). - * Only valid for credentials that are NOT externally provided, such as ApplicationDefault. - */ - def credentials(scopes: java.util.Collection[String]): OAuth2Credentials = { credentials(GoogleAuthMode.NoOptionLookup, scopes) } /** - * Alias for credentials(GoogleAuthMode.NoOptionLookup, Set.empty). + * Alias for credentials(GoogleAuthMode.NoOptionLookup, Nil). * Only valid for credentials that are NOT externally provided and do not need scopes, such as ApplicationDefault. */ private[auth] def credentials(): OAuth2Credentials = { - credentials(GoogleAuthMode.NoOptionLookup, java.util.Collections.emptySet[String]) + credentials(GoogleAuthMode.NoOptionLookup, Nil) } /** - * Alias for credentials(options, Set.empty). + * Alias for credentials(options, Nil). * Only valid for credentials that are NOT externally provided and do not need scopes, such as ApplicationDefault. */ private[auth] def credentials(options: OptionLookup): OAuth2Credentials = { - credentials(options, java.util.Collections.emptySet[String]) + credentials(options, Nil) } def requiresAuthFile: Boolean = false @@ -176,20 +129,14 @@ sealed trait GoogleAuthMode { case Success(_) => credential } } - - def apiClientGoogleCredential(options: OptionLookup): Option[GoogleCredential] = None } case object MockAuthMode extends GoogleAuthMode { override val name = "no_auth" - override def credentials(unusedOptions: OptionLookup, unusedScopes: java.util.Collection[String]): NoCredentials = { + override def credentials(unusedOptions: OptionLookup, unusedScopes: Iterable[String]): NoCredentials = { NoCredentials.getInstance } - - override def apiClientGoogleCredential(options: OptionLookup): Option[MockGoogleCredential] = { - Option(new MockGoogleCredential.Builder().build()) - } } object ServiceAccountMode { @@ -204,20 +151,12 @@ object ServiceAccountMode { } -trait HasApiClientGoogleCredentialStream { self: GoogleAuthMode => - protected def credentialStream(options: OptionLookup): InputStream - - override def apiClientGoogleCredential(options: OptionLookup): Option[GoogleCredential] = Option(GoogleCredential.fromStream(credentialStream(options))) -} - final case class ServiceAccountMode(override val name: String, fileFormat: CredentialFileFormat) - extends GoogleAuthMode with HasApiClientGoogleCredentialStream { + extends GoogleAuthMode { private val credentialsFile = File(fileFormat.file) checkReadable(credentialsFile) - override protected def credentialStream(options: OptionLookup): InputStream = credentialsFile.newInputStream - private lazy val serviceAccountCredentials: ServiceAccountCredentials = { fileFormat match { case PemFileFormat(accountId, _) => @@ -228,25 +167,24 @@ final case class ServiceAccountMode(override val name: String, } override def credentials(unusedOptions: OptionLookup, - scopes: java.util.Collection[String]): GoogleCredentials = { - val scopedCredentials = serviceAccountCredentials.createScoped(scopes) + scopes: Iterable[String]): GoogleCredentials = { + val scopedCredentials = serviceAccountCredentials.createScoped(scopes.asJavaCollection) validateCredentials(scopedCredentials) } } -final case class UserServiceAccountMode(override val name: String) - extends GoogleAuthMode with HasApiClientGoogleCredentialStream { +final case class UserServiceAccountMode(override val name: String) extends GoogleAuthMode { private def extractServiceAccount(options: OptionLookup): String = { extract(options, UserServiceAccountKey) } - override protected def credentialStream(options: OptionLookup): InputStream = { + private def credentialStream(options: OptionLookup): InputStream = { new ByteArrayInputStream(extractServiceAccount(options).getBytes("UTF-8")) } - override def credentials(options: OptionLookup, scopes: java.util.Collection[String]): GoogleCredentials = { + override def credentials(options: OptionLookup, scopes: Iterable[String]): GoogleCredentials = { val newCredentials = ServiceAccountCredentials.fromStream(credentialStream(options)) - val scopedCredentials: GoogleCredentials = newCredentials.createScoped(scopes) + val scopedCredentials: GoogleCredentials = newCredentials.createScoped(scopes.asJavaCollection) validateCredentials(scopedCredentials) } } @@ -267,7 +205,7 @@ final case class UserMode(override val name: String, validateCredentials(UserCredentials.fromStream(secretsStream)) } - override def credentials(unusedOptions: OptionLookup, unusedScopes: java.util.Collection[String]): OAuth2Credentials = { + override def credentials(unusedOptions: OptionLookup, unusedScopes: Iterable[String]): OAuth2Credentials = { userCredentials } } @@ -278,11 +216,9 @@ object ApplicationDefaultMode { final case class ApplicationDefaultMode(name: String) extends GoogleAuthMode { override def credentials(unusedOptions: OptionLookup, - unusedScopes: java.util.Collection[String]): GoogleCredentials = { + unusedScopes: Iterable[String]): GoogleCredentials = { ApplicationDefaultMode.applicationDefaultCredentials } - - override def apiClientGoogleCredential(unused: OptionLookup): Option[GoogleCredential] = Option(GoogleCredential.getApplicationDefault(httpTransport, jsonFactory)) } final case class RefreshTokenMode(name: String, @@ -297,7 +233,7 @@ final case class RefreshTokenMode(name: String, extract(options, RefreshTokenOptionKey) } - override def credentials(options: OptionLookup, unusedScopes: java.util.Collection[String]): UserCredentials = { + override def credentials(options: OptionLookup, unusedScopes: Iterable[String]): UserCredentials = { val refreshToken = extractRefreshToken(options) val newCredentials: UserCredentials = UserCredentials .newBuilder() diff --git a/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilderFactory.scala b/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilderFactory.scala index 4bb95ac6216..3f4689ca588 100644 --- a/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilderFactory.scala +++ b/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilderFactory.scala @@ -16,10 +16,8 @@ import cromwell.core.WorkflowOptions import cromwell.core.path.{PathBuilder, PathBuilderFactory} import org.apache.http.impl.client.HttpClientBuilder -import scala.collection.JavaConverters._ import scala.concurrent.{ExecutionContext, Future} - /** * Cromwell Wrapper around DrsFileSystems to load the configuration. * This class is used as the global configuration class in the drs filesystem @@ -54,10 +52,9 @@ class DrsPathBuilderFactory(globalConfig: Config, instanceConfig: Config, single private def inputReadChannel(url: String, urlScheme: String, serviceAccount: String): IO[ReadableByteChannel] = { urlScheme match { - case GcsScheme => { + case GcsScheme => val Array(bucket, fileToBeLocalized) = url.replace(s"$GcsScheme://", "").split("/", 2) gcsInputStream(GcsFilePath(bucket, fileToBeLocalized), serviceAccount) - } case otherScheme => IO.raiseError(new UnsupportedOperationException(s"DRS currently doesn't support reading files for $otherScheme.")) } } @@ -82,8 +79,8 @@ class DrsPathBuilderFactory(globalConfig: Config, instanceConfig: Config, single // Profile and Email scopes are requirements for interacting with Martha v2 Oauth2Scopes.USERINFO_EMAIL, Oauth2Scopes.USERINFO_PROFILE - ).asJavaCollection - val authCredentials = googleAuthMode.credentials((key: String) => options.get(key).get, marthaScopes) + ) + val authCredentials = googleAuthMode.credentials(options.get(_).get, marthaScopes) Future.successful(DrsPathBuilder(new DrsCloudNioFileSystemProvider(singletonConfig.config, authCredentials, httpClientBuilder, drsReadInterpreter))) } diff --git a/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GcsPathBuilder.scala b/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GcsPathBuilder.scala index 258e0991c32..93aad37be57 100644 --- a/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GcsPathBuilder.scala +++ b/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GcsPathBuilder.scala @@ -8,6 +8,7 @@ import akka.http.scaladsl.model.ContentTypes import better.files.File.OpenOptions import cats.effect.IO import com.google.api.gax.retrying.RetrySettings +import com.google.api.services.storage.StorageScopes import com.google.auth.Credentials import com.google.cloud.storage.Storage.BlobTargetOption import com.google.cloud.storage.contrib.nio.{CloudStorageConfiguration, CloudStorageFileSystem, CloudStoragePath} @@ -100,7 +101,7 @@ object GcsPathBuilder { cloudStorageConfiguration: CloudStorageConfiguration, options: WorkflowOptions, defaultProject: Option[String])(implicit as: ActorSystem, ec: ExecutionContext): Future[GcsPathBuilder] = { - authMode.retryPipelinesApiCredentials(options) map { credentials => + authMode.retryCredentials(options, List(StorageScopes.DEVSTORAGE_FULL_CONTROL)) map { credentials => fromCredentials(credentials, applicationName, retrySettings, diff --git a/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GoogleUtil.scala b/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GoogleUtil.scala index 8f3e5abf9eb..9fa2a200060 100644 --- a/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GoogleUtil.scala +++ b/filesystems/gcs/src/main/scala/cromwell/filesystems/gcs/GoogleUtil.scala @@ -24,13 +24,16 @@ object GoogleUtil { implicit class EnhancedGoogleAuthMode(val googleAuthMode: GoogleAuthMode) extends AnyVal { /** - * Retries getting the pipelines API credentials three times. + * Retries getting the credentials three times. + * + * There is nothing GCS specific about this method. This package just happens to be the lowest level with access + * to core's version of Retry + cloudSupport's implementation of GoogleAuthMode. */ - def retryPipelinesApiCredentials(options: WorkflowOptions) - (implicit as: ActorSystem, ec: ExecutionContext): Future[Credentials] = { + def retryCredentials(options: WorkflowOptions, scopes: Iterable[String]) + (implicit actorSystem: ActorSystem, executionContext: ExecutionContext): Future[Credentials] = { def credential(): Credentials = { try { - googleAuthMode.pipelinesApiCredentials((key: String) => options.get(key).get) + googleAuthMode.credentials(options.get(_).get, scopes) } catch { case exception: OptionLookupException => throw new IllegalArgumentException(s"Missing parameters in workflow options: ${exception.key}", exception) diff --git a/services/src/main/scala/cromwell/services/healthmonitor/impl/workbench/WorkbenchHealthMonitorServiceActor.scala b/services/src/main/scala/cromwell/services/healthmonitor/impl/workbench/WorkbenchHealthMonitorServiceActor.scala index c974ec32261..694cdfd8722 100644 --- a/services/src/main/scala/cromwell/services/healthmonitor/impl/workbench/WorkbenchHealthMonitorServiceActor.scala +++ b/services/src/main/scala/cromwell/services/healthmonitor/impl/workbench/WorkbenchHealthMonitorServiceActor.scala @@ -8,6 +8,8 @@ import cats.instances.future._ import cats.syntax.functor._ import com.google.api.client.http.{HttpRequest, HttpRequestInitializer} import com.google.api.gax.retrying.RetrySettings +import com.google.api.services.genomics.v2alpha1.GenomicsScopes +import com.google.api.services.storage.StorageScopes import com.google.auth.Credentials import com.google.auth.http.HttpCredentialsAdapter import com.typesafe.config.Config @@ -57,7 +59,8 @@ abstract class WorkbenchHealthMonitorServiceActor(val serviceConfig: Config, glo private def checkGcs(): Future[SubsystemStatus] = { // For any expected production usage of this check, the GCS bucket should be public read */ val gcsBucketToCheck = serviceConfig.as[String]("gcs-bucket-to-check") - val storage = Future(googleAuth.pipelinesApiCredentials(GoogleAuthMode.NoOptionLookup)) map { credentials => + val storageScopes = List(StorageScopes.DEVSTORAGE_READ_ONLY) + val storage = Future(googleAuth.credentials(storageScopes)) map { credentials => GcsStorage.gcsStorage(googleConfig.applicationName, credentials, RetrySettings.newBuilder().build()) } storage map { _.buckets.get(gcsBucketToCheck).execute() } as OkStatus @@ -71,7 +74,7 @@ abstract class WorkbenchHealthMonitorServiceActor(val serviceConfig: Config, glo val papiProjectId = papiConfig.as[String]("project") val check = for { - credentials <- Future(googleAuth.pipelinesApiCredentials(GoogleAuthMode.NoOptionLookup)) + credentials <- Future(googleAuth.credentials(List(GenomicsScopes.GENOMICS))) genomicsChecker = if (papiProviderConfig.as[String]("actor-factory").contains("v2alpha1")) GenomicsCheckerV2(googleConfig.applicationName, googleAuth, endpointUrl, credentials, papiProjectId) else diff --git a/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiInitializationActor.scala b/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiInitializationActor.scala index 3a01f744e0c..7e56eed5701 100644 --- a/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiInitializationActor.scala +++ b/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/PipelinesApiInitializationActor.scala @@ -1,10 +1,18 @@ package cromwell.backend.google.pipelines.common import akka.actor.ActorRef +import com.google.api.services.cloudkms.v1.model.EncryptRequest +import com.google.api.services.cloudkms.v1.{CloudKMS, CloudKMSScopes} +import com.google.api.services.genomics.GenomicsScopes +import com.google.api.services.storage.StorageScopes import com.google.auth.Credentials +import com.google.auth.http.HttpCredentialsAdapter +import com.google.auth.oauth2.OAuth2Credentials +import cromwell.backend.google.pipelines.common.PipelinesApiInitializationActor._ import cromwell.backend.google.pipelines.common.api.PipelinesApiRequestFactory import cromwell.backend.standard.{StandardInitializationActor, StandardInitializationActorParams, StandardValidatedRuntimeAttributesBuilder} import cromwell.backend.{BackendConfigurationDescriptor, BackendInitializationData, BackendWorkflowDescriptor} +import cromwell.cloudsupport.gcp.auth.GoogleAuthMode.{httpTransport, jsonFactory} import cromwell.cloudsupport.gcp.auth.{GoogleAuthMode, UserServiceAccountMode} import cromwell.core.Dispatcher import cromwell.core.io.AsyncIoActorClient @@ -40,12 +48,31 @@ class PipelinesApiInitializationActor(pipelinesParams: PipelinesApiInitializatio PipelinesApiRuntimeAttributes.runtimeAttributesBuilder(pipelinesConfiguration) // Credentials object for the GCS API - private lazy val gcsCredentials: Future[Credentials] = - pipelinesConfiguration.papiAttributes.auths.gcs.retryPipelinesApiCredentials(workflowOptions) + private lazy val gcsCredentials: Future[Credentials] = pipelinesConfiguration.papiAttributes.auths.gcs + .retryCredentials(workflowOptions, List(StorageScopes.DEVSTORAGE_FULL_CONTROL)) // Credentials object for the Genomics API - private lazy val genomicsCredentials: Future[Credentials] = - pipelinesConfiguration.papiAttributes.auths.genomics.retryPipelinesApiCredentials(workflowOptions) + private lazy val genomicsCredentials: Future[Credentials] = pipelinesConfiguration.papiAttributes.auths.genomics + .retryCredentials(workflowOptions, List( + GenomicsScopes.GENOMICS, + /* + Genomics Pipelines API v1alpha2 requires the COMPUTE scope. Does not seem to be required for v2alpha1. + */ + GenomicsScopes.COMPUTE, + /* + Used to write so-called "auth" files. The `gcsAuthFilePath` could probably be refactored such that *this* created + `genomicsCredentials` doesn't actually need DEVSTORAGE_FULL_CONTROL, but it's also not clear how the magic + Genomics Pipelines API parameter "__extra_config_gcs_path" works nor where it's documented. + + See also: + - cromwell.backend.google.pipelines.common.PipelinesApiWorkflowPaths#gcsAuthFilePath + - https://github.com/broadinstitute/cromwell/pull/2435 + - cromwell.backend.google.pipelines.common.PipelinesApiConfiguration#needAuthFileUpload + - cromwell.cloudsupport.gcp.auth.GoogleAuthMode#requiresAuthFile + - cromwell.backend.google.pipelines.common.PipelinesApiAsyncBackendJobExecutionActor#gcsAuthParameter + */ + StorageScopes.DEVSTORAGE_FULL_CONTROL, + )) // Genomics object to access the Genomics API private lazy val genomics: Future[PipelinesApiRequestFactory] = { @@ -97,8 +124,9 @@ class PipelinesApiInitializationActor(pipelinesParams: PipelinesApiInitializatio plain <- unencrypted auth <- effectiveAuth key <- privateDockerEncryptionKeyName - cred <- auth.apiClientGoogleCredential(k => workflowOptions.get(k).get) - } yield GoogleAuthMode.encryptKms(key, cred, plain) + credentials = auth.credentials(workflowOptions.get(_).get, List(CloudKMSScopes.CLOUD_PLATFORM)) + encrypted = encryptKms(key, credentials, plain) + } yield encrypted } override lazy val workflowPaths: Future[PipelinesApiWorkflowPaths] = for { @@ -140,4 +168,15 @@ class PipelinesApiInitializationActor(pipelinesParams: PipelinesApiInitializatio object PipelinesApiInitializationActor { // For metadata publishing purposes default to using the name of a standard stream as the stream's filename. def defaultStandardStreamNameToFileNameMetadataMapper(pipelinesApiJobPaths: PipelinesApiJobPaths, streamName: String): String = streamName + + def encryptKms(keyName: String, credentials: OAuth2Credentials, plainText: String): String = { + val httpCredentialsAdapter = new HttpCredentialsAdapter(credentials) + val kms = new CloudKMS.Builder(httpTransport, jsonFactory, httpCredentialsAdapter) + .setApplicationName("cromwell") + .build() + + val request = new EncryptRequest().encodePlaintext(plainText.toCharArray.map(_.toByte)) + val response = kms.projects.locations.keyRings.cryptoKeys.encrypt(keyName, request).execute + response.getCiphertext + } } diff --git a/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/api/PipelinesApiFactoryInterface.scala b/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/api/PipelinesApiFactoryInterface.scala index 1b5ccc49300..fe5908fae9b 100644 --- a/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/api/PipelinesApiFactoryInterface.scala +++ b/supportedBackends/google/pipelines/common/src/main/scala/cromwell/backend/google/pipelines/common/api/PipelinesApiFactoryInterface.scala @@ -1,9 +1,8 @@ package cromwell.backend.google.pipelines.common.api -import com.google.api.client.http.{HttpRequest, HttpRequestInitializer} +import com.google.api.client.http.HttpRequestInitializer import com.google.auth.Credentials import com.google.auth.http.HttpCredentialsAdapter -import mouse.all._ /** * The interface provides a single method to build a PipelinesApiRequestFactory @@ -13,17 +12,11 @@ import mouse.all._ * Pipelines API for this workflow. */ abstract class PipelinesApiFactoryInterface { - private def httpRequestInitializerFromCredentials(credentials: Credentials) = { - val delegate = new HttpCredentialsAdapter(credentials) - new HttpRequestInitializer() { - def initialize(httpRequest: HttpRequest) = { - delegate.initialize(httpRequest) - } - } + final def fromCredentials(credentials: Credentials): PipelinesApiRequestFactory = { + val httpCredentialsAdapter = new HttpCredentialsAdapter(credentials) + build(httpCredentialsAdapter) } - final def fromCredentials(credentials: Credentials): PipelinesApiRequestFactory = build(credentials |> httpRequestInitializerFromCredentials) - protected def build(httpRequestInitializer: HttpRequestInitializer): PipelinesApiRequestFactory def usesEncryptedDocker: Boolean diff --git a/supportedBackends/google/pipelines/v2alpha1/src/main/scala/cromwell/backend/google/pipelines/v2alpha1/GenomicsFactory.scala b/supportedBackends/google/pipelines/v2alpha1/src/main/scala/cromwell/backend/google/pipelines/v2alpha1/GenomicsFactory.scala index f0d852e3ccc..6e970aa03aa 100644 --- a/supportedBackends/google/pipelines/v2alpha1/src/main/scala/cromwell/backend/google/pipelines/v2alpha1/GenomicsFactory.scala +++ b/supportedBackends/google/pipelines/v2alpha1/src/main/scala/cromwell/backend/google/pipelines/v2alpha1/GenomicsFactory.scala @@ -10,6 +10,7 @@ import com.google.api.services.genomics.v2alpha1.model._ import com.google.api.services.genomics.v2alpha1.{Genomics, GenomicsScopes} import com.google.api.services.oauth2.Oauth2Scopes import com.google.api.services.storage.StorageScopes +import com.google.auth.http.HttpCredentialsAdapter import cromwell.backend.google.pipelines.common.PipelinesApiConfigurationAttributes.{LocalizationConfiguration, VirtualPrivateCloudConfiguration} import cromwell.backend.google.pipelines.common.api.PipelinesApiRequestFactory.CreatePipelineParameters import cromwell.backend.google.pipelines.common.api.{PipelinesApiFactoryInterface, PipelinesApiRequestFactory} @@ -39,7 +40,7 @@ case class GenomicsFactory(applicationName: String, authMode: GoogleAuthMode, en override def build(initializer: HttpRequestInitializer): PipelinesApiRequestFactory = new PipelinesApiRequestFactory { implicit lazy val googleProjectMetadataLabelDecoder: Decoder[ProjectLabels] = deriveDecoder - val ResourceManagerAuthScopes = List(GenomicsScopes.CLOUD_PLATFORM).asJava + val ResourceManagerAuthScopes = List(GenomicsScopes.CLOUD_PLATFORM) val VirtualPrivateCloudNetworkPath = "projects/%s/global/networks/%s/" val genomics = new Genomics.Builder( @@ -61,23 +62,18 @@ case class GenomicsFactory(applicationName: String, authMode: GoogleAuthMode, en override def runRequest(createPipelineParameters: CreatePipelineParameters, jobLogger: JobLogger): HttpRequest = { def projectMetadataRequest(vpcConfig: VirtualPrivateCloudConfiguration): IO[HttpRequest] = { - val workflowOptions = createPipelineParameters.jobDescriptor.workflowDescriptor.workflowOptions + IO { + val workflowOptions = createPipelineParameters.jobDescriptor.workflowDescriptor.workflowOptions + val credentials = vpcConfig.auth.credentials(workflowOptions.get(_).get, ResourceManagerAuthScopes) - val googleCredentialOption = vpcConfig.auth.apiClientGoogleCredential((key: String) => workflowOptions.get(key).get) + val httpCredentialsAdapter = new HttpCredentialsAdapter(credentials) + val cloudResourceManagerBuilder = new CloudResourceManager + .Builder(GoogleAuthMode.httpTransport, GoogleAuthMode.jsonFactory, httpCredentialsAdapter) + .build() - googleCredentialOption match { - case None => IO.raiseError(new RuntimeException(s"Programmer Error: Unable to find Google Credential for auth `${vpcConfig.auth.name}`.")) - case Some(googleCredential) => IO { - val auth = googleCredential.createScoped(ResourceManagerAuthScopes) + val project = cloudResourceManagerBuilder.projects().get(createPipelineParameters.projectId) - val cloudResourceManagerBuilder = new CloudResourceManager - .Builder(GoogleAuthMode.httpTransport, GoogleAuthMode.jsonFactory, auth) - .build() - - val project = cloudResourceManagerBuilder.projects().get(createPipelineParameters.projectId) - - project.buildHttpRequest() - } + project.buildHttpRequest() } } From d446f833b0646c563938103ba08d17c8331d67d6 Mon Sep 17 00:00:00 2001 From: Khalid Shakir Date: Sun, 2 Jun 2019 20:36:43 -0400 Subject: [PATCH 02/28] Increase bootDiskSizeGb for gpu_cuda_image --- .../resources/standardTestCases/gpu_on_papi/gpu_cuda_image.wdl | 1 + 1 file changed, 1 insertion(+) diff --git a/centaur/src/main/resources/standardTestCases/gpu_on_papi/gpu_cuda_image.wdl b/centaur/src/main/resources/standardTestCases/gpu_on_papi/gpu_cuda_image.wdl index 5ce0a5c33a8..f7c784cb574 100644 --- a/centaur/src/main/resources/standardTestCases/gpu_on_papi/gpu_cuda_image.wdl +++ b/centaur/src/main/resources/standardTestCases/gpu_on_papi/gpu_cuda_image.wdl @@ -33,6 +33,7 @@ task get_machine_info { runtime { docker: "nvidia/cuda:9.0-cudnn7-devel-ubuntu16.04" + bootDiskSizeGb: 20 gpuType: "nvidia-tesla-k80" gpuCount: 1 nvidiaDriverVersion: driver_version From 5c6b6f1a58e134796a0516b91fc6b768f5cff4cb Mon Sep 17 00:00:00 2001 From: root Date: Mon, 3 Jun 2019 21:57:58 +0000 Subject: [PATCH 03/28] Update cromwell version from 42 to 43 --- project/Version.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/Version.scala b/project/Version.scala index 542cb313ae7..e6392717a64 100644 --- a/project/Version.scala +++ b/project/Version.scala @@ -5,7 +5,7 @@ import sbt._ object Version { // Upcoming release, or current if we're on a master / hotfix branch - val cromwellVersion = "42" + val cromwellVersion = "43" /** * Returns true if this project should be considered a snapshot. From b8b99c3b5cbc9e780e4ee9612b8aba6c3c5747fc Mon Sep 17 00:00:00 2001 From: Ruben Vorderman Date: Wed, 5 Jun 2019 22:58:09 +0200 Subject: [PATCH 04/28] Retry jobs that have been killed by a cluster backend. [BA-5669] (#5003) --- .../sfs/config/ConfigAsyncJobExecutionActor.scala | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/supportedBackends/sfs/src/main/scala/cromwell/backend/impl/sfs/config/ConfigAsyncJobExecutionActor.scala b/supportedBackends/sfs/src/main/scala/cromwell/backend/impl/sfs/config/ConfigAsyncJobExecutionActor.scala index 97e382b4c44..71656cc10dd 100644 --- a/supportedBackends/sfs/src/main/scala/cromwell/backend/impl/sfs/config/ConfigAsyncJobExecutionActor.scala +++ b/supportedBackends/sfs/src/main/scala/cromwell/backend/impl/sfs/config/ConfigAsyncJobExecutionActor.scala @@ -304,7 +304,16 @@ class DispatchedConfigAsyncJobExecutionActor(override val standardParams: Standa jobLogger.error(s"Return file not found after ${exitCodeTimeout.getOrElse(backupError)} seconds, assuming external kill") val returnCodeTemp = jobPaths.returnCode.plusExt("kill") - returnCodeTemp.write(s"$SIGTERM${System.lineSeparator}") + + // If SIGTERM, SIGKILL or SIGINT codes are used, cromwell will assume the job has been aborted by cromwell. + // And it will therefore NOT retry the job. Which makes perfect sense. Best not to change that in the + // StandardAsyncExecutionActor code, but give a user-defined return code here. + // http://tldp.org/LDP/abs/html/exitcodes.html gives information on exit codes and suggests restricting + // user-defined exit codes to the range 64-113. + // Since it is arbitrary which code is chosen from that range, and it has to relate with the unpleasant + // business of 'killing'. 79 was chosen. The year that video killed the radio star: https://youtu.be/W8r-tXRLazs + returnCodeTemp.appendLine("79") + try { returnCodeTemp.moveTo(jobPaths.returnCode) SharedFileSystemJobFailed From 0e1a32057fd90fe8d344baaa3fac7962f2fbce51 Mon Sep 17 00:00:00 2001 From: Kris Ganjam Date: Fri, 7 Jun 2019 08:41:02 -0700 Subject: [PATCH 05/28] Ensure workflowLogger always gets closed regardless of whether the log is temporary or not. (#5016) --- .../main/scala/cromwell/engine/workflow/WorkflowActor.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/engine/src/main/scala/cromwell/engine/workflow/WorkflowActor.scala b/engine/src/main/scala/cromwell/engine/workflow/WorkflowActor.scala index 0adbb164b6f..687781defe4 100644 --- a/engine/src/main/scala/cromwell/engine/workflow/WorkflowActor.scala +++ b/engine/src/main/scala/cromwell/engine/workflow/WorkflowActor.scala @@ -441,11 +441,10 @@ class WorkflowActor(workflowToStart: WorkflowToStart, workflowOptions.get(FinalWorkflowLogDir).toOption match { case Some(destinationDir) => pathBuilders.map(pb => workflowLogCopyRouter ! CopyWorkflowLogsActor.Copy(workflowId, PathFactory.buildPath(destinationDir, pb)))(ec) - case None if WorkflowLogger.isTemporary => workflowLogger.close(andDelete = true) match { + case None => workflowLogger.close(andDelete = WorkflowLogger.isTemporary) match { case Failure(f) => log.error(f, "Failed to delete workflow log") case _ => } - case _ => } } context stop self From f0ae471331922b7302bffeeb2b01effe264973eb Mon Sep 17 00:00:00 2001 From: Alejandro Dubrovsky Date: Sat, 8 Jun 2019 04:41:33 +1000 Subject: [PATCH 06/28] Fix submit-docker to point to the right script in example configuration (#5015) --- cromwell.example.backends/cromwell.examples.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cromwell.example.backends/cromwell.examples.conf b/cromwell.example.backends/cromwell.examples.conf index 11bbf533cb3..1dc77f0f1cf 100644 --- a/cromwell.example.backends/cromwell.examples.conf +++ b/cromwell.example.backends/cromwell.examples.conf @@ -397,7 +397,7 @@ backend { ${"--user " + docker_user} \ --entrypoint ${job_shell} \ -v ${cwd}:${docker_cwd} \ - ${docker} ${script} + ${docker} ${docker_script} """ # Root directory where Cromwell writes job results. This directory must be From fe38ba71aa25839c505939edd31ba454f76dfa0e Mon Sep 17 00:00:00 2001 From: Khalid Shakir Date: Fri, 7 Jun 2019 14:51:23 -0400 Subject: [PATCH 07/28] Added logging and cleanup to DockerComposeCromwellConfiguration --- .../scala/centaur/CromwellConfiguration.scala | 19 +++++++ .../DockerComposeCromwellConfiguration.scala | 53 ++++++++++--------- .../centaur/JarCromwellConfiguration.scala | 17 ++---- .../docker-compose-horicromtal.yml | 2 +- 4 files changed, 51 insertions(+), 40 deletions(-) diff --git a/centaur/src/main/scala/centaur/CromwellConfiguration.scala b/centaur/src/main/scala/centaur/CromwellConfiguration.scala index 1b79a6573b0..f86268da4c3 100644 --- a/centaur/src/main/scala/centaur/CromwellConfiguration.scala +++ b/centaur/src/main/scala/centaur/CromwellConfiguration.scala @@ -1,5 +1,8 @@ package centaur +import java.lang.ProcessBuilder.Redirect +import better.files.File + trait CromwellProcess { def logFile: String def displayString: String @@ -7,6 +10,22 @@ trait CromwellProcess { def stop(): Unit def isAlive: Boolean def cromwellConfiguration: CromwellConfiguration + + protected def runProcess(command: Array[String]): Process = { + val processBuilder = new java.lang.ProcessBuilder() + .command(command: _*) + .redirectOutput(Redirect.appendTo(File(logFile).toJava)) + .redirectErrorStream(true) + processBuilder.start() + } + + protected def waitProcess(process: Process, destroy: Boolean = false): Unit = { + process.getOutputStream.flush() + if (destroy) + process.destroy() + process.waitFor() + () + } } trait CromwellConfiguration { diff --git a/centaur/src/main/scala/centaur/DockerComposeCromwellConfiguration.scala b/centaur/src/main/scala/centaur/DockerComposeCromwellConfiguration.scala index 12ef2bba2f3..182391491bd 100644 --- a/centaur/src/main/scala/centaur/DockerComposeCromwellConfiguration.scala +++ b/centaur/src/main/scala/centaur/DockerComposeCromwellConfiguration.scala @@ -1,11 +1,8 @@ package centaur -import java.lang.ProcessBuilder.Redirect - -import better.files.File +import centaur.CromwellManager.ManagedCromwellPort import com.typesafe.config.Config - object DockerComposeCromwellConfiguration { def apply(conf: Config): CromwellConfiguration = { val dockerTag = conf.getString("docker-tag") @@ -20,37 +17,41 @@ object DockerComposeCromwellConfiguration { case class DockerComposeCromwellConfiguration(dockerTag: String, dockerComposeFile: String, conf: String, logFile: String) extends CromwellConfiguration { override def createProcess: CromwellProcess = { case class DockerComposeCromwellProcess(override val cromwellConfiguration: DockerComposeCromwellConfiguration) extends CromwellProcess { - private val startCommand = Array( - "/bin/bash", - "-c", - s"CROMWELL_TAG=$dockerTag CROMWELL_CONFIG=$conf docker-compose -f $dockerComposeFile up -d") + + private def composeCommand(command: String): Array[String] = { + Array( + "/bin/bash", + "-c", + s"MANAGED_CROMWELL_PORT=$ManagedCromwellPort " + + s"CROMWELL_TAG=$dockerTag " + + s"CROMWELL_CONFIG=$conf " + + s"docker-compose -f $dockerComposeFile $command") + } + + private val startCommand = composeCommand("up") + private val stopCommand = composeCommand("down -v") + private val rmCommand = composeCommand("rm -fsv") override def displayString: String = startCommand.mkString(" ") + private var process: Option[Process] = None + override def start(): Unit = { - val processBuilder = new java.lang.ProcessBuilder() - .command(startCommand: _*) - .redirectOutput(Redirect.appendTo(File(logFile).toJava)) - .redirectErrorStream(true) - processBuilder.start().waitFor() - () + process = Option(runProcess(startCommand)) } override def stop(): Unit = { - val command = Array( - "/bin/bash", - "-c", - s"docker-compose -f $dockerComposeFile down" - ) - val processBuilder = new java.lang.ProcessBuilder() - .command(command: _*) - .redirectOutput(Redirect.appendTo(File(logFile).toJava)) - .redirectErrorStream(true) - processBuilder.start().waitFor() - () + waitProcess(runProcess(stopCommand)) + waitProcess(runProcess(rmCommand)) + process foreach { + waitProcess(_, destroy = true) + } + process = None } - override def isAlive: Boolean = true + override def isAlive: Boolean = process.exists { + _.isAlive + } override def logFile: String = cromwellConfiguration.logFile } diff --git a/centaur/src/main/scala/centaur/JarCromwellConfiguration.scala b/centaur/src/main/scala/centaur/JarCromwellConfiguration.scala index caccb8ef46f..3069e555316 100644 --- a/centaur/src/main/scala/centaur/JarCromwellConfiguration.scala +++ b/centaur/src/main/scala/centaur/JarCromwellConfiguration.scala @@ -1,8 +1,5 @@ package centaur -import java.lang.ProcessBuilder.Redirect - -import better.files.File import centaur.CromwellManager.ManagedCromwellPort import com.typesafe.config.Config @@ -32,20 +29,14 @@ case class JarCromwellConfiguration(jar: String, conf: String, logFile: String) override def displayString: String = command.mkString(" ") override def start(): Unit = { - val processBuilder = new java.lang.ProcessBuilder() - .command(command: _*) - .redirectOutput(Redirect.appendTo(File(logFile).toJava)) - .redirectErrorStream(true) - process = Option(processBuilder.start()) + process = Option(runProcess(command)) } override def stop(): Unit = { - process foreach { p => - p.getOutputStream.flush() - p.destroy() - p.waitFor() + process foreach { + waitProcess(_, destroy = true) } - () + process = None } override def isAlive: Boolean = process.exists { _.isAlive } diff --git a/scripts/docker-compose-mysql/docker-compose-horicromtal.yml b/scripts/docker-compose-mysql/docker-compose-horicromtal.yml index b554316de74..5faf0bc5bf8 100644 --- a/scripts/docker-compose-mysql/docker-compose-horicromtal.yml +++ b/scripts/docker-compose-mysql/docker-compose-horicromtal.yml @@ -39,7 +39,7 @@ services: - ${CROMWELL_BUILD_ROOT_DIRECTORY}:${CROMWELL_BUILD_ROOT_DIRECTORY} command: ["server"] environment: - - JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} -Dwebservice.port=8008 -Dsystem.cromwell_id=frontend + - JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} -Dwebservice.port=${MANAGED_CROMWELL_PORT-8008} -Dsystem.cromwell_id=frontend - CROMWELL_BUILD_RESOURCES_DIRECTORY=${CROMWELL_BUILD_ROOT_DIRECTORY}/target/ci/resources - CROMWELL_BUILD_MYSQL_USERNAME=${CROMWELL_BUILD_MYSQL_USERNAME} healthcheck: From 7f8d72947f9d8a501bb6aeae8b24678776ec3912 Mon Sep 17 00:00:00 2001 From: Chris Llanwarne Date: Tue, 11 Jun 2019 17:50:17 -0400 Subject: [PATCH 08/28] Cromwell perf ad-hoc deploy, test, and destroy scripts BA-4980 (#5012) --- scripts/perf/clone_database/clone_database.sh | 21 +++++ .../perf/delete_database/delete_database.sh | 15 ++++ .../perf/delete_instance/delete_instance.sh | 15 ++++ .../perf/deploy_instance/deploy_instance.sh | 58 +++++++++++++ .../perf/deploy_instance/run_on_instance.sh | 51 +++++++++++ scripts/perf/helper.inc.sh | 84 +++++++++++++++++-- scripts/perf/startup_script.sh | 2 + 7 files changed, 241 insertions(+), 5 deletions(-) create mode 100755 scripts/perf/clone_database/clone_database.sh create mode 100755 scripts/perf/delete_database/delete_database.sh create mode 100755 scripts/perf/delete_instance/delete_instance.sh create mode 100755 scripts/perf/deploy_instance/deploy_instance.sh create mode 100644 scripts/perf/deploy_instance/run_on_instance.sh diff --git a/scripts/perf/clone_database/clone_database.sh b/scripts/perf/clone_database/clone_database.sh new file mode 100755 index 00000000000..99240996ff4 --- /dev/null +++ b/scripts/perf/clone_database/clone_database.sh @@ -0,0 +1,21 @@ +#!/bin/bash +set -euo pipefail + +source scripts/perf/helper.inc.sh + +mkdir -p mnt + +# Read the DB password from vault: +DB_PASS=$(read_path_from_vault_json "secret/dsp/cromwell/perf" '.data.db_pass') + +# Read the service account credentials from vault: +read_service_account_from_vault + +# Clone the CloudSQL DB +# Note: Cloning the same database in parallel doesn't work. +# By doing it here we can run the jenkins jobs sequentially and ensure the database is cloned once at a time as well +SQL_OPERATION=$(gcloud_run_as_service_account "perf_sql_create_gcloud_${BUILD_NUMBER}" "gcloud --project broad-dsde-cromwell-perf sql instances clone --async ${CLOUD_SQL_INSTANCE_TO_CLONE} ${CLOUD_SQL_INSTANCE_NEW_NAME} --format='value(name)'" ) + +gcloud_run_as_service_account "perf_sql_await_gcloud_${BUILD_NUMBER}" "gcloud beta sql operations wait --timeout=900 --project broad-dsde-cromwell-perf ${SQL_OPERATION}" + +gcloud_run_as_service_account "perf_sql_user_gcloud_${BUILD_NUMBER}" "gcloud --project broad-dsde-cromwell-perf sql users create cromwell --instance=${CLOUD_SQL_INSTANCE_NEW_NAME} --password=${DB_PASS}" diff --git a/scripts/perf/delete_database/delete_database.sh b/scripts/perf/delete_database/delete_database.sh new file mode 100755 index 00000000000..a393cac5eb2 --- /dev/null +++ b/scripts/perf/delete_database/delete_database.sh @@ -0,0 +1,15 @@ +#!/bin/bash +set -euo pipefail + +source scripts/perf/helper.inc.sh + +mkdir -p mnt + +# Read the service account credentials from vault: +read_service_account_from_vault + +gcloud_run_as_service_account "perf_sql_delete_gcloud_${BUILD_NUMBER}" " \ + gcloud \ + --verbosity info \ + --project broad-dsde-cromwell-perf \ + sql instances delete ${CLOUD_SQL_INSTANCE} -q" diff --git a/scripts/perf/delete_instance/delete_instance.sh b/scripts/perf/delete_instance/delete_instance.sh new file mode 100755 index 00000000000..29d40ce7fb6 --- /dev/null +++ b/scripts/perf/delete_instance/delete_instance.sh @@ -0,0 +1,15 @@ +#!/bin/bash +set -euo pipefail + +source scripts/perf/helper.inc.sh + +mkdir -p mnt + +# Read the service account credentials from vault: +read_service_account_from_vault + +gcloud_run_as_service_account "perf_delete_gcloud_instance_${BUILD_NUMBER}" " \ + gcloud \ + --verbosity info \ + --project broad-dsde-cromwell-perf \ + compute instances delete ${CROMWELL_INSTANCE_NAME} --zone=us-central1-c -q" diff --git a/scripts/perf/deploy_instance/deploy_instance.sh b/scripts/perf/deploy_instance/deploy_instance.sh new file mode 100755 index 00000000000..25016d3a08d --- /dev/null +++ b/scripts/perf/deploy_instance/deploy_instance.sh @@ -0,0 +1,58 @@ +#!/bin/bash +set -euo pipefail + +source scripts/perf/helper.inc.sh + +GCS_BUCKET=gs://cromwell-perf-test/ + +DB_PASS=$(read_path_from_vault_json "secret/dsp/cromwell/perf" '.data.db_pass') + +read_service_account_from_vault + +function join() { local IFS=","; echo "$*"; } + +metadata=( + "BUILD_NUMBER=$BUILD_NUMBER" + "BUILD_TAG=$BUILD_TAG" + "CLEAN_UP=false" + "CLOUD_SQL_INSTANCE=$CLOUD_SQL_INSTANCE" + "CROMWELL_DB_USER=cromwell" + "CROMWELL_DB_PASS=$DB_PASS" + "CROMWELL_DOCKER_IMAGE=$CROMWELL_DOCKER_IMAGE" + "CROMWELL_PROJECT=broad-dsde-cromwell-perf" + "CROMWELL_BUCKET=$GCS_BUCKET" + "CROMWELL_STATSD_HOST=10.128.0.4" + "CROMWELL_STATSD_PORT=8125" + "CROMWELL_PERF_SCRIPTS_BRANCH=${REPO_BRANCH}" +) + +cp scripts/perf/deploy_instance/run_on_instance.sh mnt/ + +gcloud_run_as_service_account "perf_deploy_instance_${BUILD_NUMBER}" \ + "gcloud \ + --verbosity info \ + --project broad-dsde-cromwell-perf \ + compute \ + instances \ + create ${CROMWELL_INSTANCE_NAME} \ + --zone us-central1-c \ + --source-instance-template $INSTANCE_TEMPLATE \ + --metadata-from-file startup-script=$DOCKER_ETC_PATH/run_on_instance.sh \ + --metadata \ + $(join ${metadata[@]})" | tee dockerResult.txt + + +typeset CROMWELL_UNDER_TEST=$(cat dockerResult.txt | tail -n1 | awk '{print $5}' ) + +if test -z "$CROMWELL_UNDER_TEST" +then + echo "\$CROMWELL_UNDER_TEST is empty" + exit 1 +else + echo "Determined that CROMWELL_UNDER_TEST=${CROMWELL_UNDER_TEST}" +fi + +custom_wait_for_cromwell + +mkdir -p output +echo "CROMWELL_UNDER_TEST=${CROMWELL_UNDER_TEST}" > output/cromwell.properties diff --git a/scripts/perf/deploy_instance/run_on_instance.sh b/scripts/perf/deploy_instance/run_on_instance.sh new file mode 100644 index 00000000000..fab0bd5b839 --- /dev/null +++ b/scripts/perf/deploy_instance/run_on_instance.sh @@ -0,0 +1,51 @@ +#! /bin/bash + +set -x + +### /!\ This script assumes docker and docker compose are already installed on the host + +# Utility function to extract values from instance metadata +extract_metadata() { + curl "http://metadata.google.internal/computeMetadata/v1/instance/attributes/$1" -H "Metadata-Flavor: Google" +} + +# Exports an env var and also adds it to the root bashrc. This way if there is a need to ssh onto the machine +# for debugging one will have the env variables already set when using root +addVar() { + export $1 + echo "export $1" >> /root/.bashrc +} + +# Make sure ip forwarding is enabled by default so that docker doesn't lose connectivity +echo "net.ipv4.ip_forward = 1" > /etc/sysctl.conf + +# Set up env variables +addVar CROMWELL_BRANCH=$(extract_metadata CROMWELL_PERF_SCRIPTS_BRANCH) +addVar CLOUD_SQL_DB_USER=$(extract_metadata CROMWELL_DB_USER) +addVar CLOUD_SQL_DB_PASSWORD=$(extract_metadata CROMWELL_DB_PASS) +addVar CLOUD_SQL_INSTANCE=$(extract_metadata CLOUD_SQL_INSTANCE) +addVar CROMWELL_DOCKER_IMAGE=$(extract_metadata CROMWELL_DOCKER_IMAGE) +addVar CROMWELL_PROJECT=$(extract_metadata CROMWELL_PROJECT) +addVar CROMWELL_EXECUTION_ROOT=$(extract_metadata CROMWELL_BUCKET) +addVar CROMWELL_STATSD_HOST=$(extract_metadata CROMWELL_STATSD_HOST) +addVar CROMWELL_STATSD_PORT=$(extract_metadata CROMWELL_STATSD_PORT) +addVar BUILD_ID=$(extract_metadata BUILD_TAG) + +# Use the instance name as statsd prefix to avoid metrics collisions +addVar CROMWELL_STATSD_PREFIX=${BUILD_ID} +addVar REPORT_BUCKET=cromwell-perf-test-reporting + +addVar CROMWELL_ROOT=/app +addVar PERF_ROOT=${CROMWELL_ROOT}/scripts/perf +addVar TEST_WORKFLOW_ROOT=${PERF_ROOT}/test_cases + +# Clone cromwell to get the perf scripts. Use https to avoid ssh fingerprint prompt when the script runs +git clone -b ${CROMWELL_BRANCH} --depth 1 --single-branch https://github.com/broadinstitute/cromwell.git ${CROMWELL_ROOT} + +source ${PERF_ROOT}/helper.inc.sh + +addVar CROMWELL_CONF_DIR=${PERF_ROOT}/vm_scripts/cromwell + +# Start cromwell and cloud sql proxy +prepare_statsd_proxy +docker-compose -f ${PERF_ROOT}/vm_scripts/docker-compose.yml up -d diff --git a/scripts/perf/helper.inc.sh b/scripts/perf/helper.inc.sh index 35a6b31e2ca..cd99337ab2a 100644 --- a/scripts/perf/helper.inc.sh +++ b/scripts/perf/helper.inc.sh @@ -1,19 +1,28 @@ #!/usr/bin/env bash -set -x +# Set some global constants: +typeset DOCKER_ETC_PATH="/usr/share/etc" + +if test -f "/etc/vault-token-dsde" +then + typeset VAULT_TOKEN=$(cat /etc/vault-token-dsde) +fi + wait_for_cromwell() { git clone https://github.com/vishnubob/wait-for-it.git /wait-for-it chmod u+x /wait-for-it/wait-for-it.sh # Give 5 minutes to cromwell to be online - /wait-for-it/wait-for-it.sh localhost:8000 -t 300 + + echo "[$(date)] Waiting for http://${CROMWELL_UNDER_TEST}:8000/engine/v1/version to appear..." + /wait-for-it/wait-for-it.sh ${CROMWELL_UNDER_TEST}:8000 -t 300 READY=$? if [ ${READY} -eq 0 ] then # Just wait a bit longer - no rush, this is a chill VM - this is because cromwell responds to requests before being really ready... sleep 30 - CROMWELL_VERSION=$(curl -X GET "http://localhost:8000/engine/v1/version" -H "accept: application/json" | jq -r '.cromwell') + CROMWELL_VERSION=$(curl -X GET "http://${CROMWELL_UNDER_TEST}:8000/engine/v1/version" -H "accept: application/json" | jq -r '.cromwell') if [ -z ${CROMWELL_VERSION} ] then echo "Cromwell was up but failed to return its version, so something went wrong, shutting down" @@ -26,6 +35,42 @@ wait_for_cromwell() { fi } +# Waits for Cromwell. Works even from *outside* a google instance +custom_wait_for_cromwell() { + + set +e + + RESULT=1 + ATTEMPTS=0 + MAX_ATTEMPTS=20 + + while [ "${ATTEMPTS}" -le "${MAX_ATTEMPTS}" -a "${RESULT}" -gt "0" ] + do + echo "[$(date)] Waiting for Cromwell (http://${CROMWELL_UNDER_TEST}:8000/engine/v1/version) to come up (tried ${ATTEMPTS} times so far)" + sleep 30 + ATTEMPTS=$((ATTEMPTS + 1)) + + CROMWELL_VERSION_JSON=$(curl -X GET "http://${CROMWELL_UNDER_TEST}:8000/engine/v1/version" -H "accept: application/json") + RESULT=$? + + CROMWELL_VERSION=$(echo "${CROMWELL_VERSION_JSON}" | jq -r '.cromwell') + + if [ "${RESULT}" -eq "0" -a -z "${CROMWELL_VERSION}" ] + then + echo "Cromwell was up but failed to return its version, maybe something went wrong? Continuing waiting..." + RESULT=1 + fi + done + + if [ "${RESULT}" -gt "0" ] + then + echo "Cromwell never came up after ${ATTEMPTS} attempts" + exit 1 + else + echo "Cromwell came up after ${ATTEMPTS} attempts" + fi +} + export_logs() { export REPORT_URL="gs://${GCS_REPORT_BUCKET}/${GCS_REPORT_PATH}" @@ -58,10 +103,14 @@ clean_up() { if [ "${CLEAN_UP}" = true ] then gcloud sql instances delete ${CLOUD_SQL_INSTANCE} - gcloud compute instances delete $(curl -s "http://metadata.google.internal/computeMetadata/v1/instance/name" -H "Metadata-Flavor: Google") --zone=us-central1-c -q + clean_up_instance fi } +clean_up_instance() { + gcloud compute instances delete $(curl -s "http://metadata.google.internal/computeMetadata/v1/instance/name" -H "Metadata-Flavor: Google") --zone=us-central1-c -q +} + run_test() { cd ${CROMWELL_ROOT} @@ -70,7 +119,11 @@ run_test() { export GCS_REPORT_PATH="${TEST_CASE_DIRECTORY}/${CROMWELL_VERSION}/${BUILD_ID}" fi - sbt -Dconfig.file=${CENTAUR_CONF_DIR}/centaur.conf "centaur/it:testOnly centaur.ExternalTestCaseSpec" | tee centaur.log + sbt \ + -Dconfig.file=${CENTAUR_CONF_DIR}/centaur.conf \ + -Dcentaur.cromwell.mode="url" \ + -Dcentaur.cromwell.url="http://${CROMWELL_UNDER_TEST}:8000" \ + "centaur/it:testOnly centaur.ExternalTestCaseSpec" | tee centaur.log export TEST_RC=$? } @@ -79,3 +132,24 @@ shutdown() { docker-compose -f ${PERF_ROOT}/vm_scripts/docker-compose.yml down clean_up } + +read_path_from_vault_json() { + local path=$1 + local field=$2 + + docker run --rm -e VAULT_TOKEN=$VAULT_TOKEN broadinstitute/dsde-toolbox vault read -format=json "${path}" | jq --exit-status -r "${field}" +} + + +read_service_account_from_vault() { + mkdir -p mnt + read_path_from_vault_json "secret/dsp/cromwell/perf/service-account-deployer" '.data.service_account' > mnt/sa.json +} + +gcloud_run_as_service_account() { + local name="$1" + local command="$2" + docker run --name $name -v "$(pwd)"/mnt:${DOCKER_ETC_PATH} --rm google/cloud-sdk:slim /bin/bash -c "\ + gcloud auth activate-service-account --key-file ${DOCKER_ETC_PATH}/sa.json 2> /dev/null &&\ + ${command}" +} diff --git a/scripts/perf/startup_script.sh b/scripts/perf/startup_script.sh index 484b1093e9c..8194811fa30 100644 --- a/scripts/perf/startup_script.sh +++ b/scripts/perf/startup_script.sh @@ -43,6 +43,8 @@ addVar CROMWELL_ROOT=/app addVar PERF_ROOT=${CROMWELL_ROOT}/scripts/perf addVar TEST_WORKFLOW_ROOT=${PERF_ROOT}/test_cases +addVar CROMWELL_UNDER_TEST="localhost" + # Clone cromwell to get the perf scripts. Use https to avoid ssh fingerprint prompt when the script runs git clone -b ${CROMWELL_BRANCH} --depth 1 --single-branch https://github.com/broadinstitute/cromwell.git ${CROMWELL_ROOT} From 521cee9152519f5952a1a4021674047f4a592d93 Mon Sep 17 00:00:00 2001 From: Saloni Shah Date: Wed, 12 Jun 2019 08:34:40 -0400 Subject: [PATCH 09/28] Stackdriver Instrumentation BA-4788 (#5010) --- CHANGELOG.md | 13 ++ core/src/main/resources/reference.conf | 6 + .../cromwell.examples.conf | 16 +- docs/developers/Instrumentation.md | 87 +++++++-- project/Dependencies.scala | 25 ++- .../impl/stackdriver/StackdriverConfig.scala | 66 +++++++ ...ackdriverInstrumentationServiceActor.scala | 181 ++++++++++++++++++ .../impl/statsd/StatsDConfig.scala | 10 +- .../stackdriver/StackdriverConfigSpec.scala | 134 +++++++++++++ ...riverInstrumentationServiceActorSpec.scala | 111 +++++++++++ ...ackdriverInstrumentationServiceActor.scala | 28 +++ .../impl/statsd/StatsDConfigSpec.scala | 5 +- ...rumentationServiceActorBenchmarkSpec.scala | 3 +- ...tatsDInstrumentationServiceActorSpec.scala | 3 +- 14 files changed, 653 insertions(+), 35 deletions(-) create mode 100644 services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfig.scala create mode 100644 services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActor.scala create mode 100644 services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfigSpec.scala create mode 100644 services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActorSpec.scala create mode 100644 services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/TestStackdriverInstrumentationServiceActor.scala diff --git a/CHANGELOG.md b/CHANGELOG.md index 8895f64ab67..73ea7d40e65 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,18 @@ # Cromwell Change Log +## 43 Release Notes + +### Stackdriver Instrumentation + +Cromwell now supports sending metrics to [Google's Stackdriver API](https://cloud.google.com/monitoring/api/v3/). +Learn more on how to configure [here](https://cromwell.readthedocs.io/en/stable/developers/Instrumentation/). + +### StatsD Instrumentation Config change + +There is a small change in StatsD's configuration path. Originally, the path to the config was `services.Instrumentation.config.statsd` +which now has been updated to `services.Instrumentation.config`. More info on its configuration can be found +[here](https://cromwell.readthedocs.io/en/stable/developers/Instrumentation/). + ## 42 Release Notes ### Womtool endpoint diff --git a/core/src/main/resources/reference.conf b/core/src/main/resources/reference.conf index 0eabd4eb146..c729c46af6e 100644 --- a/core/src/main/resources/reference.conf +++ b/core/src/main/resources/reference.conf @@ -497,6 +497,12 @@ services { Instrumentation { # Default noop service - instrumentation metrics are ignored class = "cromwell.services.instrumentation.impl.noop.NoopInstrumentationServiceActor" + + # StatsD instrumentation service actor + # class = "cromwell.services.instrumentation.impl.statsd.StatsDInstrumentationServiceActor" + + # Stackdriver instrumentation service actor + # class = "cromwell.services.instrumentation.impl.stackdriver.StackdriverInstrumentationServiceActor" } HealthMonitor { class = "cromwell.services.healthmonitor.impl.HealthMonitorServiceActor" diff --git a/cromwell.example.backends/cromwell.examples.conf b/cromwell.example.backends/cromwell.examples.conf index 1dc77f0f1cf..4318bce04bd 100644 --- a/cromwell.example.backends/cromwell.examples.conf +++ b/cromwell.example.backends/cromwell.examples.conf @@ -495,12 +495,26 @@ services { Instrumentation { # StatsD - Send metrics to a StatsD server over UDP # class = "cromwell.services.instrumentation.impl.statsd.StatsDInstrumentationServiceActor" - # config.statsd { + # config { # hostname = "localhost" # port = 8125 # prefix = "" # can be used to prefix all metrics with an api key for example # flush-rate = 1 second # rate at which aggregated metrics will be sent to statsd # } + + # Stackdriver - Send metrics to Google's monitoring API + # class = "cromwell.services.instrumentation.impl.stackdriver.StackdriverInstrumentationServiceActor" + # config { + # # auth scheme can be `application_default` or `service_account` + # auth = "service-account" + # google-project = "my-project" + # # rate at which aggregated metrics will be sent to Stackdriver API, must be 1 minute or more. + # flush-rate = 1 minute + # # below 3 keys are attached as labels to each metric. `cromwell-perf-test-case` is specifically meant for perf env. + # cromwell-instance-identifier = "cromwell-101" + # cromwell-instance-role = "role" + # cromwell-perf-test-case = "perf-test-1" + # } } HealthMonitor { config { diff --git a/docs/developers/Instrumentation.md b/docs/developers/Instrumentation.md index c5a250542e1..ef3f72a76a5 100644 --- a/docs/developers/Instrumentation.md +++ b/docs/developers/Instrumentation.md @@ -1,28 +1,28 @@ -**Overview** +# Overview Cromwell's instrumentation support can be useful to collect utilization data in long-running, high-volume -production environments. While this instrumentation support can be used in smaller environments it will still require setting up a -[StatsD](https://github.com/etsy/statsd) server outside of Cromwell and it's possible not enough data would be produced to be useful. +production environments. The default implementation of this ignores these metrics, but Cromwell includes alternate implementations that can forward metrics to a +specific server. -**StatsD** +### StatsD -Cromwell collects metrics while running and sends them to an internal service. The default implementation of this service -ignores these metrics, but Cromwell includes an alternate implementation that can forward metrics to a -[StatsD](https://github.com/etsy/statsd) server. -To specify this implementation in your configuration file: +While this instrumentation support can be used in smaller environments it will still require setting up a +[StatsD](https://github.com/etsy/statsd) server outside of Cromwell and it's possible not enough data would be produced to be useful. +Cromwell collects metrics while running and sends them to an internal service. -```hocon -services.Instrumentation.class = "cromwell.services.instrumentation.impl.statsd.StatsDInstrumentationServiceActor" -``` Make sure to configure your StatsD service: ```hocon -services.Instrumentation.config.statsd { - hostname = "localhost" # Replace with your host - port = 8125 # Replace with your port - # prefix = "my_prefix" # All metrics will be prefixed by this value if present. - flush-rate = 1 second # Rate at which metrics are sent to the StatsD server - } +services.Instrumentation { + class = "cromwell.services.instrumentation.impl.statsd.StatsDInstrumentationServiceActor" + + config { + hostname = "localhost" # Replace with your host + port = 8125 # Replace with your port + # prefix = "my_prefix" # All metrics will be prefixed by this value if present. + flush-rate = 1 second # Rate at which metrics are sent to the StatsD server + } +} ``` There is also an additional configuration value that can be set: @@ -39,7 +39,7 @@ system.cromwell_id = "cromwell-instance-1" will prepend all the metrics with path `cromwell.cromwell-instance-1...` for that instance. -**Metrics** +##### Metrics The current StatsD implementation uses metrics-statsd to report instrumentation values. metrics-statsd reports all metrics with a gauge type. @@ -51,3 +51,54 @@ These are the current high level categories: * `job` * `workflow` * `io` + + +### Stackdriver + +Cromwell now supports sending metrics to [Google's Stackdriver API](https://cloud.google.com/monitoring/api/v3/). To use the Stackdriver instrumentation +specify this in your config: +```hocon +services.Instrumentation { + class = "cromwell.services.instrumentation.impl.stackdriver.StackdriverInstrumentationServiceActor" + + config { + # auth scheme can be `application_default` or `service_account` + auth = "service-account" + google-project = "my-project" + # rate at which aggregated metrics will be sent to Stackdriver. It needs to be equal or greater than 1 minute. + # Google's Stackdriver API needs each metric to be sent not more than once per minute. + flush-rate = 1 minute + # below 3 keys are attached as labels to each metric. `cromwell-perf-test-case` is specifically meant for perf env. + cromwell-instance-role = "role" + cromwell-perf-test-case = "perf-test-1" + } + } +``` +The 2 label keys are optional. If specified, each metric will have label(s) added in the form of a (key, value) pair. +So for example, if `cromwell-instance-role = "backend"` is mentioned in config, each metric data point sent to Stackdriver +will have a label (cromwell_instance_role, backend) added to it. + +There is another optional label that can be added to each metric. `cromwell_id` represents the identifier for different Cromwell instances. +```hocon +# Unique Cromwell instance identifier +system.cromwell_id = "cromwell-instance-1" +``` + + +##### Metric type and Label keys naming convention +More details on the this can be found [here](https://cloud.google.com/monitoring/api/v3/metrics-details#metric-kinds). + +You must adhere to the following spelling rules for metric type names: +- You can use upper and lower-case letters, digits, and underscores (_) in the names. +- You can use periods (.) in the domain part of the names. +- You can use forward slashes (/) to separate path elements. +- You can start each path element with a letter or digit. +- The maximum length of a metric type name is 200 characters. + +You must adhere to the following spelling rules for metric label names: +- You can use upper and lower-case letters, digits, underscores (_) in the names. +- You can start names with a letter or digit. +- The maximum length of a metric label name is 100 characters. + + + diff --git a/project/Dependencies.scala b/project/Dependencies.scala index 4ceea23f4a8..54c0e1241f7 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -29,6 +29,7 @@ object Dependencies { private val googleApiClientV = "1.28.0" private val googleCloudCoreV = "1.61.0" private val googleCloudKmsV = "v1-rev63-1.25.0" + private val googleCloudMonitoringV = "1.70.0" private val googleCloudNioV = "0.61.0-alpha" private val googleGenomicsServicesV1ApiV = "v1alpha2-rev495-1.23.0" private val googleGenomicsServicesV2ApiV = "v2alpha1-rev31-1.25.0" @@ -36,7 +37,7 @@ object Dependencies { private val googleHttpClientV = "1.29.1" private val googleOauth2V = "0.13.0" private val googleCloudResourceManagerV = "0.87.0-alpha" - private val grpcV = "1.18.0" + private val grpcV = "1.20.0" private val guavaV = "27.0.1-jre" private val heterodonV = "1.0.0-beta3" private val hsqldbV = "2.4.1" @@ -345,6 +346,10 @@ object Dependencies { "com.readytalk" % "metrics3-statsd" % metrics3StatsdV ) + val stackdriverDependencies = List( + "com.google.cloud" % "google-cloud-monitoring" % googleCloudMonitoringV + ) + val gcsFileSystemDependencies = akkaHttpDependencies val httpFileSystemDependencies = akkaHttpDependencies @@ -434,7 +439,7 @@ object Dependencies { ) ++ akkaStreamDependencies ++ configDependencies ++ catsDependencies ++ circeDependencies ++ googleApiClientDependencies ++ statsDDependencies ++ betterFilesDependencies ++ // TODO: We're not using the "F" in slf4j. Core only supports logback, specifically the WorkflowLogger. - slf4jBindingDependencies + slf4jBindingDependencies ++ stackdriverDependencies val databaseMigrationDependencies = liquibaseDependencies ++ dbmsDependencies @@ -601,6 +606,19 @@ object Dependencies { "util", ).map(m => "org.eclipse.rdf4j" % s"rdf4j-$m" % rdf4jV) + // Some libraries are importing older version of these dependencies, causing conflicts. Hence the need to override them. + val grpcDependencyOverrides = List( + "alts", + "auth", + "context", + "core", + "grpclb", + "netty-shaded", + "protobuf-lite", + "protobuf", + "stub", + ).map(m => "io.grpc" % s"grpc-$m" % grpcV) + /* If we use a version in one of our projects, that's the one we want all the libraries to use ...plus other groups of transitive dependencies shared across multiple projects @@ -609,5 +627,6 @@ object Dependencies { allProjectDependencies ++ googleHttpClientDependencies ++ nettyDependencyOverrides ++ - rdf4jDependencyOverrides + rdf4jDependencyOverrides ++ + grpcDependencyOverrides } diff --git a/services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfig.scala b/services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfig.scala new file mode 100644 index 00000000000..2131eefbeaf --- /dev/null +++ b/services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfig.scala @@ -0,0 +1,66 @@ +package cromwell.services.instrumentation.impl.stackdriver + +import cats.data.Validated._ +import cats.syntax.apply._ +import cats.syntax.validated._ +import com.typesafe.config.Config +import common.exception.AggregatedMessageException +import common.validation.ErrorOr.ErrorOr +import common.validation.Validation.validate +import cromwell.cloudsupport.gcp.GoogleConfiguration +import cromwell.cloudsupport.gcp.auth._ +import net.ceedubs.ficus.Ficus._ + +import scala.concurrent.duration._ + +case class StackdriverConfig(googleProject: String, + auth: GoogleAuthMode, + flushRate: FiniteDuration, + cromwellInstanceIdentifier: Option[String], + cromwellInstanceRole: Option[String], + cromwellPerfTestCase: Option[String]) + +object StackdriverConfig { + val CromwellInstanceIdentifier = "cromwell-instance-identifier" + val CromwellInstanceRole = "cromwell-instance-role" + val CromwellPerfTest = "cromwell-perf-test-case" + + private def validateFlushRate(rateFunc: => FiniteDuration): ErrorOr[FiniteDuration] = { + validate[FiniteDuration](rateFunc) match { + case Valid(duration) => duration match { + case _ if duration < 1.minute => (s"`flush-rate` must be 1 minute or longer, specified rate is `$duration`. " + + s"Google's Stackdriver API needs each metric to be sent not more than once per minute.").invalidNel + case _ => duration.validNel + } + case Invalid(e) => e.invalid + } + } + + + private def validateAuth(authSchemeFunc: => String, googleConfiguration: GoogleConfiguration): ErrorOr[GoogleAuthMode] = { + validate[String](authSchemeFunc) match { + case Valid(schemeString) => googleConfiguration.auth(schemeString) match { + case Valid(auth @ (_:ApplicationDefaultMode | _:ServiceAccountMode)) => auth.valid + case Valid(_) => s"`auth` scheme: $schemeString is not allowed for Stackdriver instrumentation. Only `application_default` and `service_account` modes are valid.".invalidNel + case Invalid(error) => s"`auth` scheme is invalid. Errors: $error".invalidNel + } + case Invalid(e) => e.invalid + } + } + + + def apply(serviceConfig: Config, globalConfig: Config): StackdriverConfig = { + val googleConfiguration: GoogleConfiguration = GoogleConfiguration(globalConfig) + val cromwellInstanceId: ErrorOr[Option[String]] = globalConfig.getAs[String]("system.cromwell_id").validNel + + val googleProject: ErrorOr[String] = validate[String] { serviceConfig.as[String]("google-project") } + val authScheme: ErrorOr[GoogleAuthMode] = validateAuth(serviceConfig.as[String]("auth"), googleConfiguration) + val flushRate: ErrorOr[FiniteDuration] = validateFlushRate(serviceConfig.as[FiniteDuration]("flush-rate")) + val cromwellInstanceRole: ErrorOr[Option[String]] = serviceConfig.getAs[String](CromwellInstanceRole).validNel + val cromwellPerfTestCase: ErrorOr[Option[String]] = serviceConfig.getAs[String](CromwellPerfTest).validNel + + (googleProject, authScheme, flushRate, cromwellInstanceId, cromwellInstanceRole, cromwellPerfTestCase).mapN({ (p, a, f, i, r, t) => + new StackdriverConfig(p, a, f, i, r, t) + }).valueOr(errors => throw AggregatedMessageException("Stackdriver instrumentation config is invalid. Error(s)", errors.toList)) + } +} diff --git a/services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActor.scala b/services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActor.scala new file mode 100644 index 00000000000..1dd5462f62d --- /dev/null +++ b/services/src/main/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActor.scala @@ -0,0 +1,181 @@ +package cromwell.services.instrumentation.impl.stackdriver + +import java.util + +import akka.actor.{Actor, ActorRef, Props} +import com.google.api.gax.core.FixedCredentialsProvider +import com.google.api.{Metric, MetricDescriptor, MonitoredResource} +import com.google.cloud.monitoring.v3.{MetricServiceClient, MetricServiceSettings} +import com.google.monitoring.v3._ +import com.google.protobuf.util.Timestamps +import com.typesafe.config.Config +import com.typesafe.scalalogging.StrictLogging +import cromwell.services.instrumentation.InstrumentationService.InstrumentationServiceMessage +import cromwell.services.instrumentation._ +import cromwell.services.instrumentation.impl.stackdriver.StackdriverConfig._ +import cromwell.services.instrumentation.impl.stackdriver.StackdriverInstrumentationServiceActor._ +import cromwell.util.GracefulShutdownHelper.ShutdownCommand + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.util.Try + + +class StackdriverInstrumentationServiceActor(serviceConfig: Config, globalConfig: Config, serviceRegistryActor: ActorRef) extends Actor with StrictLogging { + implicit lazy val executionContext = context.dispatcher + + val stackdriverConfig = StackdriverConfig(serviceConfig, globalConfig) + + lazy val projectName: ProjectName = ProjectName.of(stackdriverConfig.googleProject) + val credentials = stackdriverConfig.auth.credentials(List(MonitoringScope)) + lazy val metricLabelsMap = generateMetricLabels() + + var metricsMap = Map.empty[StackdriverMetric, Vector[Double]] + + // Instantiates a client + val metricServiceSettings = MetricServiceSettings.newBuilder.setCredentialsProvider(FixedCredentialsProvider.create(credentials)).build + final val metricServiceClient = MetricServiceClient.create(metricServiceSettings) + + // Prepares the monitored resource descriptor + lazy val resourceLabels = Map[String, String](("project_id", stackdriverConfig.googleProject)) + lazy val monitoredResource = MonitoredResource.newBuilder.setType("global").putAllLabels(resourceLabels.asJava).build + + // Give the actor time to warm up, then start sending the metrics to Stackdriver at an interval + context.system.scheduler.schedule(InitialDelay, stackdriverConfig.flushRate, self, SendStackdriverMetricCommand) + + + override def receive = { + case SendStackdriverMetricCommand => sendMetricData() + case InstrumentationServiceMessage(cromwellMetric) => cromwellMetric match { + case CromwellTiming(bucket, value, _) => updateMetricMap(bucket, value.toMillis.toDouble, StackdriverGauge) + case CromwellGauge(bucket, value) => updateMetricMap(bucket, value.toDouble, StackdriverGauge) + case CromwellCount(bucket, value, _) => updateMetricMap(bucket, value.toDouble, StackdriverCumulative) + case CromwellIncrement(bucket) => updateMetricMap(bucket, metricValue = 1D, metricKind = StackdriverCumulative) + } + case ShutdownCommand => + // flush out metrics (if any) before shut down + sendMetricData() + metricServiceClient.close() + context stop self + } + + + private def generateMetricLabels(): Map[String, String] = { + def labelFromConfig(op: StackdriverConfig => Option[String], key: String): Option[(String, String)] = { + op(stackdriverConfig).map(v => (key.replace("-", "_"), v)) + } + + labelFromConfig(_.cromwellInstanceIdentifier, CromwellInstanceIdentifier).toMap ++ + labelFromConfig(_.cromwellInstanceRole, CromwellInstanceRole) ++ + labelFromConfig(_.cromwellPerfTestCase, CromwellPerfTest) + } + + + private def updateMetricMap(bucket: CromwellBucket, metricValue: Double, metricKind: StackdriverMetricKind): Unit = { + val metricObj = StackdriverMetric(bucket.toStackdriverString, metricKind) + + if (metricsMap.contains(metricObj)) { + val valueVector: Vector[Double] = metricsMap(metricObj) :+ metricValue + metricsMap += metricObj -> valueVector + } + else metricsMap += metricObj -> Vector(metricValue) + } + + + private def sendMetricData(): Unit = { + metricsMap.foreach { case (key, value: Vector[Double]) => + val dataPointVectorSum = value.sum + val dataPoint = key.kind match { + case StackdriverGauge => dataPointVectorSum / value.length + case StackdriverCumulative => dataPointVectorSum + } + + writeMetrics(key, dataPoint) recover { + case e => logger.error(s"Failed to send metrics to Stackdriver API for metric ${key.name} with value $dataPoint.", e) + } + } + + metricsMap = Map.empty[StackdriverMetric, Vector[Double]] + } + + + private def writeMetrics(metricObj: StackdriverMetric, value: Double): Try[Unit] = { + def timeInterval(metricKind: StackdriverMetricKind): TimeInterval = { + metricKind match { + case StackdriverGauge => TimeInterval.newBuilder.setEndTime(Timestamps.fromMillis(System.currentTimeMillis)).build + case StackdriverCumulative => TimeInterval.newBuilder.setStartTime(ActorCreationTime).setEndTime(Timestamps.fromMillis(System.currentTimeMillis)).build + } + } + + def createTimeSeries(metricKind: StackdriverMetricKind, metric: Metric, resource: MonitoredResource, dataPointList: util.List[Point]): TimeSeries = { + metricKind match { + case StackdriverGauge => TimeSeries.newBuilder.setMetric(metric).setResource(resource).addAllPoints(dataPointList).build + case StackdriverCumulative => TimeSeries.newBuilder.setMetric(metric).setResource(resource).setMetricKind(MetricDescriptor.MetricKind.CUMULATIVE).addAllPoints(dataPointList).build + } + } + + // Prepares an individual data point + val interval = timeInterval(metricObj.kind) + val pointValue = TypedValue.newBuilder().setDoubleValue(value).build() + val dataPoint: Point = Point.newBuilder.setInterval(interval).setValue(pointValue).build + val dataPointList: List[Point] = List[Point](dataPoint) + + // Prepares the metric descriptor + val metric: Metric = Metric.newBuilder.setType(s"$CustomMetricDomain/${metricObj.name}").putAllLabels(metricLabelsMap.asJava).build + + // Prepares the time series request + val timeSeries = createTimeSeries(metricObj.kind, metric, monitoredResource, dataPointList.asJava) + val timeSeriesList = List[TimeSeries](timeSeries) + + val timeSeriesRequest = CreateTimeSeriesRequest.newBuilder.setName(projectName.toString).addAllTimeSeries(timeSeriesList.asJava).build + + // Writes time series data + Try(sendTimeSeriesToStackdriver(timeSeriesRequest)) + } + + + // This single line of code is a separate function to help with StackdriverInstrumentationActor tests + def sendTimeSeriesToStackdriver(timeSeriesRequest: CreateTimeSeriesRequest): Unit = { + metricServiceClient.createTimeSeries(timeSeriesRequest) + } +} + + +object StackdriverInstrumentationServiceActor { + val CromwellMetricPrefix = List("cromwell") + + val ActorCreationTime = Timestamps.fromMillis(System.currentTimeMillis()) + + // Custom metrics must begin with this domain + val CustomMetricDomain = "custom.googleapis.com" + + /** + * Scope to write metrics to Stackdriver Monitoring API. + * Used by the monitoring action. + * + * For some reason we couldn't find this scope within Google libraries + */ + val MonitoringScope = "https://www.googleapis.com/auth/monitoring" + + val InitialDelay = 1.minute + + def props(serviceConfig: Config, globalConfig: Config, serviceRegistryActor: ActorRef) = Props(new StackdriverInstrumentationServiceActor(serviceConfig, globalConfig, serviceRegistryActor)) + + implicit class CromwellBucketEnhanced(val cromwellBucket: CromwellBucket) extends AnyVal { + /** + * Transforms a CromwellBucket to a Stackdriver path + */ + def toStackdriverString = (CromwellMetricPrefix ++ cromwellBucket.prefix ++ cromwellBucket.path.toList).mkString("/").replace(" ", "_") + } +} + + +sealed trait StackdriverMetricKind +object StackdriverGauge extends StackdriverMetricKind +object StackdriverCumulative extends StackdriverMetricKind + + +case class StackdriverMetric(name: String, kind: StackdriverMetricKind) + + +object SendStackdriverMetricCommand diff --git a/services/src/main/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfig.scala b/services/src/main/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfig.scala index 4e18a479d23..d67eb348bdf 100644 --- a/services/src/main/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfig.scala +++ b/services/src/main/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfig.scala @@ -2,7 +2,6 @@ package cromwell.services.instrumentation.impl.statsd import cats.data.Validated._ import cats.syntax.apply._ -import cats.syntax.validated._ import com.typesafe.config.Config import common.exception.MessageAggregation import common.validation.ErrorOr.ErrorOr @@ -15,12 +14,11 @@ case class StatsDConfig(hostname: String, port: Int, prefix: Option[String], flu object StatsDConfig { def apply(serviceConfig: Config): StatsDConfig = { - val statsDConfig = serviceConfig.getConfig("statsd") - val hostname: ErrorOr[String] = validate[String] { statsDConfig.as[String]("hostname") } - val port: ErrorOr[Int] = validate[Int] { statsDConfig.as[Int]("port") } - val prefix: ErrorOr[Option[String]] = statsDConfig.as[Option[String]]("prefix").validNel - val flushRate: ErrorOr[FiniteDuration] = validate[FiniteDuration] { statsDConfig.as[FiniteDuration]("flush-rate") } + val hostname: ErrorOr[String] = validate[String] { serviceConfig.as[String]("hostname") } + val port: ErrorOr[Int] = validate[Int] { serviceConfig.as[Int]("port") } + val prefix: ErrorOr[Option[String]] = validate { serviceConfig.as[Option[String]]("prefix") } + val flushRate: ErrorOr[FiniteDuration] = validate[FiniteDuration] { serviceConfig.as[FiniteDuration]("flush-rate") } (hostname, port, prefix, flushRate).mapN({ (h, p, n, f) => new StatsDConfig(h, p, n, f) diff --git a/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfigSpec.scala b/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfigSpec.scala new file mode 100644 index 00000000000..aad69c94212 --- /dev/null +++ b/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverConfigSpec.scala @@ -0,0 +1,134 @@ +package cromwell.services.instrumentation.impl.stackdriver + +import com.typesafe.config.ConfigFactory +import common.exception.AggregatedMessageException +import cromwell.core.TestKitSuite +import org.scalatest.{BeforeAndAfterAll, FlatSpecLike, Matchers} + +import scala.concurrent.duration._ + +class StackdriverConfigSpec extends TestKitSuite with FlatSpecLike with BeforeAndAfterAll with Matchers { + behavior of "StackdriverConfig" + + val googleConfig = ConfigFactory.parseString( + s""" + |google { + | application-name = "cromwell" + | auths = [ + | { + | name = "application-default" + | scheme = "application_default" + | } + | ] + |} + """.stripMargin + ) + + it should "correctly parse all config" in { + val config = ConfigFactory.parseString( + """ + |auth = "application-default" + |google-project = "my-project" + |flush-rate = 1 minute + |cromwell-instance-role = "backend" + |cromwell-perf-test-case = "perf-test-1" + """.stripMargin + ) + + val stackdriverConfig = StackdriverConfig(config, googleConfig) + + stackdriverConfig.auth.name shouldBe "application-default" + stackdriverConfig.googleProject shouldBe "my-project" + stackdriverConfig.flushRate shouldBe 1.minute + stackdriverConfig.cromwellInstanceRole shouldBe Option("backend") + stackdriverConfig.cromwellPerfTestCase shouldBe Option("perf-test-1") + } + + + it should "correctly parse config with optional values" in { + val config = ConfigFactory.parseString( + """ + |auth = "application-default" + |google-project = "my-project" + |flush-rate = 1 minute + """.stripMargin + ) + + val stackdriverConfig = StackdriverConfig(config, googleConfig) + + stackdriverConfig.auth.name shouldBe "application-default" + stackdriverConfig.googleProject shouldBe "my-project" + stackdriverConfig.flushRate shouldBe 1.minute + stackdriverConfig.cromwellInstanceIdentifier shouldBe None + stackdriverConfig.cromwellInstanceRole shouldBe None + stackdriverConfig.cromwellPerfTestCase shouldBe None + } + + + it should "correctly add cromwell instance identifier to config" in { + val globalConfig = ConfigFactory.parseString( + s""" + |google { + | application-name = "cromwell" + | auths = [ + | { + | name = "application-default" + | scheme = "application_default" + | } + | ] + |} + | + |system.cromwell_id = "cromwell-101" + """.stripMargin + ) + + val config = ConfigFactory.parseString( + """ + |auth = "application-default" + |google-project = "my-project" + |flush-rate = 1 minute + """.stripMargin + ) + + val stackdriverConfig = StackdriverConfig(config, globalConfig) + + stackdriverConfig.auth.name shouldBe "application-default" + stackdriverConfig.googleProject shouldBe "my-project" + stackdriverConfig.flushRate shouldBe 1.minute + stackdriverConfig.cromwellInstanceIdentifier shouldBe Option("cromwell-101") + stackdriverConfig.cromwellInstanceRole shouldBe None + stackdriverConfig.cromwellPerfTestCase shouldBe None + } + + + it should "throw error for invalid auth" in { + val config = ConfigFactory.parseString( + """ + |auth = "my-auth" + |google-project = "my-project" + |flush-rate = 1 minute + """.stripMargin + ) + + val exception = the[AggregatedMessageException] thrownBy StackdriverConfig(config, googleConfig) + + exception.getMessage shouldBe """Stackdriver instrumentation config is invalid. Error(s): + |`auth` scheme is invalid. Errors: NonEmptyList(`google` configuration stanza does not contain an auth named 'my-auth'. Known auth names: application-default)""".stripMargin + } + + + it should "throw error for invalid flush rate" in { + val config = ConfigFactory.parseString( + """ + |auth = "application-default" + |google-project = "my-project" + |flush-rate = 30 seconds + """.stripMargin + ) + + val exception = the[AggregatedMessageException] thrownBy StackdriverConfig(config, googleConfig) + + exception.getMessage shouldBe """Stackdriver instrumentation config is invalid. Error(s): + |`flush-rate` must be 1 minute or longer, specified rate is `30 seconds`. Google's Stackdriver API needs each metric to be sent not more than once per minute.""".stripMargin + } +} diff --git a/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActorSpec.scala b/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActorSpec.scala new file mode 100644 index 00000000000..02d4a6acdcf --- /dev/null +++ b/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/StackdriverInstrumentationServiceActorSpec.scala @@ -0,0 +1,111 @@ +package cromwell.services.instrumentation.impl.stackdriver + +import akka.testkit.{TestActorRef, TestProbe} +import cats.data.NonEmptyList +import com.typesafe.config.ConfigFactory +import cromwell.core.TestKitSuite +import cromwell.services.instrumentation.InstrumentationService.InstrumentationServiceMessage +import cromwell.services.instrumentation._ +import org.scalatest.concurrent.Eventually +import org.scalatest.{FlatSpecLike, Matchers} + +import scala.concurrent.duration._ + +class StackdriverInstrumentationServiceActorSpec extends TestKitSuite with FlatSpecLike with Matchers with Eventually { + behavior of "StackdriverInstrumentationServiceActor" + + val MaxWaitTime = 2.minutes + implicit val pc: PatienceConfig = PatienceConfig(MaxWaitTime) + + val globalConfig = ConfigFactory.parseString( + s""" + |google { + | application-name = "cromwell" + | auths = [ + | { + | name = "application-default" + | scheme = "application_default" + | } + | ] + |} + """.stripMargin + ) + + val registryProbe = TestProbe().ref + val resourceLabels = Map("project_id" -> "my-project") + + val testBucket = CromwellBucket(List("test_prefix"), NonEmptyList.of("test", "metric", "bucket")) + val testGaugeBucket = CromwellBucket(List("test_prefix"), NonEmptyList.of("test", "gauge", "metric", "bucket")) + + + it should "correctly receive the metrics with resource labels" in { + val stackdriverConfig = ConfigFactory.parseString( + """ + |auth = "application-default" + |google-project = "my-project" + |flush-rate = 1 minute + """.stripMargin + ) + + val rawMetricList = List( + CromwellIncrement(testBucket), + CromwellCount(testBucket, 80), + CromwellGauge(testGaugeBucket, 20), + CromwellTiming(testBucket.expand("timing"), 5.seconds), + CromwellIncrement(testBucket), + CromwellCount(testBucket, 80), + CromwellGauge(testGaugeBucket, 60), + CromwellTiming(testBucket.expand("timing"), 10.seconds) + ) + + val expectedCumulativeMetrics = ("custom.googleapis.com/cromwell/test_prefix/test/metric/bucket", 162.0) + val expectedGaugeMetrics = ("custom.googleapis.com/cromwell/test_prefix/test/gauge/metric/bucket", 40.0) + val expectedTimingMetrics = ("custom.googleapis.com/cromwell/test_prefix/test/metric/bucket/timing", 7500.0) + + val stackdriverActor = TestActorRef(new TestStackdriverInstrumentationServiceActor(stackdriverConfig, globalConfig, registryProbe)) + + rawMetricList foreach (metric => stackdriverActor ! InstrumentationServiceMessage(metric)) + + eventually { + stackdriverActor.underlyingActor.metricsReceived should have length 3 + + stackdriverActor.underlyingActor.metricsReceived.map(m => (m.metricPath, m.metricValue)) should contain (expectedCumulativeMetrics) + stackdriverActor.underlyingActor.metricsReceived.map(m => (m.metricPath, m.metricValue)) should contain (expectedGaugeMetrics) + stackdriverActor.underlyingActor.metricsReceived.map(m => (m.metricPath, m.metricValue)) should contain (expectedTimingMetrics) + + stackdriverActor.underlyingActor.metricsReceived.map(_.resourceLabels) should contain (resourceLabels) + } + } + + + it should "correctly receive metrics with metric labels" in { + val stackdriverConfig = ConfigFactory.parseString( + """ + |auth = "application-default" + |google-project = "my-project" + |flush-rate = 1 minute + |cromwell-instance-role = "backend" + |cromwell-perf-test-case = "perf-test-1" + """.stripMargin + ) + + val rawMetricList = List( + CromwellIncrement(testBucket), + CromwellCount(testBucket, 49) + ) + + val expectedCumulativeMetrics = ("custom.googleapis.com/cromwell/test_prefix/test/metric/bucket", 50.0) + val metricLabels = Map("cromwell_instance_role" -> "backend", "cromwell_perf_test_case" -> "perf-test-1") + + val stackdriverActor = TestActorRef(new TestStackdriverInstrumentationServiceActor(stackdriverConfig, globalConfig, registryProbe)) + + rawMetricList foreach (metric => stackdriverActor ! InstrumentationServiceMessage(metric)) + + eventually { + stackdriverActor.underlyingActor.metricsReceived should have length 1 + stackdriverActor.underlyingActor.metricsReceived.map(m => (m.metricPath, m.metricValue)) should contain (expectedCumulativeMetrics) + stackdriverActor.underlyingActor.metricsReceived.map(_.resourceLabels) should contain (resourceLabels) + stackdriverActor.underlyingActor.metricsReceived.map(_.metricLabels) should contain (metricLabels) + } + } +} diff --git a/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/TestStackdriverInstrumentationServiceActor.scala b/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/TestStackdriverInstrumentationServiceActor.scala new file mode 100644 index 00000000000..e804b3593ac --- /dev/null +++ b/services/src/test/scala/cromwell/services/instrumentation/impl/stackdriver/TestStackdriverInstrumentationServiceActor.scala @@ -0,0 +1,28 @@ +package cromwell.services.instrumentation.impl.stackdriver + +import akka.actor.ActorRef +import com.google.monitoring.v3.CreateTimeSeriesRequest +import com.typesafe.config.Config +import scala.collection.JavaConverters._ + +class TestStackdriverInstrumentationServiceActor(serviceConfig: Config, globalConfig: Config, serviceRegistryActor: ActorRef) + extends StackdriverInstrumentationServiceActor(serviceConfig, globalConfig, serviceRegistryActor) { + + var metricsReceived = List[TimeSeriesRequest]() + + override def sendTimeSeriesToStackdriver(timeSeriesRequest: CreateTimeSeriesRequest) = { + val timeSeries = timeSeriesRequest.getTimeSeries(0) + val metric = timeSeries.getMetric + + metricsReceived = metricsReceived :+ TimeSeriesRequest(metric.getType, + timeSeries.getPoints(0).getValue.getDoubleValue, + timeSeries.getResource.getLabelsMap.asScala.toMap, + metric.getLabelsMap.asScala.toMap) + } +} + + +case class TimeSeriesRequest(metricPath: String, + metricValue: Double, + resourceLabels: Map[String, String], + metricLabels: Map[String, String]) diff --git a/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfigSpec.scala b/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfigSpec.scala index e221587c476..cc491ca4859 100644 --- a/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfigSpec.scala +++ b/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDConfigSpec.scala @@ -9,12 +9,11 @@ class StatsDConfigSpec extends FlatSpec with Matchers { it should "parse correct service configuration" in { val config = ConfigFactory.parseString( - """statsd { + """ |hostname = "localhost" |port = 8125 |prefix = "prefix_value" # can be used to prefix all metrics with an api key for example |flush-rate = 1 second # rate at which aggregated metrics will be sent to statsd - |} """.stripMargin ) @@ -27,7 +26,7 @@ class StatsDConfigSpec extends FlatSpec with Matchers { } it should "not parse incorrect service configuration" in { - val config = ConfigFactory.parseString("statsd {}") + val config = ConfigFactory.parseString("{}") val exception = the[IllegalArgumentException] thrownBy StatsDConfig(config) diff --git a/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorBenchmarkSpec.scala b/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorBenchmarkSpec.scala index 14b0e7b9817..f806b24a98e 100644 --- a/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorBenchmarkSpec.scala +++ b/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorBenchmarkSpec.scala @@ -16,12 +16,11 @@ class StatsDInstrumentationServiceActorBenchmarkSpec extends TestKitSuite with F behavior of "StatsDInstrumentationServiceActor" val config = ConfigFactory.parseString( - """statsd { + """ |hostname = "localhost" |port = 8125 |prefix = "prefix_value" # can be used to prefix all metrics with an api key for example |flush-rate = 100 ms # rate at which aggregated metrics will be sent to statsd - |} """.stripMargin ) diff --git a/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorSpec.scala b/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorSpec.scala index 81b6d67c510..5e5087e8bdb 100644 --- a/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorSpec.scala +++ b/services/src/test/scala/cromwell/services/instrumentation/impl/statsd/StatsDInstrumentationServiceActorSpec.scala @@ -17,12 +17,11 @@ class StatsDInstrumentationServiceActorSpec extends TestKitSuite with FlatSpecLi behavior of "StatsDInstrumentationServiceActor" val config = ConfigFactory.parseString( - """statsd { + """ |hostname = "localhost" |port = 8125 |prefix = "prefix_value" # can be used to prefix all metrics with an api key for example |flush-rate = 100 ms # rate at which aggregated metrics will be sent to statsd - |} """.stripMargin ) From 305e6640709122d7f36151f2da4c4db75a2f744e Mon Sep 17 00:00:00 2001 From: Khalid Shakir Date: Wed, 12 Jun 2019 09:25:03 -0400 Subject: [PATCH 10/28] Running dockers for postgres and mariadb during tests - Switched from running Travis' mysql/postgresql instances to dockerized versions - Added a couple of centaur and some unit tests for mariadb, using mysql liquibase/drivers for now - NOTE: No postgresql tests enabled in this commit - Changed mysql CI login from travis: to cromwell:test - DRYed out more of the CI scripts into test.inc.sh - Switched CI broadinstitute/dsde-toolbox from :latest to :dev - Don't fail with false negatives when MetadataDatabaseAccessSpec is re-run against persistent RDBMS --- .travis.yml | 93 +++- core/src/test/resources/application.conf | 59 ++- .../CallCachingSlickDatabaseSpec.scala | 2 + .../workflowstore/SqlWorkflowStoreSpec.scala | 2 + project/ContinuousIntegration.scala | 2 +- .../docker-compose-horicromtal.yml | 60 ++- .../cromwell/services/ServicesStoreSpec.scala | 2 + .../keyvalue/impl/KeyValueDatabaseSpec.scala | 5 + .../impl/MetadataDatabaseAccessSpec.scala | 41 +- src/ci/bin/test.inc.sh | 416 +++++++++++++----- src/ci/bin/testCentaurEngineUpgradeLocal.sh | 15 +- ...stCentaurHoricromtalEngineUpgradePapiV2.sh | 77 ++-- src/ci/bin/testCentaurHoricromtalPapiV2.sh | 1 - ...estCentaurPapiUpgradeNewWorkflowsPapiV1.sh | 19 +- src/ci/bin/testCentaurPapiUpgradePapiV1.sh | 17 +- .../cromwell-test/docker-setup.sh | 3 +- src/ci/docker-compose/docker-compose.yml | 26 +- src/ci/docker-compose/mariadb-conf.d/init.cnf | 3 + src/ci/docker-compose/mysql-conf.d/init.cnf | 3 + .../postgresql-initdb.d/init.sh | 5 + .../postgresql-initdb.d/init.sql | 1 + src/ci/resources/build_application.inc.conf | 2 +- src/ci/resources/cromwell_database.inc.conf | 21 +- 23 files changed, 635 insertions(+), 240 deletions(-) create mode 100644 src/ci/docker-compose/mariadb-conf.d/init.cnf create mode 100644 src/ci/docker-compose/mysql-conf.d/init.cnf create mode 100755 src/ci/docker-compose/postgresql-initdb.d/init.sh create mode 100644 src/ci/docker-compose/postgresql-initdb.d/init.sql diff --git a/.travis.yml b/.travis.yml index 4a40d3e5af3..724c2eba532 100644 --- a/.travis.yml +++ b/.travis.yml @@ -2,7 +2,6 @@ sudo: required dist: trusty services: - docker - - mysql language: scala scala: - 2.12.6 @@ -22,27 +21,77 @@ before_cache: env: matrix: # Setting this variable twice will cause the 'script' section to run twice with the respective env var invoked - - BUILD_TYPE=centaurAws - - BUILD_TYPE=centaurBcs - - BUILD_TYPE=centaurEngineUpgradeLocal - - BUILD_TYPE=centaurEngineUpgradePapiV2 - - BUILD_TYPE=centaurHoricromtalPapiV2 - - BUILD_TYPE=centaurHoricromtalEngineUpgradePapiV2 - - BUILD_TYPE=centaurPapiUpgradePapiV1 - - BUILD_TYPE=centaurPapiUpgradeNewWorkflowsPapiV1 - - BUILD_TYPE=centaurLocal - - BUILD_TYPE=centaurPapiV1 - - BUILD_TYPE=centaurPapiV2 - - BUILD_TYPE=centaurSlurm - - BUILD_TYPE=centaurTes - - BUILD_TYPE=centaurWdlUpgradeLocal - - BUILD_TYPE=checkPublish - - BUILD_TYPE=conformanceLocal - - BUILD_TYPE=conformancePapiV2 - - BUILD_TYPE=conformanceTesk - - BUILD_TYPE=dockerDeadlock - - BUILD_TYPE=dockerScripts - - BUILD_TYPE=sbt + - >- + BUILD_TYPE=centaurAws + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurBcs + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurEngineUpgradeLocal + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurEngineUpgradePapiV2 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurHoricromtalPapiV2 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurHoricromtalPapiV2 + BUILD_MARIADB=10.3 + - >- + BUILD_TYPE=centaurHoricromtalEngineUpgradePapiV2 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurHoricromtalEngineUpgradePapiV2 + BUILD_MARIADB=10.3 + - >- + BUILD_TYPE=centaurPapiUpgradePapiV1 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurPapiUpgradeNewWorkflowsPapiV1 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurLocal + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurPapiV1 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurPapiV2 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurSlurm + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurTes + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurWdlUpgradeLocal + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=checkPublish + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=conformanceLocal + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=conformancePapiV2 + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=conformanceTesk + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=dockerDeadlock + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=dockerScripts + BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=sbt + BUILD_MYSQL=5.7 + BUILD_POSTGRESQL=11.3 + BUILD_MARIADB=10.3 script: - src/ci/bin/test.sh notifications: diff --git a/core/src/test/resources/application.conf b/core/src/test/resources/application.conf index d6d14d3c9b7..6d479c92e23 100644 --- a/core/src/test/resources/application.conf +++ b/core/src/test/resources/application.conf @@ -21,21 +21,60 @@ database.db.connectionTimeout = 30000 database-test-mysql { # Run the following to (optionally) drop and (re-)create the database: - # mysql -utravis -e "DROP DATABASE IF EXISTS cromwell_test" && mysql -utravis -e "CREATE DATABASE cromwell_test" + # mysql -ucromwell -ptest -e "DROP DATABASE IF EXISTS cromwell_test; CREATE DATABASE cromwell_test;" profile = "slick.jdbc.MySQLProfile$" db { - hostname = localhost - hostname = ${?CROMWELL_BUILD_MYSQL_HOSTNAME} - port = 3306 - port = ${?CROMWELL_BUILD_MYSQL_PORT} - schema = cromwell_test - schema = ${?CROMWELL_BUILD_MYSQL_SCHEMA} - url = "jdbc:mysql://"${database-test-mysql.db.hostname}":"${database-test-mysql.db.port}"/"${database-test-mysql.db.schema}"?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" - user = "travis" + driver = "com.mysql.cj.jdbc.Driver" + url = "jdbc:mysql://localhost:3306/cromwell_test?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" + url = ${?CROMWELL_BUILD_MYSQL_JDBC_URL} + user = "cromwell" user = ${?CROMWELL_BUILD_MYSQL_USERNAME} - password = "" + password = "test" password = ${?CROMWELL_BUILD_MYSQL_PASSWORD} + connectionTimeout = 5000 + } +} + +database-test-mariadb { + # Installing both mysql and mariadb takes skill... Instead, try running this docker from the cromwell directory: + # + # docker run \ + # --rm \ + # --env MYSQL_ROOT_PASSWORD=private \ + # --env MYSQL_USER=cromwell \ + # --env MYSQL_PASSWORD=test \ + # --env MYSQL_DATABASE=cromwell_test \ + # --publish 13306:3306 \ + # --volume ${PWD}/src/ci/docker-compose/mariadb-conf.d:/etc/mysql/conf.d \ + # mariadb:10.3 + + # Run the following to (optionally) drop and (re-)create the database: + # mysql --protocol=tcp -P13306 -ucromwell -ptest -e "DROP DATABASE IF EXISTS cromwell_test; CREATE DATABASE cromwell_test;" + profile = "slick.jdbc.MySQLProfile$" + db { driver = "com.mysql.cj.jdbc.Driver" + url = "jdbc:mysql://localhost:13306/cromwell_test?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" + url = ${?CROMWELL_BUILD_MARIADB_JDBC_URL} + user = "cromwell" + user = ${?CROMWELL_BUILD_MARIADB_USERNAME} + password = "test" + password = ${?CROMWELL_BUILD_MARIADB_PASSWORD} + connectionTimeout = 5000 + } +} + +database-test-postgres { + # Run the following to (optionally) drop and (re-)create the database: + # psql postgres <<< 'drop database if exists cromwell_test; create database cromwell_test;' + profile = "slick.jdbc.PostgresProfile$" + db { + driver = "org.postgresql.Driver" + url = "jdbc:postgresql://localhost:5432/cromwell_test" + url = ${?CROMWELL_BUILD_POSTGRES_JDBC_URL} + user = "cromwell" + user = ${?CROMWELL_BUILD_POSTGRES_USERNAME} + password = "test" + password = ${?CROMWELL_BUILD_POSTGRES_PASSWORD} connectionTimeout = 5000 } } diff --git a/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala b/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala index e6ddda07103..b28c250490b 100644 --- a/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala +++ b/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala @@ -25,6 +25,8 @@ class CallCachingSlickDatabaseSpec extends FlatSpec with Matchers with ScalaFutu "SlickDatabase (mysql)" should behave like testWith("database-test-mysql") + "SlickDatabase (mariadb)" should behave like testWith("database-test-mariadb") + def testWith(configPath: String): Unit = { lazy val databaseConfig = ConfigFactory.load.getConfig(configPath) lazy val dataAccess = new EngineSlickDatabase(databaseConfig) diff --git a/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala b/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala index 69a3b901f30..aab401c9934 100644 --- a/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala +++ b/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala @@ -29,6 +29,8 @@ class SqlWorkflowStoreSpec extends FlatSpec with Matchers with ScalaFutures with "SqlWorkflowStore (mysql)" should behave like testWith("database-test-mysql") + "SqlWorkflowStore (mariadb)" should behave like testWith("database-test-mariadb") + def testWith(configPath: String): Unit = { lazy val databaseConfig = ConfigFactory.load.getConfig(configPath) diff --git a/project/ContinuousIntegration.scala b/project/ContinuousIntegration.scala index b4cff3af85f..ec6a3107a42 100644 --- a/project/ContinuousIntegration.scala +++ b/project/ContinuousIntegration.scala @@ -41,7 +41,7 @@ object ContinuousIntegration { "-e", "ENVIRONMENT=not_used", "-e", s"INPUT_PATH=${srcCiResources.value}", "-e", s"OUT_PATH=${targetCiResources.value}", - "broadinstitute/dsde-toolbox", "render-templates.sh" + "broadinstitute/dsde-toolbox:dev", "render-templates.sh" ) val result = cmd ! log if (result != 0) { diff --git a/scripts/docker-compose-mysql/docker-compose-horicromtal.yml b/scripts/docker-compose-mysql/docker-compose-horicromtal.yml index 5faf0bc5bf8..f6458ecc6c2 100644 --- a/scripts/docker-compose-mysql/docker-compose-horicromtal.yml +++ b/scripts/docker-compose-mysql/docker-compose-horicromtal.yml @@ -13,6 +13,33 @@ # into the CI config files. version: '2.3' services: + # Runs the database initialization but is NOT a workflow-running backend. + cromwell_database_master: + image: "broadinstitute/cromwell:${CROMWELL_TAG}" + network_mode: host + working_dir: /cromwell-working-dir + volumes: + - ${CROMWELL_BUILD_ROOT_DIRECTORY}:${CROMWELL_BUILD_ROOT_DIRECTORY} + command: ["server"] + environment: + - >- + JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} + -Dwebservice.port=8080 + -Dsystem.cromwell_id=master + -Dsystem.max-workflow-launch-count=0 + -Dsystem.new-workflow-poll-rate=999999 + -Dservices.MetadataService.config.metadata-summary-refresh-interval=Inf + - CROMWELL_BUILD_RESOURCES_DIRECTORY + - CROMWELL_BUILD_CENTAUR_SLICK_PROFILE + - CROMWELL_BUILD_CENTAUR_JDBC_DRIVER + - CROMWELL_BUILD_CENTAUR_JDBC_USERNAME + - CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD + - CROMWELL_BUILD_CENTAUR_JDBC_URL + healthcheck: + test: ["CMD", "curl", "-f", "http://localhost:8080"] + interval: 2s + timeout: 30s + retries: 60 # Runs the summarizer and is also a regular Cromwell workflow-running backend. cromwell_summarizer_plus_backend: image: "broadinstitute/cromwell:${CROMWELL_TAG}" @@ -22,9 +49,19 @@ services: - ${CROMWELL_BUILD_ROOT_DIRECTORY}:${CROMWELL_BUILD_ROOT_DIRECTORY} command: ["server"] environment: - - JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} -Dwebservice.port=8000 -Dsystem.cromwell_id=summarizer - - CROMWELL_BUILD_RESOURCES_DIRECTORY=${CROMWELL_BUILD_ROOT_DIRECTORY}/target/ci/resources - - CROMWELL_BUILD_MYSQL_USERNAME=${CROMWELL_BUILD_MYSQL_USERNAME} + - >- + JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} + -Dwebservice.port=8000 + -Dsystem.cromwell_id=summarizer + - CROMWELL_BUILD_RESOURCES_DIRECTORY + - CROMWELL_BUILD_CENTAUR_SLICK_PROFILE + - CROMWELL_BUILD_CENTAUR_JDBC_DRIVER + - CROMWELL_BUILD_CENTAUR_JDBC_USERNAME + - CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD + - CROMWELL_BUILD_CENTAUR_JDBC_URL + depends_on: + cromwell_database_master: + condition: service_healthy healthcheck: test: ["CMD", "curl", "-f", "http://localhost:8000"] interval: 2s @@ -37,11 +74,22 @@ services: working_dir: /cromwell-working-dir volumes: - ${CROMWELL_BUILD_ROOT_DIRECTORY}:${CROMWELL_BUILD_ROOT_DIRECTORY} + depends_on: + cromwell_database_master: + condition: service_healthy command: ["server"] environment: - - JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} -Dwebservice.port=${MANAGED_CROMWELL_PORT-8008} -Dsystem.cromwell_id=frontend - - CROMWELL_BUILD_RESOURCES_DIRECTORY=${CROMWELL_BUILD_ROOT_DIRECTORY}/target/ci/resources - - CROMWELL_BUILD_MYSQL_USERNAME=${CROMWELL_BUILD_MYSQL_USERNAME} + - >- + JAVA_OPTS=-Dconfig.file=${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_CONFIG} + -Dwebservice.port=${MANAGED_CROMWELL_PORT-8008} + -Dsystem.cromwell_id=frontend + -Dservices.MetadataService.config.metadata-summary-refresh-interval=Inf + - CROMWELL_BUILD_RESOURCES_DIRECTORY + - CROMWELL_BUILD_CENTAUR_SLICK_PROFILE + - CROMWELL_BUILD_CENTAUR_JDBC_DRIVER + - CROMWELL_BUILD_CENTAUR_JDBC_USERNAME + - CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD + - CROMWELL_BUILD_CENTAUR_JDBC_URL healthcheck: test: ["CMD", "curl", "-f", "http://localhost:8008"] interval: 2s diff --git a/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala b/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala index 119114ccc84..275398a6609 100644 --- a/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala +++ b/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala @@ -251,6 +251,8 @@ class ServicesStoreSpec extends FlatSpec with Matchers with ScalaFutures with St "SlickDatabase (mysql)" should behave like testWith("database-test-mysql") + "SlickDatabase (mariadb)" should behave like testWith("database-test-mariadb") + def testWith(configPath: String): Unit = { import ServicesStore.EnhancedSqlDatabase diff --git a/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala b/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala index d6c0d591f42..95749caddd7 100644 --- a/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala +++ b/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala @@ -32,6 +32,11 @@ class KeyValueDatabaseSpec extends FlatSpec with Matchers with ScalaFutures with "Column 'STORE_VALUE' cannot be null" ) + "SlickDatabase (mariadb)" should behave like testWith[BatchUpdateException]( + "database-test-mariadb", + "Column 'STORE_VALUE' cannot be null" + ) + def testWith[E <: Throwable](configPath: String, failureMessage: String)(implicit classTag: ClassTag[E]): Unit = { lazy val databaseConfig = ConfigFactory.load.getConfig(configPath) lazy val dataAccess = new EngineSlickDatabase(databaseConfig) diff --git a/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala b/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala index de476a1eee7..1e74eb0bb49 100644 --- a/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala +++ b/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala @@ -39,6 +39,8 @@ class MetadataDatabaseAccessSpec extends FlatSpec with Matchers with ScalaFuture "MetadataDatabaseAccess (mysql)" should behave like testWith("database-test-mysql") + "MetadataDatabaseAccess (mariadb)" should behave like testWith("database-test-mariadb") + implicit val ec = ExecutionContext.global implicit val defaultPatience = PatienceConfig(scaled(Span(30, Seconds)), scaled(Span(100, Millis))) @@ -117,6 +119,12 @@ class MetadataDatabaseAccessSpec extends FlatSpec with Matchers with ScalaFuture } it should "sort metadata events by timestamp from older to newer" taggedAs DbmsTest in { + + if (configPath == "database-test-mariadb") { + // Do NOT want to change the test. Instead should fix it so that MariaDB is storing at least milliseconds. + cancel("Having issues with MariaDB and fractional seconds. https://broadworkbench.atlassian.net/browse/BA-5692") + } + def unorderedEvents(id: WorkflowId): Future[Vector[MetadataEvent]] = { val workflowKey = MetadataKey(id, jobKey = None, key = null) val now = OffsetDateTime.now() @@ -273,7 +281,9 @@ class MetadataDatabaseAccessSpec extends FlatSpec with Matchers with ScalaFuture _ <- dataAccess.queryWorkflowSummaries(WorkflowQueryParameters(Seq( WorkflowQueryKey.ExcludeLabelAndKeyValue.name -> s"${testLabel2.key}:${testLabel2.value}"))) map { case (response, _) => val resultByName = response.results groupBy (_.name) - withClue("Filter by exclude label using AND") { resultByName.keys.toSet.flatten should equal(Set(Workflow1Name)) } + withClue("Filter by exclude label using AND") { + resultByName.keys.toSet.flatten should contain(Workflow1Name) + } } // Filter by multiple exclude labels using AND _ <- dataAccess.queryWorkflowSummaries(WorkflowQueryParameters( @@ -281,30 +291,31 @@ class MetadataDatabaseAccessSpec extends FlatSpec with Matchers with ScalaFuture .map(label => WorkflowQueryKey.ExcludeLabelAndKeyValue.name -> s"${label.key}:${label.value}")) ) map { case (response, _) => val resultByName = response.results groupBy (_.name) - withClue("Filter by multiple exclude labels using AND") { resultByName.keys.toSet.flatten should equal(Set(Workflow1Name)) } - response.totalResultsCount match { - case 3 => //good - case ct => fail(s"totalResultsCount for multiple exclude labels using AND query is expected to be 3 but is actually $ct. " + - s"Something has gone horribly wrong!") + val ids = response.results.map(_.id) + withClue("Filter by multiple exclude labels using AND") { + resultByName.keys.toSet.flatten should contain(Workflow1Name) + ids should contain(workflow1Id.toString) + ids shouldNot contain(workflow2Id.toString) } } // Filter by exclude label using OR _ <- dataAccess.queryWorkflowSummaries(WorkflowQueryParameters(Seq( WorkflowQueryKey.ExcludeLabelOrKeyValue.name -> s"${testLabel2.key}:${testLabel2.value}"))) map { case (response, _) => val resultByName = response.results groupBy (_.name) - withClue("Filter to exclude label using OR") { resultByName.keys.toSet.flatten should equal(Set(Workflow1Name)) } + withClue("Filter to exclude label using OR") { + resultByName.keys.toSet.flatten should contain(Workflow1Name) + } } // Filter by multiple exclude labels using OR _ <- dataAccess.queryWorkflowSummaries(WorkflowQueryParameters( Seq(testLabel2, testLabel3) .map(label => WorkflowQueryKey.ExcludeLabelOrKeyValue.name -> s"${label.key}:${label.value}")) ) map { case (response, _) => - val resultByName = response.results groupBy (_.name) - withClue("Filter by multiple exclude labels using OR") { resultByName.keys.toSet.flatten should equal(Set(Workflow1Name)) } - response.totalResultsCount match { - case 2 => //good - case ct => fail(s"totalResultsCount is for multiple exclude labels using OR query is expected to be 2 but is actually $ct. " + - s"Something has gone horribly wrong!") + // NOTE: On persistent databases other workflows will be returned. Just verify that our two workflows are not. + val ids = response.results.map(_.id) + withClue("Filter by multiple exclude labels using OR") { + ids shouldNot contain(workflow1Id.toString) + ids shouldNot contain(workflow2Id.toString) } } // Filter by start date @@ -345,7 +356,9 @@ class MetadataDatabaseAccessSpec extends FlatSpec with Matchers with ScalaFuture _ <- dataAccess.queryWorkflowSummaries(WorkflowQueryParameters(Seq( WorkflowQueryKey.AdditionalQueryResultFields.name -> "labels"))) map { case (response, _) => - response.results partition { r => r.labels.isDefined} match { + response.results filter { + workflowQueryResult => List(workflow1Id.toString, workflow1Id.toString).contains(workflowQueryResult.id) + } partition { r => r.labels.isDefined } match { case (y, n) if y.nonEmpty && n.isEmpty => //good case (y, n) => fail(s"Something went horribly wrong since labels were populated for ${y.size} and were missing for ${n.size} workflow(s)!") } diff --git a/src/ci/bin/test.inc.sh b/src/ci/bin/test.inc.sh index d3ee08d39ea..9749082c628 100644 --- a/src/ci/bin/test.inc.sh +++ b/src/ci/bin/test.inc.sh @@ -63,6 +63,7 @@ cromwell::private::create_build_variables() { CROMWELL_BUILD_LOG_DIRECTORY="${CROMWELL_BUILD_ROOT_DIRECTORY}/target/ci/logs" CROMWELL_BUILD_CROMWELL_LOG="${CROMWELL_BUILD_LOG_DIRECTORY}/cromwell.log" + CROMWELL_BUILD_DOCKER_DIRECTORY="${CROMWELL_BUILD_ROOT_DIRECTORY}/src/ci/docker-compose" CROMWELL_BUILD_SCRIPTS_DIRECTORY="${CROMWELL_BUILD_ROOT_DIRECTORY}/src/ci/bin" CROMWELL_BUILD_RESOURCES_SOURCES="${CROMWELL_BUILD_ROOT_DIRECTORY}/src/ci/resources" CROMWELL_BUILD_RESOURCES_DIRECTORY="${CROMWELL_BUILD_ROOT_DIRECTORY}/target/ci/resources" @@ -79,12 +80,17 @@ cromwell::private::create_build_variables() { CROMWELL_BUILD_IS_VIRTUAL_ENV=false fi - CROMWELL_BUILD_RUN_TESTS=true - case "${CROMWELL_BUILD_PROVIDER}" in "${CROMWELL_BUILD_PROVIDER_TRAVIS}") CROMWELL_BUILD_IS_CI=true CROMWELL_BUILD_IS_SECURE="${TRAVIS_SECURE_ENV_VARS}" + + if [[ -n "${TRAVIS_PULL_REQUEST_BRANCH:+set}" ]]; then + CROMWELL_BUILD_IS_PULL_REQUEST=true + else + CROMWELL_BUILD_IS_PULL_REQUEST=false + fi + CROMWELL_BUILD_TYPE="${BUILD_TYPE}" CROMWELL_BUILD_BRANCH="${TRAVIS_PULL_REQUEST_BRANCH:-${TRAVIS_BRANCH}}" CROMWELL_BUILD_BRANCH_PULL_REQUEST="${TRAVIS_PULL_REQUEST_BRANCH:-""}" @@ -95,27 +101,24 @@ cromwell::private::create_build_variables() { CROMWELL_BUILD_GIT_USER_EMAIL="travis@travis-ci.com" CROMWELL_BUILD_GIT_USER_NAME="Travis CI" CROMWELL_BUILD_HEARTBEAT_PATTERN="…" - CROMWELL_BUILD_MYSQL_HOSTNAME="localhost" - CROMWELL_BUILD_MYSQL_PORT="3306" - CROMWELL_BUILD_MYSQL_USERNAME="travis" - CROMWELL_BUILD_MYSQL_PASSWORD="" - CROMWELL_BUILD_MYSQL_SCHEMA="cromwell_test" CROMWELL_BUILD_GENERATE_COVERAGE=true # Always run on sbt, even for 'push'. # This allows quick sanity checks before starting PRs *and* publishing after merges into develop. - if [[ "${CROMWELL_BUILD_TYPE}" == "sbt" ]]; then - CROMWELL_BUILD_RUN_TESTS=true - elif [[ "${TRAVIS_COMMIT_MESSAGE}" != *"[force ci]"* ]] && [[ "${TRAVIS_EVENT_TYPE}" == "push" ]]; then + if [[ "${TRAVIS_EVENT_TYPE}" == "push" ]] && \ + [[ "${BUILD_TYPE}" != "sbt" ]] && \ + [[ "${TRAVIS_COMMIT_MESSAGE}" != *"[force ci]"* ]]; then CROMWELL_BUILD_RUN_TESTS=false + else + CROMWELL_BUILD_RUN_TESTS=true fi ;; "${CROMWELL_BUILD_PROVIDER_JENKINS}") # External variables must be passed through in the ENVIRONMENT of src/ci/docker-compose/docker-compose.yml CROMWELL_BUILD_IS_CI=true CROMWELL_BUILD_IS_SECURE=true + CROMWELL_BUILD_IS_PULL_REQUEST=false CROMWELL_BUILD_TYPE="${JENKINS_BUILD_TYPE}" - CROMWELL_BUILD_CENTAUR_TEST_ADDITIONAL_PARAMETERS="${CENTAUR_TEST_ADDITIONAL_PARAMETERS:-""}" CROMWELL_BUILD_BRANCH="${GIT_BRANCH#origin/}" CROMWELL_BUILD_BRANCH_PULL_REQUEST="" CROMWELL_BUILD_EVENT="" @@ -125,16 +128,13 @@ cromwell::private::create_build_variables() { CROMWELL_BUILD_GIT_USER_EMAIL="jenkins@jenkins.io" CROMWELL_BUILD_GIT_USER_NAME="Jenkins CI" CROMWELL_BUILD_HEARTBEAT_PATTERN="…\n" - CROMWELL_BUILD_MYSQL_HOSTNAME="mysql-db" - CROMWELL_BUILD_MYSQL_PORT="3306" - CROMWELL_BUILD_MYSQL_USERNAME="root" - CROMWELL_BUILD_MYSQL_PASSWORD="" - CROMWELL_BUILD_MYSQL_SCHEMA="cromwell_test" CROMWELL_BUILD_GENERATE_COVERAGE=false + CROMWELL_BUILD_RUN_TESTS=true ;; *) CROMWELL_BUILD_IS_CI=false CROMWELL_BUILD_IS_SECURE=true + CROMWELL_BUILD_IS_PULL_REQUEST=false CROMWELL_BUILD_TYPE="unknown" CROMWELL_BUILD_BRANCH="unknown" CROMWELL_BUILD_BRANCH_PULL_REQUEST="" @@ -145,12 +145,8 @@ cromwell::private::create_build_variables() { CROMWELL_BUILD_GIT_USER_EMAIL="unknown.git.user@example.org" CROMWELL_BUILD_GIT_USER_NAME="Unknown Git User" CROMWELL_BUILD_HEARTBEAT_PATTERN="…" - CROMWELL_BUILD_MYSQL_HOSTNAME="${CROMWELL_BUILD_MYSQL_HOSTNAME-localhost}" - CROMWELL_BUILD_MYSQL_PORT="${CROMWELL_BUILD_MYSQL_PORT-3306}" - CROMWELL_BUILD_MYSQL_USERNAME="${CROMWELL_BUILD_MYSQL_USERNAME-root}" - CROMWELL_BUILD_MYSQL_PASSWORD="${CROMWELL_BUILD_MYSQL_PASSWORD-}" - CROMWELL_BUILD_MYSQL_SCHEMA="${CROMWELL_BUILD_MYSQL_SCHEMA-cromwell_test}" CROMWELL_BUILD_GENERATE_COVERAGE=true + CROMWELL_BUILD_RUN_TESTS=true local bash_script for bash_script in "${BASH_SOURCE[@]}"; do @@ -178,25 +174,24 @@ cromwell::private::create_build_variables() { backend_type="$(echo "${backend_type}" | sed 's/\([A-Z]\)/_\1/g' | tr '[:upper:]' '[:lower:]' | cut -c 2-)" CROMWELL_BUILD_BACKEND_TYPE="${backend_type}" - CROMWELL_BUILD_CROMWELL_CONFIG="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_BUILD_BACKEND_TYPE}_application.conf" - - if [[ -z "${CROMWELL_BUILD_OPTIONAL_SECURE-}" ]]; then - CROMWELL_BUILD_OPTIONAL_SECURE=false - fi - - if [[ -z "${CROMWELL_BUILD_REQUIRES_SECURE-}" ]]; then - CROMWELL_BUILD_REQUIRES_SECURE=false - fi - - if [[ -z "${CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND-}" ]]; then + if [[ "${CROMWELL_BUILD_TYPE}" == conformance* ]]; then + CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND="server/assembly centaurCwlRunner/assembly" + else CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND="assembly" fi - if [[ -z "${VAULT_TOKEN-}" ]]; then - VAULT_TOKEN="vault token is not set as an environment variable" + if [[ "${CROMWELL_BUILD_TYPE}" == centaurPapiUpgrade* ]] || \ + [[ "${CROMWELL_BUILD_TYPE}" == centaurHoricromtalEngineUpgrade* ]]; then + CROMWELL_BUILD_CROMWELL_CONFIG="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/papi_v1_v2_upgrade_application.conf" + else + CROMWELL_BUILD_CROMWELL_CONFIG="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/${CROMWELL_BUILD_BACKEND_TYPE}_application.conf" fi - CROMWELL_BUILD_RANDOM_256_BITS_BASE64="$(dd bs=1 count=32 if=/dev/urandom 2> /dev/null | base64 | tr -d '\n')" + CROMWELL_BUILD_OPTIONAL_SECURE="${CROMWELL_BUILD_OPTIONAL_SECURE-false}" + CROMWELL_BUILD_REQUIRES_SECURE="${CROMWELL_BUILD_REQUIRES_SECURE-false}" + CROMWELL_BUILD_REQUIRES_PULL_REQUEST="${CROMWELL_BUILD_REQUIRES_PULL_REQUEST-false}" + CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND="${CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND-assembly}" + VAULT_TOKEN="${VAULT_TOKEN-vault token is not set as an environment variable}" local hours_to_minutes hours_to_minutes=60 @@ -205,26 +200,22 @@ cromwell::private::create_build_variables() { export CROMWELL_BUILD_BACKEND_TYPE export CROMWELL_BUILD_BRANCH export CROMWELL_BUILD_BRANCH_PULL_REQUEST - export CROMWELL_BUILD_CENTAUR_TEST_ADDITIONAL_PARAMETERS export CROMWELL_BUILD_CROMWELL_CONFIG export CROMWELL_BUILD_CROMWELL_LOG + export CROMWELL_BUILD_DOCKER_DIRECTORY export CROMWELL_BUILD_EVENT export CROMWELL_BUILD_EXIT_FUNCTIONS export CROMWELL_BUILD_GENERATE_COVERAGE export CROMWELL_BUILD_GIT_USER_EMAIL export CROMWELL_BUILD_GIT_USER_NAME - export CROMWELL_BUILD_HEARTBEAT_PATTERN export CROMWELL_BUILD_HEARTBEAT_MINUTES + export CROMWELL_BUILD_HEARTBEAT_PATTERN export CROMWELL_BUILD_HOME_DIRECTORY export CROMWELL_BUILD_IS_CI + export CROMWELL_BUILD_IS_PULL_REQUEST export CROMWELL_BUILD_IS_SECURE export CROMWELL_BUILD_IS_VIRTUAL_ENV export CROMWELL_BUILD_LOG_DIRECTORY - export CROMWELL_BUILD_MYSQL_HOSTNAME - export CROMWELL_BUILD_MYSQL_PASSWORD - export CROMWELL_BUILD_MYSQL_PORT - export CROMWELL_BUILD_MYSQL_SCHEMA - export CROMWELL_BUILD_MYSQL_USERNAME export CROMWELL_BUILD_NUMBER export CROMWELL_BUILD_OPTIONAL_SECURE export CROMWELL_BUILD_OS @@ -234,11 +225,11 @@ cromwell::private::create_build_variables() { export CROMWELL_BUILD_PROVIDER_JENKINS export CROMWELL_BUILD_PROVIDER_TRAVIS export CROMWELL_BUILD_PROVIDER_UNKNOWN - export CROMWELL_BUILD_RANDOM_256_BITS_BASE64 export CROMWELL_BUILD_REQUIRES_SECURE export CROMWELL_BUILD_RESOURCES_DIRECTORY export CROMWELL_BUILD_RESOURCES_SOURCES export CROMWELL_BUILD_ROOT_DIRECTORY + export CROMWELL_BUILD_RUN_TESTS export CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND export CROMWELL_BUILD_SCRIPTS_DIRECTORY export CROMWELL_BUILD_TAG @@ -266,6 +257,100 @@ cromwell::private::echo_build_variables() { echo "CROMWELL_BUILD_URL='${CROMWELL_BUILD_URL}'" } +cromwell::private::create_database_variables() { + case "${CROMWELL_BUILD_PROVIDER}" in + "${CROMWELL_BUILD_PROVIDER_TRAVIS}") + CROMWELL_BUILD_MARIADB_HOSTNAME="localhost" + CROMWELL_BUILD_MARIADB_PORT="13306" + CROMWELL_BUILD_MARIADB_USERNAME="cromwell" + CROMWELL_BUILD_MARIADB_PASSWORD="test" + CROMWELL_BUILD_MARIADB_SCHEMA="cromwell_test" + CROMWELL_BUILD_MARIADB_DOCKER_TAG="${BUILD_MARIADB-}" + CROMWELL_BUILD_MYSQL_HOSTNAME="localhost" + CROMWELL_BUILD_MYSQL_PORT="3306" + CROMWELL_BUILD_MYSQL_USERNAME="cromwell" + CROMWELL_BUILD_MYSQL_PASSWORD="test" + CROMWELL_BUILD_MYSQL_SCHEMA="cromwell_test" + CROMWELL_BUILD_MYSQL_DOCKER_TAG="${BUILD_MYSQL-}" + CROMWELL_BUILD_POSTGRESQL_HOSTNAME="localhost" + CROMWELL_BUILD_POSTGRESQL_PORT="5432" + CROMWELL_BUILD_POSTGRESQL_USERNAME="cromwell" + CROMWELL_BUILD_POSTGRESQL_PASSWORD="test" + CROMWELL_BUILD_POSTGRESQL_SCHEMA="cromwell_test" + CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG="${BUILD_POSTGRESQL-}" + ;; + "${CROMWELL_BUILD_PROVIDER_JENKINS}") + # NOTE: Jenkins uses src/ci/docker-compose/docker-compose.yml. + # We don't define a docker tag because the docker-compose has already spun up the database containers by the + # time this script is run. Other variables here must match the database service names and settings the yaml. + CROMWELL_BUILD_MARIADB_DOCKER_TAG="" + CROMWELL_BUILD_MARIADB_HOSTNAME="mariadb-db" + CROMWELL_BUILD_MARIADB_PORT="3306" + CROMWELL_BUILD_MARIADB_USERNAME="cromwell" + CROMWELL_BUILD_MARIADB_PASSWORD="test" + CROMWELL_BUILD_MARIADB_SCHEMA="cromwell_test" + CROMWELL_BUILD_MYSQL_DOCKER_TAG="" + CROMWELL_BUILD_MYSQL_HOSTNAME="mysql-db" + CROMWELL_BUILD_MYSQL_PORT="3306" + CROMWELL_BUILD_MYSQL_USERNAME="cromwell" + CROMWELL_BUILD_MYSQL_PASSWORD="test" + CROMWELL_BUILD_MYSQL_SCHEMA="cromwell_test" + CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG="" + CROMWELL_BUILD_POSTGRESQL_HOSTNAME="postgresql-db" + CROMWELL_BUILD_POSTGRESQL_PORT="5432" + CROMWELL_BUILD_POSTGRESQL_USERNAME="cromwell" + CROMWELL_BUILD_POSTGRESQL_PASSWORD="test" + CROMWELL_BUILD_POSTGRESQL_SCHEMA="cromwell_test" + ;; + *) + CROMWELL_BUILD_MARIADB_HOSTNAME="${CROMWELL_BUILD_MARIADB_HOSTNAME-localhost}" + CROMWELL_BUILD_MARIADB_PORT="${CROMWELL_BUILD_MARIADB_PORT-13306}" + CROMWELL_BUILD_MARIADB_USERNAME="${CROMWELL_BUILD_MARIADB_USERNAME-cromwell}" + CROMWELL_BUILD_MARIADB_PASSWORD="${CROMWELL_BUILD_MARIADB_PASSWORD-test}" + CROMWELL_BUILD_MARIADB_SCHEMA="${CROMWELL_BUILD_MARIADB_SCHEMA-cromwell_test}" + CROMWELL_BUILD_MARIADB_DOCKER_TAG="" + CROMWELL_BUILD_MYSQL_HOSTNAME="${CROMWELL_BUILD_MYSQL_HOSTNAME-localhost}" + CROMWELL_BUILD_MYSQL_PORT="${CROMWELL_BUILD_MYSQL_PORT-3306}" + CROMWELL_BUILD_MYSQL_USERNAME="${CROMWELL_BUILD_MYSQL_USERNAME-cromwell}" + CROMWELL_BUILD_MYSQL_PASSWORD="${CROMWELL_BUILD_MYSQL_PASSWORD-test}" + CROMWELL_BUILD_MYSQL_SCHEMA="${CROMWELL_BUILD_MYSQL_SCHEMA-cromwell_test}" + CROMWELL_BUILD_MYSQL_DOCKER_TAG="" + CROMWELL_BUILD_POSTGRESQL_HOSTNAME="${CROMWELL_BUILD_POSTGRESQL_HOSTNAME-localhost}" + CROMWELL_BUILD_POSTGRESQL_PORT="${CROMWELL_BUILD_POSTGRESQL_PORT-5432}" + CROMWELL_BUILD_POSTGRESQL_USERNAME="${CROMWELL_BUILD_POSTGRESQL_USERNAME-cromwell}" + CROMWELL_BUILD_POSTGRESQL_PASSWORD="${CROMWELL_BUILD_POSTGRESQL_PASSWORD-test}" + CROMWELL_BUILD_POSTGRESQL_SCHEMA="${CROMWELL_BUILD_POSTGRESQL_SCHEMA-cromwell_test}" + CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG="" + ;; + esac + + CROMWELL_BUILD_MARIADB_JDBC_URL="jdbc:mysql://${CROMWELL_BUILD_MARIADB_HOSTNAME}:${CROMWELL_BUILD_MARIADB_PORT}/${CROMWELL_BUILD_MARIADB_SCHEMA}?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" + CROMWELL_BUILD_MYSQL_JDBC_URL="jdbc:mysql://${CROMWELL_BUILD_MYSQL_HOSTNAME}:${CROMWELL_BUILD_MYSQL_PORT}/${CROMWELL_BUILD_MYSQL_SCHEMA}?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" + CROMWELL_BUILD_POSTGRESQL_JDBC_URL="jdbc:postgresql://${CROMWELL_BUILD_POSTGRESQL_HOSTNAME}:${CROMWELL_BUILD_POSTGRESQL_PORT}/${CROMWELL_BUILD_POSTGRESQL_SCHEMA}" + + export CROMWELL_BUILD_MARIADB_DOCKER_TAG + export CROMWELL_BUILD_MARIADB_HOSTNAME + export CROMWELL_BUILD_MARIADB_JDBC_URL + export CROMWELL_BUILD_MARIADB_PASSWORD + export CROMWELL_BUILD_MARIADB_PORT + export CROMWELL_BUILD_MARIADB_SCHEMA + export CROMWELL_BUILD_MARIADB_USERNAME + export CROMWELL_BUILD_MYSQL_DOCKER_TAG + export CROMWELL_BUILD_MYSQL_HOSTNAME + export CROMWELL_BUILD_MYSQL_JDBC_URL + export CROMWELL_BUILD_MYSQL_PASSWORD + export CROMWELL_BUILD_MYSQL_PORT + export CROMWELL_BUILD_MYSQL_SCHEMA + export CROMWELL_BUILD_MYSQL_USERNAME + export CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG + export CROMWELL_BUILD_POSTGRESQL_HOSTNAME + export CROMWELL_BUILD_POSTGRESQL_JDBC_URL + export CROMWELL_BUILD_POSTGRESQL_PASSWORD + export CROMWELL_BUILD_POSTGRESQL_PORT + export CROMWELL_BUILD_POSTGRESQL_SCHEMA + export CROMWELL_BUILD_POSTGRESQL_USERNAME +} + cromwell::private::create_centaur_variables() { CROMWELL_BUILD_CENTAUR_TYPE_STANDARD="standard" CROMWELL_BUILD_CENTAUR_TYPE_INTEGRATION="integration" @@ -310,6 +395,57 @@ cromwell::private::create_centaur_variables() { CROMWELL_BUILD_CENTAUR_TEST_RENDERED="${CROMWELL_BUILD_CENTAUR_TEST_DIRECTORY}/rendered" CROMWELL_BUILD_CENTAUR_LOG="${CROMWELL_BUILD_LOG_DIRECTORY}/centaur.log" + # Pick **one** of the databases to run Centaur against + case "${CROMWELL_BUILD_PROVIDER}" in + "${CROMWELL_BUILD_PROVIDER_TRAVIS}") + + if [[ -n "${CROMWELL_BUILD_MYSQL_DOCKER_TAG:+set}" ]]; then + CROMWELL_BUILD_CENTAUR_SLICK_PROFILE="slick.jdbc.MySQLProfile$" + CROMWELL_BUILD_CENTAUR_JDBC_DRIVER="com.mysql.cj.jdbc.Driver" + CROMWELL_BUILD_CENTAUR_JDBC_USERNAME="${CROMWELL_BUILD_MYSQL_USERNAME}" + CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD="${CROMWELL_BUILD_MYSQL_PASSWORD}" + CROMWELL_BUILD_CENTAUR_JDBC_URL="${CROMWELL_BUILD_MYSQL_JDBC_URL}" + + elif [[ -n "${CROMWELL_BUILD_MARIADB_DOCKER_TAG:+set}" ]]; then + CROMWELL_BUILD_CENTAUR_SLICK_PROFILE="slick.jdbc.MySQLProfile$" + CROMWELL_BUILD_CENTAUR_JDBC_DRIVER="com.mysql.cj.jdbc.Driver" + CROMWELL_BUILD_CENTAUR_JDBC_USERNAME="${CROMWELL_BUILD_MARIADB_USERNAME}" + CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD="${CROMWELL_BUILD_MARIADB_PASSWORD}" + CROMWELL_BUILD_CENTAUR_JDBC_URL="${CROMWELL_BUILD_MARIADB_JDBC_URL}" + + elif [[ -n "${CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG:+set}" ]]; then + CROMWELL_BUILD_CENTAUR_SLICK_PROFILE="slick.jdbc.PostgresProfile$" + CROMWELL_BUILD_CENTAUR_JDBC_DRIVER="org.postgresql.Driver" + CROMWELL_BUILD_CENTAUR_JDBC_USERNAME="${CROMWELL_BUILD_POSTGRESQL_USERNAME}" + CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD="${CROMWELL_BUILD_POSTGRESQL_PASSWORD}" + CROMWELL_BUILD_CENTAUR_JDBC_URL="${CROMWELL_BUILD_POSTGRESQL_JDBC_URL}" + + else + echo "Error: Unable to determine which RDBMS to use for Centaur." >&2 + exit 1 + + fi + + CROMWELL_BUILD_CENTAUR_TEST_ADDITIONAL_PARAMETERS= + ;; + "${CROMWELL_BUILD_PROVIDER_JENKINS}") + CROMWELL_BUILD_CENTAUR_SLICK_PROFILE="slick.jdbc.MySQLProfile$" + CROMWELL_BUILD_CENTAUR_JDBC_DRIVER="com.mysql.cj.jdbc.Driver" + CROMWELL_BUILD_CENTAUR_JDBC_USERNAME="${CROMWELL_BUILD_MYSQL_USERNAME}" + CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD="${CROMWELL_BUILD_MYSQL_PASSWORD}" + CROMWELL_BUILD_CENTAUR_JDBC_URL="${CROMWELL_BUILD_MYSQL_JDBC_URL}" + CROMWELL_BUILD_CENTAUR_TEST_ADDITIONAL_PARAMETERS="${CENTAUR_TEST_ADDITIONAL_PARAMETERS-}" + ;; + *) + CROMWELL_BUILD_CENTAUR_SLICK_PROFILE="${CROMWELL_BUILD_CENTAUR_SLICK_PROFILE-slick.jdbc.MySQLProfile\$}" + CROMWELL_BUILD_CENTAUR_JDBC_DRIVER="${CROMWELL_BUILD_CENTAUR_JDBC_DRIVER-com.mysql.cj.jdbc.Driver}" + CROMWELL_BUILD_CENTAUR_JDBC_USERNAME="${CROMWELL_BUILD_CENTAUR_JDBC_USERNAME-${CROMWELL_BUILD_MYSQL_USERNAME}}" + CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD="${CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD-${CROMWELL_BUILD_MYSQL_PASSWORD}}" + CROMWELL_BUILD_CENTAUR_JDBC_URL="${CROMWELL_BUILD_CENTAUR_JDBC_URL-${CROMWELL_BUILD_MYSQL_JDBC_URL}}" + CROMWELL_BUILD_CENTAUR_TEST_ADDITIONAL_PARAMETERS= + ;; + esac + case "${CROMWELL_BUILD_CENTAUR_TYPE}" in "${CROMWELL_BUILD_CENTAUR_TYPE_INTEGRATION}") CROMWELL_BUILD_CENTAUR_READ_LINES_LIMIT=512000 @@ -319,39 +455,29 @@ cromwell::private::create_centaur_variables() { ;; esac + CROMWELL_BUILD_CENTAUR_256_BITS_KEY="$(dd bs=1 count=32 if=/dev/urandom 2> /dev/null | base64 | tr -d '\n')" + + export CROMWELL_BUILD_CENTAUR_256_BITS_KEY export CROMWELL_BUILD_CENTAUR_CONFIG + export CROMWELL_BUILD_CENTAUR_JDBC_DRIVER + export CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD + export CROMWELL_BUILD_CENTAUR_JDBC_URL + export CROMWELL_BUILD_CENTAUR_JDBC_USERNAME export CROMWELL_BUILD_CENTAUR_LOG + export CROMWELL_BUILD_CENTAUR_TEST_ADDITIONAL_PARAMETERS export CROMWELL_BUILD_CENTAUR_TEST_DIRECTORY export CROMWELL_BUILD_CENTAUR_TEST_RENDERED export CROMWELL_BUILD_CENTAUR_READ_LINES_LIMIT export CROMWELL_BUILD_CENTAUR_RESOURCES + export CROMWELL_BUILD_CENTAUR_SLICK_PROFILE export CROMWELL_BUILD_CENTAUR_TYPE export CROMWELL_BUILD_CENTAUR_TYPE_STANDARD export CROMWELL_BUILD_CENTAUR_TYPE_INTEGRATION export CROMWELL_BUILD_CENTAUR_TYPE_ENGINE_UPGRADE } -cromwell::private::verify_secure_build() { - case "${CROMWELL_BUILD_PROVIDER}" in - "${CROMWELL_BUILD_PROVIDER_TRAVIS}") - if [[ "${CROMWELL_BUILD_IS_SECURE}" != "true" ]] && \ - [[ "${CROMWELL_BUILD_REQUIRES_SECURE}" == "true" ]]; then - echo "********************************************************" - echo "********************************************************" - echo "** **" - echo "** WARNING: Encrypted keys are unavailable. Exiting. **" - echo "** **" - echo "********************************************************" - echo "********************************************************" - exit 0 - fi - ;; - *) - ;; - esac -} - -cromwell::private::export_conformance_variables() { +cromwell::private::create_conformance_variables() { + CROMWELL_BUILD_CWL_RUNNER_MODE="${CROMWELL_BUILD_BACKEND_TYPE}" CROMWELL_BUILD_CWL_TOOL_VERSION="1.0.20190228155703" CROMWELL_BUILD_CWL_TEST_VERSION="1.0.20190228134645" CROMWELL_BUILD_CWL_TEST_COMMIT="1f501e38ff692a408e16b246ac7d64d32f0822c2" # use known git hash to avoid changes @@ -363,6 +489,7 @@ cromwell::private::export_conformance_variables() { CROMWELL_BUILD_CWL_TEST_OUTPUT="${CROMWELL_BUILD_LOG_DIRECTORY}/cwl_conformance_test.out.txt" CROMWELL_BUILD_CWL_TEST_PARALLELISM=10 # Set too high will cause false negatives due to cromwell server timeouts. + export CROMWELL_BUILD_CWL_RUNNER_MODE export CROMWELL_BUILD_CWL_TOOL_VERSION export CROMWELL_BUILD_CWL_TEST_VERSION export CROMWELL_BUILD_CWL_TEST_COMMIT @@ -375,12 +502,58 @@ cromwell::private::export_conformance_variables() { export CROMWELL_BUILD_CWL_TEST_PARALLELISM } +cromwell::private::verify_secure_build() { + case "${CROMWELL_BUILD_PROVIDER}" in + "${CROMWELL_BUILD_PROVIDER_TRAVIS}") + if [[ "${CROMWELL_BUILD_IS_SECURE}" != "true" ]] && \ + [[ "${CROMWELL_BUILD_REQUIRES_SECURE}" == "true" ]]; then + echo "********************************************************" + echo "********************************************************" + echo "** **" + echo "** WARNING: Encrypted keys are unavailable. Exiting. **" + echo "** **" + echo "********************************************************" + echo "********************************************************" + exit 0 + fi + ;; + *) + ;; + esac +} + +cromwell::private::verify_pull_request_build() { + case "${CROMWELL_BUILD_PROVIDER}" in + "${CROMWELL_BUILD_PROVIDER_TRAVIS}") + if [[ "${CROMWELL_BUILD_IS_PULL_REQUEST}" != "true" ]] && \ + [[ "${CROMWELL_BUILD_REQUIRES_PULL_REQUEST}" == "true" ]]; then + echo "**************************************************" + echo "**************************************************" + echo "** **" + echo "** NOTE: Build is not a Pull Request. Exiting. **" + echo "** **" + echo "**************************************************" + echo "**************************************************" + exit 0 + fi + ;; + *) + ;; + esac +} + cromwell::private::exec_test_script() { local upper_build_type upper_build_type="$(tr '[:lower:]' '[:upper:]' <<< "${CROMWELL_BUILD_TYPE:0:1}")${CROMWELL_BUILD_TYPE:1}" exec "${CROMWELL_BUILD_SCRIPTS_DIRECTORY}/test${upper_build_type}.sh" } +cromwell::private::stop_travis_defaults() { + # https://stackoverflow.com/questions/27382295/how-to-stop-services-on-travis-ci-running-by-default#answer-27410479 + sudo /etc/init.d/mysql stop + sudo /etc/init.d/postgresql stop +} + cromwell::private::delete_boto_config() { # https://github.com/travis-ci/travis-ci/issues/7940#issuecomment-310759657 sudo rm -f /etc/boto.cfg @@ -417,18 +590,65 @@ cromwell::private::install_wait_for_it() { chmod +x "$CROMWELL_BUILD_WAIT_FOR_IT_SCRIPT" } -cromwell::private::create_mysql_cromwell_test() { +cromwell::private::start_docker() { + local docker_image + local docker_cid_file + docker_image="${1:?foo called without a docker image}"; shift + docker_cid_file="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/$(echo "${docker_image}" | tr "/" "_" | tr ":" "-").cid.$$" + + docker run --cidfile="${docker_cid_file}" --detach "$@" "${docker_image}" + + cromwell::private::add_exit_function docker rm --force --volumes "$(cat "${docker_cid_file}")" + cromwell::private::add_exit_function rm "${docker_cid_file}" +} + +cromwell::private::start_docker_mysql() { if cromwell::private::is_xtrace_enabled; then - cromwell::private::exec_silent_function cromwell::private::create_mysql_cromwell_test - else - "$CROMWELL_BUILD_WAIT_FOR_IT_SCRIPT" -t 120 "${CROMWELL_BUILD_MYSQL_HOSTNAME}:${CROMWELL_BUILD_MYSQL_PORT}" - mysql \ - --host="${CROMWELL_BUILD_MYSQL_HOSTNAME}" \ - --port="${CROMWELL_BUILD_MYSQL_PORT}" \ - --user="${CROMWELL_BUILD_MYSQL_USERNAME}" \ - --password="${CROMWELL_BUILD_MYSQL_PASSWORD}" \ - --execute \ - "SET GLOBAL sql_mode = 'STRICT_ALL_TABLES'; CREATE DATABASE IF NOT EXISTS ${CROMWELL_BUILD_MYSQL_SCHEMA};" + cromwell::private::exec_silent_function cromwell::private::start_docker_mysql + + elif [[ -n "${CROMWELL_BUILD_MYSQL_DOCKER_TAG:+set}" ]]; then + cromwell::private::start_docker \ + mysql:"${CROMWELL_BUILD_MYSQL_DOCKER_TAG}" \ + --env MYSQL_ROOT_PASSWORD=private \ + --env MYSQL_USER="${CROMWELL_BUILD_MYSQL_USERNAME}" \ + --env MYSQL_PASSWORD="${CROMWELL_BUILD_MYSQL_PASSWORD}" \ + --env MYSQL_DATABASE="${CROMWELL_BUILD_MYSQL_SCHEMA}" \ + --publish "${CROMWELL_BUILD_MYSQL_PORT}":3306 \ + --volume "${CROMWELL_BUILD_DOCKER_DIRECTORY}"/mysql-conf.d:/etc/mysql/conf.d \ + + fi +} + +cromwell::private::start_docker_mariadb() { + if cromwell::private::is_xtrace_enabled; then + cromwell::private::exec_silent_function cromwell::private::start_docker_mariadb + + elif [[ -n "${CROMWELL_BUILD_MARIADB_DOCKER_TAG:+set}" ]]; then + cromwell::private::start_docker \ + mariadb:"${CROMWELL_BUILD_MARIADB_DOCKER_TAG}" \ + --env MYSQL_ROOT_PASSWORD=private \ + --env MYSQL_USER="${CROMWELL_BUILD_MARIADB_USERNAME}" \ + --env MYSQL_PASSWORD="${CROMWELL_BUILD_MARIADB_PASSWORD}" \ + --env MYSQL_DATABASE="${CROMWELL_BUILD_MARIADB_SCHEMA}" \ + --publish "${CROMWELL_BUILD_MARIADB_PORT}":3306 \ + --volume "${CROMWELL_BUILD_DOCKER_DIRECTORY}"/mariadb-conf.d:/etc/mysql/conf.d \ + + fi +} + +cromwell::private::start_docker_postgresql() { + if cromwell::private::is_xtrace_enabled; then + cromwell::private::exec_silent_function cromwell::private::start_docker_postgresql + + elif [[ -n "${CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG:+set}" ]]; then + cromwell::private::start_docker \ + postgres:"${CROMWELL_BUILD_POSTGRESQL_DOCKER_TAG}" \ + --env POSTGRES_USER="${CROMWELL_BUILD_POSTGRESQL_USERNAME}" \ + --env POSTGRES_PASSWORD="${CROMWELL_BUILD_POSTGRESQL_PASSWORD}" \ + --env POSTGRES_DB="${CROMWELL_BUILD_POSTGRESQL_SCHEMA}" \ + --publish "${CROMWELL_BUILD_POSTGRESQL_PORT}":5432 \ + --volume "${CROMWELL_BUILD_DOCKER_DIRECTORY}"/postgresql-initdb.d:/docker-entrypoint-initdb.d \ + fi } @@ -498,7 +718,7 @@ cromwell::private::vault_login() { # shellcheck disable=SC2015 docker run --rm \ -v "${CROMWELL_BUILD_HOME_DIRECTORY}:/root:rw" \ - broadinstitute/dsde-toolbox \ + broadinstitute/dsde-toolbox:dev \ vault auth "${vault_token}" < /dev/null > /dev/null && echo vault auth success \ || true ;; @@ -576,7 +796,7 @@ cromwell::private::calculate_prior_version_tag() { )" # This function should only ever run on PR builds. - if [[ -z "${CROMWELL_BUILD_BRANCH_PULL_REQUEST}" ]]; then + if [[ -z "${CROMWELL_BUILD_BRANCH_PULL_REQUEST-}" ]]; then echo "Error: the CROMWELL_BUILD_BRANCH_PULL_REQUEST variable is not set. calculate_prior_version_tag expects to only run on Travis Pull Request builds in which this variable is set." >&2 exit 1 fi @@ -626,7 +846,8 @@ cromwell::private::exists_cromwell_jar() { cromwell::private::assemble_jars() { # CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND allows for an override of the default `assembly` command for assembly. # This can be useful to reduce time and memory that might otherwise be spent assembling unused subprojects. - CROMWELL_SBT_ASSEMBLY_LOG_LEVEL=error sbt coverage ${CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND:-assembly} -error + # shellcheck disable=SC2086 + CROMWELL_SBT_ASSEMBLY_LOG_LEVEL=error sbt coverage ${CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND} -error } cromwell::private::generate_code_coverage() { @@ -737,11 +958,13 @@ cromwell::private::run_exit_functions() { } # Adds the function to the list of functions to run on exit. -# Requires one positional parameter, the function to run. +# Requires at least one positional parameter, the function to run. cromwell::private::add_exit_function() { - local exit_function - exit_function="${1:?add_exit_function called without a function}"; shift - echo "${exit_function}" >> "${CROMWELL_BUILD_EXIT_FUNCTIONS}" + if [[ "$#" -eq 0 ]]; then + echo "Error: add_exit_function called without a function" >&2 + exit 1 + fi + echo "$@" >> "${CROMWELL_BUILD_EXIT_FUNCTIONS}" trap cromwell::private::run_exit_functions TERM EXIT } @@ -823,25 +1046,29 @@ cromwell::build::setup_common_environment() { cromwell::private::check_debug cromwell::private::create_build_variables cromwell::private::echo_build_variables + cromwell::private::create_database_variables cromwell::private::verify_secure_build + cromwell::private::verify_pull_request_build cromwell::private::make_build_directories cromwell::private::setup_secure_resources case "${CROMWELL_BUILD_PROVIDER}" in "${CROMWELL_BUILD_PROVIDER_TRAVIS}") + cromwell::private::stop_travis_defaults cromwell::private::delete_boto_config cromwell::private::delete_sbt_boot cromwell::private::upgrade_pip cromwell::private::pull_common_docker_images cromwell::private::install_wait_for_it - cromwell::private::create_mysql_cromwell_test + cromwell::private::start_docker_mysql + cromwell::private::start_docker_mariadb + cromwell::private::start_docker_postgresql ;; "${CROMWELL_BUILD_PROVIDER_JENKINS}") cromwell::private::delete_boto_config cromwell::private::delete_sbt_boot cromwell::private::upgrade_pip cromwell::private::install_wait_for_it - cromwell::private::create_mysql_cromwell_test ;; *) cromwell::private::pull_common_docker_images @@ -851,14 +1078,10 @@ cromwell::build::setup_common_environment() { cromwell::build::setup_centaur_environment() { cromwell::private::create_centaur_variables - if [[ "${CROMWELL_BUILD_CENTAUR_TYPE}" == "${CROMWELL_BUILD_CENTAUR_TYPE_ENGINE_UPGRADE}" ]]; then - cromwell::private::setup_prior_version_resources - elif [[ "${CROMWELL_BUILD_CENTAUR_TYPE}" == "${CROMWELL_BUILD_CENTAUR_TYPE_PAPI_UPGRADE}" ]]; then + if [[ "${CROMWELL_BUILD_CENTAUR_TYPE}" == "${CROMWELL_BUILD_CENTAUR_TYPE_ENGINE_UPGRADE}" ]] || \ + [[ "${CROMWELL_BUILD_CENTAUR_TYPE}" == "${CROMWELL_BUILD_CENTAUR_TYPE_PAPI_UPGRADE}" ]]; then cromwell::private::setup_prior_version_resources - export CROMWELL_BUILD_CROMWELL_CONFIG="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/papi_v1_v2_upgrade_application.conf" - elif [[ "${CROMWELL_BUILD_CENTAUR_TYPE}" == "${CROMWELL_BUILD_CENTAUR_TYPE_PAPI_UPGRADE_NEW_WORKFLOWS}" ]]; then - export CROMWELL_BUILD_CROMWELL_CONFIG="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/papi_v1_v2_upgrade_application.conf" - fi; + fi cromwell::private::start_build_heartbeat cromwell::private::start_cromwell_log_tail cromwell::private::start_centaur_log_tail @@ -871,11 +1094,8 @@ cromwell::build::setup_centaur_environment() { } cromwell::build::setup_conformance_environment() { - # Override of the default sbt assembly command which is just `assembly`. - # The conformance runs only need these two subprojects so save a couple of minutes and skip the rest. - export CROMWELL_BUILD_SBT_ASSEMBLY_COMMAND="server/assembly centaurCwlRunner/assembly" - - cromwell::private::export_conformance_variables + cromwell::private::create_centaur_variables + cromwell::private::create_conformance_variables if [[ "${CROMWELL_BUILD_IS_CI}" == "true" ]]; then cromwell::private::install_cwltest fi @@ -936,10 +1156,6 @@ cromwell::build::run_centaur() { } cromwell::build::run_conformance() { - CROMWELL_BUILD_CWL_RUNNER_MODE="${CROMWELL_BUILD_BACKEND_TYPE}" - - export CROMWELL_BUILD_CWL_RUNNER_MODE - cromwell::private::start_conformance_cromwell cromwell::private::add_exit_function cromwell::private::kill_conformance_cromwell diff --git a/src/ci/bin/testCentaurEngineUpgradeLocal.sh b/src/ci/bin/testCentaurEngineUpgradeLocal.sh index 8bffc195fb2..9da358220df 100755 --- a/src/ci/bin/testCentaurEngineUpgradeLocal.sh +++ b/src/ci/bin/testCentaurEngineUpgradeLocal.sh @@ -2,19 +2,16 @@ set -o errexit -o nounset -o pipefail export CROMWELL_BUILD_OPTIONAL_SECURE=true +export CROMWELL_BUILD_REQUIRES_PULL_REQUEST=true # import in shellcheck / CI / IntelliJ compatible ways # shellcheck source=/dev/null source "${BASH_SOURCE%/*}/test.inc.sh" || source test.inc.sh -if [ "${CROMWELL_BUILD_PROVIDER}" = "${CROMWELL_BUILD_PROVIDER_TRAVIS}" ] && [ -n "${TRAVIS_PULL_REQUEST_BRANCH}" ]; then +cromwell::build::setup_common_environment - cromwell::build::setup_common_environment +cromwell::build::setup_centaur_environment - cromwell::build::setup_centaur_environment +cromwell::build::assemble_jars - cromwell::build::assemble_jars - - cromwell::build::run_centaur \ - -s "centaur.EngineUpgradeTestCaseSpec" - -fi +cromwell::build::run_centaur \ + -s "centaur.EngineUpgradeTestCaseSpec" diff --git a/src/ci/bin/testCentaurHoricromtalEngineUpgradePapiV2.sh b/src/ci/bin/testCentaurHoricromtalEngineUpgradePapiV2.sh index d21d2c2c992..09a2c0010db 100755 --- a/src/ci/bin/testCentaurHoricromtalEngineUpgradePapiV2.sh +++ b/src/ci/bin/testCentaurHoricromtalEngineUpgradePapiV2.sh @@ -2,51 +2,48 @@ set -o errexit -o nounset -o pipefail export CROMWELL_BUILD_REQUIRES_SECURE=true +export CROMWELL_BUILD_REQUIRES_PULL_REQUEST=true # import in shellcheck / CI / IntelliJ compatible ways # shellcheck source=/dev/null source "${BASH_SOURCE%/*}/test.inc.sh" || source test.inc.sh -if [ "${CROMWELL_BUILD_PROVIDER}" = "${CROMWELL_BUILD_PROVIDER_TRAVIS}" ] && [ -n "${TRAVIS_PULL_REQUEST_BRANCH}" ]; then - cromwell::build::setup_common_environment +cromwell::build::setup_common_environment - prior_version=$(cromwell::private::calculate_prior_version_tag) - export TEST_CROMWELL_PRIOR_VERSION_TAG="${prior_version}" - WOULD_BE_PRIOR_VERSION_CONF="papi_v2_${prior_version}_application.conf" - if [[ -f "$CROMWELL_BUILD_RESOURCES_DIRECTORY/$WOULD_BE_PRIOR_VERSION_CONF" ]]; then - export TEST_CROMWELL_PRIOR_VERSION_CONF="$WOULD_BE_PRIOR_VERSION_CONF" - else - export TEST_CROMWELL_PRIOR_VERSION_CONF="papi_v2_application.conf" - fi - # This is the Docker tag that will be applied to the Docker image that is created for the code being built. This image - # will *not* be pushed to Docker Hub or any other repo, it only lives local to the build. - export TEST_CROMWELL_TAG=just-testing-horicromtal - export TEST_CROMWELL_CONF="papi_v2_application.conf" - export CROMWELL_BUILD_MYSQL_USERNAME=travis - - cromwell::build::setup_centaur_environment - - cromwell::build::assemble_jars - - GOOGLE_AUTH_MODE="service-account" - GOOGLE_REFRESH_TOKEN_PATH="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/papi_refresh_token.txt" +cromwell::build::setup_centaur_environment - # Export variables used in conf files - export GOOGLE_AUTH_MODE - export GOOGLE_REFRESH_TOKEN_PATH - export TEST_CROMWELL_COMPOSE_FILE="${CROMWELL_BUILD_ROOT_DIRECTORY}/scripts/docker-compose-mysql/docker-compose-horicromtal.yml" - - # Copy rendered files - mkdir -p "${CROMWELL_BUILD_CENTAUR_TEST_RENDERED}" - cp \ - "${CROMWELL_BUILD_RESOURCES_DIRECTORY}/private_docker_papi_v2_usa.options" \ - "${TEST_CROMWELL_COMPOSE_FILE}" \ - "${CROMWELL_BUILD_CENTAUR_TEST_RENDERED}" - - docker image ls -q broadinstitute/cromwell:"${TEST_CROMWELL_TAG}" | grep . || \ - CROMWELL_SBT_DOCKER_TAGS="${TEST_CROMWELL_TAG}" sbt server/docker - - cromwell::build::run_centaur \ - -s "centaur.EngineUpgradeTestCaseSpec" \ - -e localdockertest +cromwell::build::assemble_jars +prior_version=$(cromwell::private::calculate_prior_version_tag) +export TEST_CROMWELL_PRIOR_VERSION_TAG="${prior_version}" +WOULD_BE_PRIOR_VERSION_CONF="papi_v2_${prior_version}_application.conf" +if [[ -f "$CROMWELL_BUILD_RESOURCES_DIRECTORY/$WOULD_BE_PRIOR_VERSION_CONF" ]]; then + export TEST_CROMWELL_PRIOR_VERSION_CONF="$WOULD_BE_PRIOR_VERSION_CONF" +else + export TEST_CROMWELL_PRIOR_VERSION_CONF="papi_v2_application.conf" fi +# This is the Docker tag that will be applied to the Docker image that is created for the code being built. This image +# will *not* be pushed to Docker Hub or any other repo, it only lives local to the build. +export TEST_CROMWELL_TAG=just-testing-horicromtal +export TEST_CROMWELL_CONF="papi_v2_application.conf" + +GOOGLE_AUTH_MODE="service-account" +GOOGLE_REFRESH_TOKEN_PATH="${CROMWELL_BUILD_RESOURCES_DIRECTORY}/papi_refresh_token.txt" + +# Export variables used in conf files +export GOOGLE_AUTH_MODE +export GOOGLE_REFRESH_TOKEN_PATH +export TEST_CROMWELL_COMPOSE_FILE="${CROMWELL_BUILD_ROOT_DIRECTORY}/scripts/docker-compose-mysql/docker-compose-horicromtal.yml" + +# Copy rendered files +mkdir -p "${CROMWELL_BUILD_CENTAUR_TEST_RENDERED}" +cp \ + "${CROMWELL_BUILD_RESOURCES_DIRECTORY}/private_docker_papi_v2_usa.options" \ + "${TEST_CROMWELL_COMPOSE_FILE}" \ + "${CROMWELL_BUILD_CENTAUR_TEST_RENDERED}" + +docker image ls -q broadinstitute/cromwell:"${TEST_CROMWELL_TAG}" | grep . || \ +CROMWELL_SBT_DOCKER_TAGS="${TEST_CROMWELL_TAG}" sbt server/docker + +cromwell::build::run_centaur \ + -s "centaur.EngineUpgradeTestCaseSpec" \ + -e localdockertest diff --git a/src/ci/bin/testCentaurHoricromtalPapiV2.sh b/src/ci/bin/testCentaurHoricromtalPapiV2.sh index 3607e599184..a386a3739ad 100755 --- a/src/ci/bin/testCentaurHoricromtalPapiV2.sh +++ b/src/ci/bin/testCentaurHoricromtalPapiV2.sh @@ -10,7 +10,6 @@ source "${BASH_SOURCE%/*}/test.inc.sh" || source test.inc.sh # There should probably be more indirections in CI scripts but that can wait. export TEST_CROMWELL_TAG=just-testing-horicromtal export TEST_CROMWELL_CONF=papi_v2_horicromtal_application.conf -export CROMWELL_BUILD_MYSQL_USERNAME=travis cromwell::build::setup_common_environment diff --git a/src/ci/bin/testCentaurPapiUpgradeNewWorkflowsPapiV1.sh b/src/ci/bin/testCentaurPapiUpgradeNewWorkflowsPapiV1.sh index e04fd68de81..ae1ce769ef9 100755 --- a/src/ci/bin/testCentaurPapiUpgradeNewWorkflowsPapiV1.sh +++ b/src/ci/bin/testCentaurPapiUpgradeNewWorkflowsPapiV1.sh @@ -2,21 +2,18 @@ set -o errexit -o nounset -o pipefail export CROMWELL_BUILD_REQUIRES_SECURE=true +export CROMWELL_BUILD_REQUIRES_PULL_REQUEST=true # import in shellcheck / CI / IntelliJ compatible ways # shellcheck source=/dev/null source "${BASH_SOURCE%/*}/test.inc.sh" || source test.inc.sh -if [ "${CROMWELL_BUILD_PROVIDER}" = "${CROMWELL_BUILD_PROVIDER_TRAVIS}" ] && [ -n "${TRAVIS_PULL_REQUEST_BRANCH}" ]; then +cromwell::build::setup_common_environment - cromwell::build::setup_common_environment +cromwell::build::setup_centaur_environment - cromwell::build::setup_centaur_environment +cromwell::build::assemble_jars - cromwell::build::assemble_jars - - cromwell::build::run_centaur \ - -p 100 \ - -e localdockertest \ - -e gpu_on_papi \ - -fi +cromwell::build::run_centaur \ + -p 100 \ + -e localdockertest \ + -e gpu_on_papi \ diff --git a/src/ci/bin/testCentaurPapiUpgradePapiV1.sh b/src/ci/bin/testCentaurPapiUpgradePapiV1.sh index b6a83f51755..bfe2ec5101a 100755 --- a/src/ci/bin/testCentaurPapiUpgradePapiV1.sh +++ b/src/ci/bin/testCentaurPapiUpgradePapiV1.sh @@ -2,20 +2,17 @@ set -o errexit -o nounset -o pipefail export CROMWELL_BUILD_REQUIRES_SECURE=true +export CROMWELL_BUILD_REQUIRES_PULL_REQUEST=true # import in shellcheck / CI / IntelliJ compatible ways # shellcheck source=/dev/null source "${BASH_SOURCE%/*}/test.inc.sh" || source test.inc.sh -if [ "${CROMWELL_BUILD_PROVIDER}" = "${CROMWELL_BUILD_PROVIDER_TRAVIS}" ] && [ -n "${TRAVIS_PULL_REQUEST_BRANCH}" ]; then +cromwell::build::setup_common_environment - cromwell::build::setup_common_environment +cromwell::build::setup_centaur_environment - cromwell::build::setup_centaur_environment +cromwell::build::assemble_jars - cromwell::build::assemble_jars - - cromwell::build::run_centaur \ - -s "centaur.PapiUpgradeTestCaseSpec" \ - -e localdockertest \ - -fi +cromwell::build::run_centaur \ + -s "centaur.PapiUpgradeTestCaseSpec" \ + -e localdockertest \ diff --git a/src/ci/docker-compose/cromwell-test/docker-setup.sh b/src/ci/docker-compose/cromwell-test/docker-setup.sh index 03b1cacf16e..fb13b6bd714 100755 --- a/src/ci/docker-compose/cromwell-test/docker-setup.sh +++ b/src/ci/docker-compose/cromwell-test/docker-setup.sh @@ -13,8 +13,9 @@ apt-get install -y \ curl \ gnupg \ gnupg2 \ - python-dev \ mysql-client \ + postgresql-client \ + python-dev \ software-properties-common \ sudo \ diff --git a/src/ci/docker-compose/docker-compose.yml b/src/ci/docker-compose/docker-compose.yml index a13448367e2..14c7b07afd5 100644 --- a/src/ci/docker-compose/docker-compose.yml +++ b/src/ci/docker-compose/docker-compose.yml @@ -19,9 +19,31 @@ services: - /var/run/docker.sock:/var/run/docker.sock links: - mysql-db + - mariadb-db + - postgresql-db mysql-db: image: "mysql:5.7" environment: - - MYSQL_ALLOW_EMPTY_PASSWORD=yes - - MYSQL_ROOT_PASSWORD= + - MYSQL_ROOT_PASSWORD=private + - MYSQL_USER=cromwell + - MYSQL_PASSWORD=test - MYSQL_DATABASE=cromwell_test + volumes: + - ./mysql-conf.d:/etc/mysql/conf.d + mariadb-db: + image: "mariadb:10.3" + environment: + - MYSQL_ROOT_PASSWORD=private + - MYSQL_USER=cromwell + - MYSQL_PASSWORD=test + - MYSQL_DATABASE=cromwell_test + volumes: + - ./mariadb-conf.d:/etc/mysql/conf.d + postgresql-db: + image: "postgres:11.3" + environment: + - POSTGRES_USER=cromwell + - POSTGRES_PASSWORD=test + - POSTGRES_DB=cromwell_test + volumes: + - ./postgresql-initdb.d:/docker-entrypoint-initdb.d diff --git a/src/ci/docker-compose/mariadb-conf.d/init.cnf b/src/ci/docker-compose/mariadb-conf.d/init.cnf new file mode 100644 index 00000000000..ae32e540b7f --- /dev/null +++ b/src/ci/docker-compose/mariadb-conf.d/init.cnf @@ -0,0 +1,3 @@ +[mysqld] +max_connections=300 +sql_mode=STRICT_ALL_TABLES diff --git a/src/ci/docker-compose/mysql-conf.d/init.cnf b/src/ci/docker-compose/mysql-conf.d/init.cnf new file mode 100644 index 00000000000..ae32e540b7f --- /dev/null +++ b/src/ci/docker-compose/mysql-conf.d/init.cnf @@ -0,0 +1,3 @@ +[mysqld] +max_connections=300 +sql_mode=STRICT_ALL_TABLES diff --git a/src/ci/docker-compose/postgresql-initdb.d/init.sh b/src/ci/docker-compose/postgresql-initdb.d/init.sh new file mode 100755 index 00000000000..f861d9c4551 --- /dev/null +++ b/src/ci/docker-compose/postgresql-initdb.d/init.sh @@ -0,0 +1,5 @@ +#!/usr/bin/env bash + +cat << EOF >> /var/lib/postgresql/data/postgresql.conf +max_connections = 300 +EOF diff --git a/src/ci/docker-compose/postgresql-initdb.d/init.sql b/src/ci/docker-compose/postgresql-initdb.d/init.sql new file mode 100644 index 00000000000..8cff561ceee --- /dev/null +++ b/src/ci/docker-compose/postgresql-initdb.d/init.sql @@ -0,0 +1 @@ +create extension lo; diff --git a/src/ci/resources/build_application.inc.conf b/src/ci/resources/build_application.inc.conf index 9dad320b135..fc87166fc11 100644 --- a/src/ci/resources/build_application.inc.conf +++ b/src/ci/resources/build_application.inc.conf @@ -2,7 +2,7 @@ akka.http.host-connection-pool.max-open-requests = 1024 workflow-options { base64-encryption-key = "AAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA=" - base64-encryption-key = ${?CROMWELL_BUILD_RANDOM_256_BITS_BASE64} + base64-encryption-key = ${?CROMWELL_BUILD_CENTAUR_256_BITS_KEY} encrypted-fields = [ "docker_credentials_key_name", "docker_credentials_token", diff --git a/src/ci/resources/cromwell_database.inc.conf b/src/ci/resources/cromwell_database.inc.conf index c726df7f091..4c93bb2635b 100644 --- a/src/ci/resources/cromwell_database.inc.conf +++ b/src/ci/resources/cromwell_database.inc.conf @@ -1,17 +1,14 @@ database { + profile = "slick.jdbc.MySQLProfile$" + profile = ${?CROMWELL_BUILD_CENTAUR_SLICK_PROFILE} db { - hostname = localhost - hostname = ${?CROMWELL_BUILD_MYSQL_HOSTNAME} - port = 3306 - port = ${?CROMWELL_BUILD_MYSQL_PORT} - schema = cromwell_test - schema = ${?CROMWELL_BUILD_MYSQL_SCHEMA} - url = "jdbc:mysql://"${database.db.hostname}":"${database.db.port}"/"${database.db.schema}"?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" - user = root - user = ${?CROMWELL_BUILD_MYSQL_USERNAME} - password = "" - password = ${?CROMWELL_BUILD_MYSQL_PASSWORD} driver = "com.mysql.cj.jdbc.Driver" + driver = ${?CROMWELL_BUILD_CENTAUR_JDBC_DRIVER} + url = "jdbc:mysql://localhost:3306/cromwell_test?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" + url = ${?CROMWELL_BUILD_CENTAUR_JDBC_URL} + user = "cromwell" + user = ${?CROMWELL_BUILD_CENTAUR_JDBC_USERNAME} + password = "test" + password = ${?CROMWELL_BUILD_CENTAUR_JDBC_PASSWORD} } - profile = "slick.jdbc.MySQLProfile$" } From 49d111ac4c77616182bb704bc049d9bec6cce725 Mon Sep 17 00:00:00 2001 From: Chris Llanwarne Date: Thu, 13 Jun 2019 15:53:29 -0400 Subject: [PATCH 11/28] Fixup perf configuration around statsd instrumentation (#5024) --- scripts/perf/vm_scripts/cromwell/cromwell.conf | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/scripts/perf/vm_scripts/cromwell/cromwell.conf b/scripts/perf/vm_scripts/cromwell/cromwell.conf index 5c1df82c341..1747f25ed5d 100644 --- a/scripts/perf/vm_scripts/cromwell/cromwell.conf +++ b/scripts/perf/vm_scripts/cromwell/cromwell.conf @@ -72,7 +72,7 @@ services { Instrumentation { # StatsD - Send metrics to a StatsD server over UDP class = "cromwell.services.instrumentation.impl.statsd.StatsDInstrumentationServiceActor" - config.statsd { + config { hostname = statsd-proxy port = 9125 prefix = ${?CROMWELL_STATSD_PREFIX} # can be used to prefix all metrics with an api key for example From 05c9dbfd9b03410640082e2da6718605e5e02030 Mon Sep 17 00:00:00 2001 From: Ruchi Date: Fri, 14 Jun 2019 08:32:12 -0400 Subject: [PATCH 12/28] added server logging for error code 10 [BA-5703] (#5030) * added server logging for error code 10 --- .../PipelinesApiAsyncBackendJobExecutionActor.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) 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 a2051b98e38..01fa8aebcc0 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 @@ -66,6 +66,7 @@ object PipelinesApiAsyncBackendJobExecutionActor { val JesPreemption = 14 val PapiFailedPreConditionErrorCode = 9 + val PapiMysteriouslyCrashedErrorCode = 10 // If the JES code is 2 (UNKNOWN), this sub-string indicates preemption: val FailedToStartDueToPreemptionSubstring = "failed to start due to preemption" @@ -409,7 +410,7 @@ class PipelinesApiAsyncBackendJobExecutionActor(override val standardParams: Sta * In CWL-land we tend to be aggressive in pre-fetching the size in order to be able to evaluate JS expressions, * but less in WDL as we can get it last minute and on demand because size is a WDL function, whereas in CWL * we don't inspect the JS to know if size is called and therefore always pre-fetch it. - * + * * We could decide to call withSize before in which case we would retrieve the size for all files and have * a guaranteed more accurate total size, but there might be performance impacts ? */ @@ -602,7 +603,13 @@ class PipelinesApiAsyncBackendJobExecutionActor(override val standardParams: Sta && errorMsg.contains("Execution failed") && (errorMsg.contains("Localization") || errorMsg.contains("Delocalization"))) { s"Please check the log file for more details: $jesLogPath." - } else errorMsg + } + //If error code 10, add some extra messaging to the server logging + else if (runStatus.errorCode.getCode.value == PapiMysteriouslyCrashedErrorCode) { + jobLogger.info(s"Job Failed with Error Code 10 for a machine where Preemptible is set to $preemptible") + errorMsg + } + else errorMsg } // Inner function: Handles a 'Failed' runStatus (or Preempted if preemptible was false) From 8bf12a8b88f0ae776e9301e84613818597711b1a Mon Sep 17 00:00:00 2001 From: Chris Llanwarne Date: Fri, 14 Jun 2019 10:25:31 -0400 Subject: [PATCH 13/28] Allow includeKey and excludeKey non exclusively [BA-5671] (#5021) --- CHANGELOG.md | 11 ++- .../slick/MetadataSlickDatabase.scala | 28 ++----- .../slick/tables/MetadataEntryComponent.scala | 73 ++++++++----------- .../database/sql/MetadataSqlDatabase.scala | 11 +-- docs/api/RESTAPI.md | 6 +- .../src/main/resources/swagger/cromwell.yaml | 9 ++- .../routes/MetadataRouteSupport.scala | 13 +--- .../routes/CromwellApiServiceSpec.scala | 36 +++++---- .../routes/MetadataRouteSupportSpec.scala | 39 ++++------ .../services/metadata/MetadataService.scala | 7 +- .../impl/MetadataDatabaseAccess.scala | 33 ++++----- .../metadata/impl/ReadMetadataActor.scala | 2 +- .../impl/WriteMetadataActorSpec.scala | 34 +++------ 13 files changed, 132 insertions(+), 170 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 73ea7d40e65..4420f5a1dd0 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -7,12 +7,21 @@ Cromwell now supports sending metrics to [Google's Stackdriver API](https://cloud.google.com/monitoring/api/v3/). Learn more on how to configure [here](https://cromwell.readthedocs.io/en/stable/developers/Instrumentation/). -### StatsD Instrumentation Config change +### Configuration Changes + +#### StatsD Instrumentation There is a small change in StatsD's configuration path. Originally, the path to the config was `services.Instrumentation.config.statsd` which now has been updated to `services.Instrumentation.config`. More info on its configuration can be found [here](https://cromwell.readthedocs.io/en/stable/developers/Instrumentation/). +### API Changes + +#### Workflow Metadata + +* It is now possible to use `includeKey` and `excludeKey` at the same time. If so, the metadata key must match the `includeKey` **and not** match the `excludeKey` to be included. +* It is now possible to use "`calls`" as one of your `excludeKey`s, to request that only workflow metadata gets returned. + ## 42 Release Notes ### Womtool endpoint diff --git a/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala b/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala index 91dbc44c3a0..2bb2f46ff8f 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala @@ -2,7 +2,6 @@ package cromwell.database.slick import java.sql.Timestamp -import cats.data.NonEmptyList import com.typesafe.config.{Config, ConfigFactory} import cromwell.database.slick.tables.MetadataDataAccessComponent import cromwell.database.sql.MetadataSqlDatabase @@ -83,29 +82,18 @@ class MetadataSlickDatabase(originalDatabaseConfig: Config) runTransaction(action) } - override def queryMetadataEntriesLikeMetadataKeys(workflowExecutionUuid: String, - metadataKeys: NonEmptyList[String], + override def queryMetadataEntryWithKeyConstraints(workflowExecutionUuid: String, + metadataKeysToFilterFor: List[String], + metadataKeysToFilterOut: List[String], metadataJobQueryValue: MetadataJobQueryValue) (implicit ec: ExecutionContext): Future[Seq[MetadataEntry]] = { val action = metadataJobQueryValue match { case CallQuery(callFqn, jobIndex, jobAttempt) => - dataAccess.metadataEntriesLikeMetadataKeysWithJob(workflowExecutionUuid, metadataKeys, callFqn, jobIndex, jobAttempt).result - case WorkflowQuery => dataAccess.metadataEntriesLikeMetadataKeys(workflowExecutionUuid, metadataKeys, requireEmptyJobKey = true).result - case CallOrWorkflowQuery => dataAccess.metadataEntriesLikeMetadataKeys(workflowExecutionUuid, metadataKeys, requireEmptyJobKey = false).result - } - - runTransaction(action) - } - - override def queryMetadataEntryNotLikeMetadataKeys(workflowExecutionUuid: String, - metadataKeys: NonEmptyList[String], - metadataJobQueryValue: MetadataJobQueryValue) - (implicit ec: ExecutionContext): Future[Seq[MetadataEntry]] = { - val action = metadataJobQueryValue match { - case CallQuery(callFqn, jobIndex, jobAttempt) => - dataAccess.metadataEntriesNotLikeMetadataKeysWithJob(workflowExecutionUuid, metadataKeys, callFqn, jobIndex, jobAttempt).result - case WorkflowQuery => dataAccess.metadataEntriesNotLikeMetadataKeys(workflowExecutionUuid, metadataKeys, requireEmptyJobKey = true).result - case CallOrWorkflowQuery => dataAccess.metadataEntriesNotLikeMetadataKeys(workflowExecutionUuid, metadataKeys, requireEmptyJobKey = false).result + dataAccess.metadataEntriesForJobWithKeyConstraints(workflowExecutionUuid, metadataKeysToFilterFor, metadataKeysToFilterOut, callFqn, jobIndex, jobAttempt).result + case WorkflowQuery => + dataAccess.metadataEntriesWithKeyConstraints(workflowExecutionUuid, metadataKeysToFilterFor, metadataKeysToFilterOut, requireEmptyJobKey = true).result + case CallOrWorkflowQuery => + dataAccess.metadataEntriesWithKeyConstraints(workflowExecutionUuid, metadataKeysToFilterFor, metadataKeysToFilterOut, requireEmptyJobKey = false).result } runTransaction(action) } diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala index 711cde84f56..9e8106aebd4 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala @@ -2,7 +2,6 @@ package cromwell.database.slick.tables import java.sql.{Clob, Timestamp} -import cats.data.NonEmptyList import cromwell.database.sql.tables.MetadataEntry trait MetadataEntryComponent { @@ -152,12 +151,14 @@ trait MetadataEntryComponent { * If requireEmptyJobKey is true, only workflow level keys are returned, otherwise both workflow and call level * keys are returned. */ - def metadataEntriesLikeMetadataKeys(workflowExecutionUuid: String, metadataKeys: NonEmptyList[String], - requireEmptyJobKey: Boolean) = { + def metadataEntriesWithKeyConstraints(workflowExecutionUuid: String, + metadataKeysToFilterFor: List[String], + metadataKeysToFilterOut: List[String], + requireEmptyJobKey: Boolean) = { (for { metadataEntry <- metadataEntries if metadataEntry.workflowExecutionUuid === workflowExecutionUuid - if metadataEntryHasMetadataKeysLike(metadataEntry, metadataKeys) + if metadataEntryHasMetadataKeysLike(metadataEntry, metadataKeysToFilterFor, metadataKeysToFilterOut) if metadataEntryHasEmptyJobKey(metadataEntry, requireEmptyJobKey) } yield metadataEntry).sortBy(_.metadataTimestamp) } @@ -166,59 +167,43 @@ trait MetadataEntryComponent { * Returns metadata entries that are "like" metadataKeys for the specified call. * If jobAttempt has no value, all metadata keys for all attempts are returned. */ - def metadataEntriesLikeMetadataKeysWithJob(workflowExecutionUuid: String, metadataKeys: NonEmptyList[String], - callFqn: String, jobIndex: Option[Int], jobAttempt: Option[Int]) = { + def metadataEntriesForJobWithKeyConstraints(workflowExecutionUuid: String, + metadataKeysToFilterFor: List[String], + metadataKeysToFilterOut: List[String], + callFqn: String, + jobIndex: Option[Int], + jobAttempt: Option[Int]) = { (for { metadataEntry <- metadataEntries if metadataEntry.workflowExecutionUuid === workflowExecutionUuid - if metadataEntryHasMetadataKeysLike(metadataEntry, metadataKeys) + if metadataEntryHasMetadataKeysLike(metadataEntry, metadataKeysToFilterFor, metadataKeysToFilterOut) if metadataEntry.callFullyQualifiedName === callFqn if hasSameIndex(metadataEntry, jobIndex) // Assume that every metadata entry for a call should have a non null attempt value - // Because of that, if the jobAttempt paramater is Some(_), make sure it matches, otherwise take all entries + // Because of that, if the jobAttempt parameter is Some(_), make sure it matches, otherwise take all entries // regardless of the attempt if (metadataEntry.jobAttempt === jobAttempt) || jobAttempt.isEmpty } yield metadataEntry).sortBy(_.metadataTimestamp) } - /** - * Returns metadata entries that are NOT "like" metadataKeys for the specified workflow. - * If requireEmptyJobKey is true, only workflow level keys are returned, otherwise both workflow and call level - * keys are returned. - */ - def metadataEntriesNotLikeMetadataKeys(workflowExecutionUuid: String, metadataKeys: NonEmptyList[String], - requireEmptyJobKey: Boolean) = { - (for { - metadataEntry <- metadataEntries - if metadataEntry.workflowExecutionUuid === workflowExecutionUuid - if !metadataEntryHasMetadataKeysLike(metadataEntry, metadataKeys) - if metadataEntryHasEmptyJobKey(metadataEntry, requireEmptyJobKey) - } yield metadataEntry).sortBy(_.metadataTimestamp) - } + private[this] def metadataEntryHasMetadataKeysLike(metadataEntry: MetadataEntries, + metadataKeysToFilterFor: List[String], + metadataKeysToFilterOut: List[String]): Rep[Boolean] = { - /** - * Returns metadata entries that are NOT "like" metadataKeys for the specified call. - * If jobIndex (resp. jobAttempt) has no value, all metadata keys for all indices (resp. attempt) - * are returned. - */ - def metadataEntriesNotLikeMetadataKeysWithJob(workflowExecutionUuid: String, metadataKeys: NonEmptyList[String], - callFqn: String, jobIndex: Option[Int], jobAttempt: Option[Int]) = { - (for { - metadataEntry <- metadataEntries - if metadataEntry.workflowExecutionUuid === workflowExecutionUuid - if !metadataEntryHasMetadataKeysLike(metadataEntry, metadataKeys) - if metadataEntry.callFullyQualifiedName === callFqn - if hasSameIndex(metadataEntry, jobIndex) - // Assume that every metadata entry for a call should have a non null attempt value - // Because of that, if the jobAttempt parameter is Some(_), make sure it matches, otherwise take all entries - // regardless of the attempt - if (metadataEntry.jobAttempt === jobAttempt) || jobAttempt.isEmpty - } yield metadataEntry).sortBy(_.metadataTimestamp) - } + def containsKey(key: String): Rep[Boolean] = metadataEntry.metadataKey like key - private[this] def metadataEntryHasMetadataKeysLike(metadataEntry: MetadataEntries, - metadataKeys: NonEmptyList[String]): Rep[Boolean] = { - metadataKeys.toList.map(metadataEntry.metadataKey like _).reduce(_ || _) + val positiveFilter: Option[Rep[Boolean]] = metadataKeysToFilterFor.map(containsKey).reduceOption(_ || _) + val negativeFilter: Option[Rep[Boolean]] = metadataKeysToFilterOut.map(containsKey).reduceOption(_ || _) + + (positiveFilter, negativeFilter) match { + case (Some(pf), Some(nf)) => pf && !nf + case (Some(pf), None) => pf + case (None, Some(nf)) => !nf + + // We should never get here, but there's no reason not to handle it: + // ps: is there a better literal "true" in slick? + case (None, None) => true: Rep[Boolean] + } } private[this] def hasSameIndex(metadataEntry: MetadataEntries, jobIndex: Rep[Option[Int]]) = { diff --git a/database/sql/src/main/scala/cromwell/database/sql/MetadataSqlDatabase.scala b/database/sql/src/main/scala/cromwell/database/sql/MetadataSqlDatabase.scala index 30e07f94d37..eaf883fa2bf 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/MetadataSqlDatabase.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/MetadataSqlDatabase.scala @@ -2,7 +2,6 @@ package cromwell.database.sql import java.sql.Timestamp -import cats.data.NonEmptyList import cromwell.database.sql.joins.MetadataJobQueryValue import cromwell.database.sql.tables.{MetadataEntry, WorkflowMetadataSummaryEntry} @@ -51,16 +50,12 @@ trait MetadataSqlDatabase extends SqlDatabase { jobAttempt: Option[Int]) (implicit ec: ExecutionContext): Future[Seq[MetadataEntry]] - def queryMetadataEntriesLikeMetadataKeys(workflowExecutionUuid: String, - metadataKeys: NonEmptyList[String], + def queryMetadataEntryWithKeyConstraints(workflowExecutionUuid: String, + metadataKeysToFilterFor: List[String], + metadataKeysToFilterAgainst: List[String], metadataJobQueryValue: MetadataJobQueryValue) (implicit ec: ExecutionContext): Future[Seq[MetadataEntry]] - def queryMetadataEntryNotLikeMetadataKeys(workflowExecutionUuid: String, - metadataKeys: NonEmptyList[String], - metadataJobQueryValue: MetadataJobQueryValue) - (implicit ec: ExecutionContext): Future[Seq[MetadataEntry]] - /** * Retrieves next summarizable block of metadata satisfying the specified criteria. * diff --git a/docs/api/RESTAPI.md b/docs/api/RESTAPI.md index 380bdc15a63..62d9cda44fc 100644 --- a/docs/api/RESTAPI.md +++ b/docs/api/RESTAPI.md @@ -1,5 +1,5 @@ + Factored into a separate changeset from the above to allow for handling various RDBMS implementations differently. From 195de29cd4f918222311047ce4272876041e3731 Mon Sep 17 00:00:00 2001 From: Ruben Vorderman Date: Tue, 25 Jun 2019 19:22:42 +0200 Subject: [PATCH 26/28] Make cached-copy localization strategy aware of the maximum number of hardlinks. [BA-5748] (#5043) --- .../cromwell.examples.conf | 4 +++ .../backend/sfs/SharedFileSystem.scala | 20 +++++++++---- .../backend/sfs/SharedFileSystemSpec.scala | 28 +++++++++++++++++++ 3 files changed, 46 insertions(+), 6 deletions(-) diff --git a/cromwell.example.backends/cromwell.examples.conf b/cromwell.example.backends/cromwell.examples.conf index 4318bce04bd..c5c76e5c404 100644 --- a/cromwell.example.backends/cromwell.examples.conf +++ b/cromwell.example.backends/cromwell.examples.conf @@ -421,6 +421,10 @@ backend { localization: [ "hard-link", "soft-link", "copy" ] + # An experimental localization strategy called "cached-copy" is also available for SFS backends. + # This will copy a file to a cache and then hard-link from the cache. It will copy the file to the cache again + # when the maximum number of hardlinks for a file is reached. The maximum number of hardlinks can be set with: + # max-hardlinks: 950 # Call caching strategies caching { diff --git a/supportedBackends/sfs/src/main/scala/cromwell/backend/sfs/SharedFileSystem.scala b/supportedBackends/sfs/src/main/scala/cromwell/backend/sfs/SharedFileSystem.scala index 850e2e5871d..4486cae6bcf 100644 --- a/supportedBackends/sfs/src/main/scala/cromwell/backend/sfs/SharedFileSystem.scala +++ b/supportedBackends/sfs/src/main/scala/cromwell/backend/sfs/SharedFileSystem.scala @@ -14,6 +14,7 @@ import cromwell.backend.io.JobPaths import cromwell.core.CromwellFatalExceptionMarker import cromwell.core.path.{DefaultPath, DefaultPathBuilder, Path, PathFactory} import cromwell.filesystems.http.HttpPathBuilder +import net.ceedubs.ficus.Ficus._ import wom.WomFileMapper import wom.values._ @@ -101,13 +102,17 @@ object SharedFileSystem extends StrictLogging { Thread.sleep(1) } } + + private def countLinks(path: Path): Int = { + path.getAttribute("unix:nlink").asInstanceOf[Int] + } } trait SharedFileSystem extends PathFactory { import SharedFileSystem._ def sharedFileSystemConfig: Config - + lazy val maxHardLinks: Int = sharedFileSystemConfig.getOrElse[Int]("max-hardlinks",950) // Windows limit 1024. Keep a safe margin. lazy val cachedCopyDir: Option[Path] = None private def localizePathViaCachedCopy(originalPath: Path, executionPath: Path, docker: Boolean): Try[Unit] = { @@ -124,7 +129,7 @@ trait SharedFileSystem extends PathFactory { val cachedCopyPath: Path = cachedCopySubDir./(pathAndModTime) val cachedCopyPathLockFile: Path = cachedCopyPath.plusSuffix(".lock") - if (!cachedCopyPath.exists) { + if (!cachedCopyPath.exists || countLinks(cachedCopyPath) >= maxHardLinks) { // This variable is used so we can release the lock before we start with the copying. var shouldCopy = false @@ -133,10 +138,11 @@ trait SharedFileSystem extends PathFactory { // decisions which are not time consuming. SharedFileSystem.synchronized { - // We check again if cachedCopyPath is there. It may have been created while waiting on the lock. - // If it is not there, is it already being copied by another thread? + // We check again if cachedCopyPath is there or if the number of Hardlinks is still exceeded. + // The copying may have been started while waiting on the lock. + // If it is not there or the maxHardLinks are exceeded, is it already being copied by another thread? // if not copied by another thread, is it copied by another cromwell process? (Lock file present) - if (!cachedCopyPath.exists && + if ((!cachedCopyPath.exists || countLinks(cachedCopyPath) >= maxHardLinks) && !SharedFileSystem.beingCopied.getOrElse(cachedCopyPath, false) && !cachedCopyPathLockFile.exists) { // Create a lock file so other cromwell processes know copying has started @@ -163,7 +169,9 @@ trait SharedFileSystem extends PathFactory { if (shouldCopy) { try { val cachedCopyTmpPath = cachedCopyPath.plusExt("tmp") - originalPath.copyTo(cachedCopyTmpPath, overwrite = true).moveTo(cachedCopyPath) + // CachedCopyPath is overwritten. It is possible that the number of hardlinks is exceeded. In which case + // the file is already there. + originalPath.copyTo(cachedCopyTmpPath, overwrite = true).moveTo(cachedCopyPath, overwrite = true) } catch { case e: Exception => throw e } diff --git a/supportedBackends/sfs/src/test/scala/cromwell/backend/sfs/SharedFileSystemSpec.scala b/supportedBackends/sfs/src/test/scala/cromwell/backend/sfs/SharedFileSystemSpec.scala index 6a4f000dc9b..92bf01d2e8c 100644 --- a/supportedBackends/sfs/src/test/scala/cromwell/backend/sfs/SharedFileSystemSpec.scala +++ b/supportedBackends/sfs/src/test/scala/cromwell/backend/sfs/SharedFileSystemSpec.scala @@ -20,6 +20,7 @@ class SharedFileSystemSpec extends FlatSpec with Matchers with Mockito with Tabl val hardLinkLocalization = ConfigFactory.parseString(""" localization: [hard-link] """) val softLinkLocalization = ConfigFactory.parseString(""" localization: [soft-link] """) val cachedCopyLocalization = ConfigFactory.parseString(""" localization: [cached-copy] """) + val cachedCopyLocalizationMaxHardlinks = ConfigFactory.parseString("""{localization: [cached-copy], max-hardlinks: 3 }""") val localPathBuilder = List(DefaultPathBuilder) @@ -141,6 +142,33 @@ class SharedFileSystemSpec extends FlatSpec with Matchers with Mockito with Tabl dests.foreach(_.delete(swallowIOExceptions = true)) } +it should "copy the file again when the copy-cached file has exceeded the maximum number of hardlinks" in { + val callDirs: IndexedSeq[Path] = 1 to 3 map { _ => DefaultPathBuilder.createTempDirectory("SharedFileSystem") } + val orig = DefaultPathBuilder.createTempFile("inputFile") + val dests = callDirs.map(_./(orig.parent.pathAsString.hashCode.toString())./(orig.name)) + orig.touch() + val inputs = fqnWdlMapToDeclarationMap(Map("input" -> WomSingleFile(orig.pathAsString))) + val sharedFS = new SharedFileSystem { + override val pathBuilders = localPathBuilder + override val sharedFileSystemConfig = cachedCopyLocalizationMaxHardlinks + override implicit def actorContext: ActorContext = null + override lazy val cachedCopyDir = Some(DefaultPathBuilder.createTempDirectory("cached-copy")) + } + val cachedFile: Option[Path] = sharedFS.cachedCopyDir.map( + _./(orig.parent.pathAsString.hashCode.toString)./(orig.lastModifiedTime.toEpochMilli.toString + orig.name)) + + val results = callDirs.map(sharedFS.localizeInputs(_, docker = true)(inputs)) + + results.foreach(_.isSuccess shouldBe true) + dests.foreach(_.exists shouldBe true) + dests.foreach(countLinks(_) should be <= 3) + + cachedFile.foreach(_.exists shouldBe true) + cachedFile.foreach(countLinks(_) should be <= 3) + orig.delete(swallowIOExceptions = true) + dests.foreach(_.delete(swallowIOExceptions = true)) + } + private[this] def countLinks(file: Path): Int = file.getAttribute("unix:nlink").asInstanceOf[Int] From 958663944d9236828daa8bff8e991bba4de963e9 Mon Sep 17 00:00:00 2001 From: Nat Echols Date: Mon, 29 Apr 2019 12:46:37 -0700 Subject: [PATCH 27/28] Postgresql support Including migration changes and workarounds for auto-incremented columns and large objects. --- .travis.yml | 3 + CHANGELOG.md | 7 + core/src/test/resources/application.conf | 4 +- .../cromwell.examples.conf | 13 + .../src/main/resources/changelog.xml | 1 + .../add_attempt_in_call_caching_entry.xml | 2 +- .../changesets/add_attempt_in_execution.xml | 2 +- .../changesets/add_index_in_execution.xml | 2 +- .../add_start_end_time_in_execution.xml | 2 +- .../changesets/add_unique_constraints.xml | 2 +- .../changesets/add_workflow_name.xml | 2 +- ...d_workflow_url_in_workflow_store_entry.xml | 2 +- .../resources/changesets/backend_KV_Store.xml | 4 +- .../call_caching_aggregated_hashes.xml | 4 +- .../call_caching_allow_result_reuse_fix.xml | 6 +- .../changesets/call_caching_job_detritus.xml | 6 +- .../changesets/call_result_caching.xml | 6 +- .../main/resources/changesets/callcaching.xml | 16 +- .../change_execution_unique_constraint.xml | 2 +- .../change_max_size_label_entry.xml | 2 +- .../change_max_size_workflow_url.xml | 2 +- .../changesets/custom_label_entry.xml | 8 +- .../main/resources/changesets/db_schema.xml | 6 +- .../changesets/docker_hash_store.xml | 4 +- .../docker_hash_store_add_size_column.xml | 2 +- .../drop_workflow_uri_and_local_command.xml | 2 +- .../changesets/embiggen_detritus_value.xml | 2 +- .../changesets/embiggen_metadata_value.xml | 6 +- .../encrypt_and_clear_workflow_options.xml | 4 +- .../enlarge_call_caching_hash_entry_id.xml | 2 +- .../resources/changesets/events_table.xml | 2 +- .../changesets/execution_backend_info.xml | 16 +- .../resources/changesets/failure_table.xml | 2 +- .../resources/changesets/jes_id_update.xml | 2 +- .../main/resources/changesets/job_store.xml | 6 +- .../changesets/job_store_simpletons.xml | 8 +- .../changesets/job_store_tinyints.xml | 8 +- .../changesets/lengthen_wdl_value.xml | 2 +- .../changesets/local_job_allow_null.xml | 2 +- .../resources/changesets/metadata_journal.xml | 8 +- .../metadata_journal_subsecond_timestamp.xml | 4 +- .../changesets/metadata_typed_values.xml | 2 +- .../move_sql_metadata_changelog.xml | 2 +- .../resources/changesets/nullable_lobs.xml | 2 +- .../resources/changesets/optional_sge_ids.xml | 2 +- .../changesets/postgresql_engine_schema.xml | 326 ++++++++++++++++++ .../src/main/resources/changesets/rc.xml | 4 +- .../changesets/remove_pre_pbe_tables.xml | 2 +- .../changesets/rename_iteration_to_index.xml | 2 +- .../resources/changesets/rename_jes_id.xml | 4 +- .../rename_workflow_options_in_metadata.xml | 2 +- .../replace_empty_custom_labels.xml | 2 +- .../restart_and_recover_migration.xml | 12 +- .../changesets/runtime_attributes_table.xml | 2 +- .../src/main/resources/changesets/sge.xml | 4 +- .../sge_job_execution_unique_key.xml | 2 +- .../changesets/standardize_column_names.xml | 6 +- .../standardize_column_names_again.xml | 2 +- .../standardize_column_names_patches.xml | 2 +- .../changesets/sub_workflow_store.xml | 8 +- .../changesets/summary_status_table.xml | 4 +- .../changesets/symbol_iteration_not_null.xml | 2 +- .../changesets/symbol_iteration_null.xml | 2 +- .../changesets/sync_not_null_constraints.xml | 4 +- .../resources/changesets/top_level_output.xml | 4 +- .../resources/changesets/wdl_and_inputs.xml | 4 +- .../workflow_execution_aux_not_null.xml | 2 +- .../changesets/workflow_metadata_summary.xml | 10 +- .../resources/changesets/workflow_options.xml | 2 +- .../resources/changesets/workflow_store.xml | 6 +- .../workflow_store_horizontal_db.xml | 2 +- .../workflow_store_imports_file.xml | 2 +- .../changesets/workflow_store_labels_file.xml | 2 +- .../workflow_store_restarted_column.xml | 4 +- .../workflow_store_state_widening.xml | 2 +- .../workflow_store_type_and_version.xml | 2 +- .../workflow_store_workflow_root_column.xml | 2 +- .../postgresql_metadata_schema.xml | 130 +++++++ .../main/resources/sql_metadata_changelog.xml | 1 + .../slick/MetadataSlickDatabase.scala | 2 +- .../database/slick/SlickDatabase.scala | 41 +++ .../CallCachingDetritusEntryComponent.scala | 4 +- .../CallCachingSimpletonEntryComponent.scala | 4 +- .../tables/CustomLabelEntryComponent.scala | 14 +- .../slick/tables/DriverComponent.scala | 50 ++- .../slick/tables/JobStoreEntryComponent.scala | 4 +- .../JobStoreSimpletonEntryComponent.scala | 4 +- .../slick/tables/MetadataEntryComponent.scala | 5 +- ...orkflowMetadataSummaryEntryComponent.scala | 162 ++++++--- .../tables/WorkflowStoreEntryComponent.scala | 14 +- .../cromwell/database/sql/SqlConverters.scala | 26 +- .../sql/tables/CallCachingDetritusEntry.scala | 4 +- .../tables/CallCachingSimpletonEntry.scala | 4 +- .../database/sql/tables/JobStoreEntry.scala | 4 +- .../sql/tables/JobStoreSimpletonEntry.scala | 4 +- .../database/sql/tables/MetadataEntry.scala | 6 +- .../sql/tables/WorkflowStoreEntry.scala | 14 +- .../database/sql/tables/package.scala | 5 +- docs/Configuring.md | 27 ++ .../CallCachingSlickDatabaseSpec.scala | 2 + .../workflowstore/SqlWorkflowStoreSpec.scala | 2 + project/Dependencies.scala | 9 +- .../cromwell/services/ServicesStoreSpec.scala | 14 +- .../keyvalue/impl/KeyValueDatabaseSpec.scala | 6 + .../impl/MetadataDatabaseAccessSpec.scala | 2 + src/ci/bin/test.inc.sh | 2 +- 106 files changed, 946 insertions(+), 250 deletions(-) create mode 100644 database/migration/src/main/resources/changesets/postgresql_engine_schema.xml create mode 100644 database/migration/src/main/resources/metadata_changesets/postgresql_metadata_schema.xml diff --git a/.travis.yml b/.travis.yml index 724c2eba532..9971664b845 100644 --- a/.travis.yml +++ b/.travis.yml @@ -54,6 +54,9 @@ env: - >- BUILD_TYPE=centaurLocal BUILD_MYSQL=5.7 + - >- + BUILD_TYPE=centaurLocal + BUILD_POSTGRESQL=11.3 - >- BUILD_TYPE=centaurPapiV1 BUILD_MYSQL=5.7 diff --git a/CHANGELOG.md b/CHANGELOG.md index 09dd1bd6e54..ddf68b64e27 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -55,6 +55,13 @@ YAML](https://cromwell.readthedocs.io/en/stable/Configuring/#yaml) for more info * It is now possible to use `includeKey` and `excludeKey` at the same time. If so, the metadata key must match the `includeKey` **and not** match the `excludeKey` to be included. * It is now possible to use "`calls`" as one of your `excludeKey`s, to request that only workflow metadata gets returned. +### PostgreSQL support + +Cromwell now supports PostgreSQL (version 9.6 or higher, with the Large Object +extension installed) as a database backend. +See [here](https://cromwell.readthedocs.io/en/stable/Configuring/#database) for +instructions for configuring the database connection. + ## 42 Release Notes ### Womtool endpoint diff --git a/core/src/test/resources/application.conf b/core/src/test/resources/application.conf index 6d479c92e23..22ce88ed919 100644 --- a/core/src/test/resources/application.conf +++ b/core/src/test/resources/application.conf @@ -63,13 +63,13 @@ database-test-mariadb { } } -database-test-postgres { +database-test-postgresql { # Run the following to (optionally) drop and (re-)create the database: # psql postgres <<< 'drop database if exists cromwell_test; create database cromwell_test;' profile = "slick.jdbc.PostgresProfile$" db { driver = "org.postgresql.Driver" - url = "jdbc:postgresql://localhost:5432/cromwell_test" + url = "jdbc:postgresql://localhost:5432/cromwell_test?reWriteBatchedInserts=true" url = ${?CROMWELL_BUILD_POSTGRES_JDBC_URL} user = "cromwell" user = ${?CROMWELL_BUILD_POSTGRES_USERNAME} diff --git a/cromwell.example.backends/cromwell.examples.conf b/cromwell.example.backends/cromwell.examples.conf index 4318bce04bd..243b32d4210 100644 --- a/cromwell.example.backends/cromwell.examples.conf +++ b/cromwell.example.backends/cromwell.examples.conf @@ -630,4 +630,17 @@ database { # connectionTimeout = 3000 # } #} + + # Postgresql example + #database { + # profile = "slick.jdbc.PostgresProfile$" + # db { + # driver = "org.postgresql.Driver" + # url = "jdbc:postgresql://localhost:5432/cromwell" + # user = "" + # password = "" + # port = 5432 + # connectionTimeout = 5000 + # } + #} } diff --git a/database/migration/src/main/resources/changelog.xml b/database/migration/src/main/resources/changelog.xml index 5b755a12cba..2accb5c6f94 100644 --- a/database/migration/src/main/resources/changelog.xml +++ b/database/migration/src/main/resources/changelog.xml @@ -77,6 +77,7 @@ + - + - + - + - + Adding some tracking columns for determining eligibility for Call Result Caching. @@ -38,4 +38,4 @@ constraintName="FK_RESULTS_CLONED_FROM" onDelete="SET NULL" /> - \ No newline at end of file + diff --git a/database/migration/src/main/resources/changesets/callcaching.xml b/database/migration/src/main/resources/changesets/callcaching.xml index 69f87ef6554..391c7a6d26e 100644 --- a/database/migration/src/main/resources/changesets/callcaching.xml +++ b/database/migration/src/main/resources/changesets/callcaching.xml @@ -6,7 +6,7 @@ - + One row per cached job result. Stores meta info about which job the result came from. @@ -41,12 +41,12 @@ - + - + One row per hashkey per call cache meta info. Allows us to link hash keys and values to any matching call cache results. @@ -71,13 +71,13 @@ - + - + - + One row per result simpleton in the job result. Simpleton: a single non-complex WDL value. @@ -115,13 +115,13 @@ - + - + - + Change unique constraint for Execution Table to include IDX column. For MySQL this requires first dropping the foreign key constraint, which we then restore after adding back the enhanced diff --git a/database/migration/src/main/resources/changesets/change_max_size_label_entry.xml b/database/migration/src/main/resources/changesets/change_max_size_label_entry.xml index dd82e754bd3..22bf0f6f760 100644 --- a/database/migration/src/main/resources/changesets/change_max_size_label_entry.xml +++ b/database/migration/src/main/resources/changesets/change_max_size_label_entry.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/change_max_size_workflow_url.xml b/database/migration/src/main/resources/changesets/change_max_size_workflow_url.xml index b549371b65d..d9512d4626a 100644 --- a/database/migration/src/main/resources/changesets/change_max_size_workflow_url.xml +++ b/database/migration/src/main/resources/changesets/change_max_size_workflow_url.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/custom_label_entry.xml b/database/migration/src/main/resources/changesets/custom_label_entry.xml index 49f09da8534..a207982f0bc 100644 --- a/database/migration/src/main/resources/changesets/custom_label_entry.xml +++ b/database/migration/src/main/resources/changesets/custom_label_entry.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + @@ -20,7 +20,7 @@ - + - + - + diff --git a/database/migration/src/main/resources/changesets/db_schema.xml b/database/migration/src/main/resources/changesets/db_schema.xml index 6fa0c397eee..1aad77dd3d7 100644 --- a/database/migration/src/main/resources/changesets/db_schema.xml +++ b/database/migration/src/main/resources/changesets/db_schema.xml @@ -2,7 +2,7 @@ - + @@ -104,7 +104,7 @@ - + @@ -131,7 +131,7 @@ - + diff --git a/database/migration/src/main/resources/changesets/docker_hash_store.xml b/database/migration/src/main/resources/changesets/docker_hash_store.xml index 7ec9fbbc4ff..37b3971daf1 100644 --- a/database/migration/src/main/resources/changesets/docker_hash_store.xml +++ b/database/migration/src/main/resources/changesets/docker_hash_store.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + Temporary storage area for docker hashes from workflows that are still in progress. @@ -26,7 +26,7 @@ - + - + Add a size column corresponding to the sum of all the layers size from the manifest diff --git a/database/migration/src/main/resources/changesets/drop_workflow_uri_and_local_command.xml b/database/migration/src/main/resources/changesets/drop_workflow_uri_and_local_command.xml index 645a496c41b..863c31733ed 100644 --- a/database/migration/src/main/resources/changesets/drop_workflow_uri_and_local_command.xml +++ b/database/migration/src/main/resources/changesets/drop_workflow_uri_and_local_command.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + Workflow URI is not needed in the DB. Local jobs don't need to store the command either. diff --git a/database/migration/src/main/resources/changesets/embiggen_detritus_value.xml b/database/migration/src/main/resources/changesets/embiggen_detritus_value.xml index 096f17a4c63..d1c9c397b57 100644 --- a/database/migration/src/main/resources/changesets/embiggen_detritus_value.xml +++ b/database/migration/src/main/resources/changesets/embiggen_detritus_value.xml @@ -6,7 +6,7 @@ - + diff --git a/database/migration/src/main/resources/changesets/embiggen_metadata_value.xml b/database/migration/src/main/resources/changesets/embiggen_metadata_value.xml index e0b1ed38d85..d965a9b9ee1 100644 --- a/database/migration/src/main/resources/changesets/embiggen_metadata_value.xml +++ b/database/migration/src/main/resources/changesets/embiggen_metadata_value.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + @@ -25,7 +25,7 @@ - + @@ -36,7 +36,7 @@ - + diff --git a/database/migration/src/main/resources/changesets/encrypt_and_clear_workflow_options.xml b/database/migration/src/main/resources/changesets/encrypt_and_clear_workflow_options.xml index 2fe2d766100..e44422d1fda 100644 --- a/database/migration/src/main/resources/changesets/encrypt_and_clear_workflow_options.xml +++ b/database/migration/src/main/resources/changesets/encrypt_and_clear_workflow_options.xml @@ -3,11 +3,11 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + - + diff --git a/database/migration/src/main/resources/changesets/enlarge_call_caching_hash_entry_id.xml b/database/migration/src/main/resources/changesets/enlarge_call_caching_hash_entry_id.xml index 9e69c154730..e875f45dc0d 100644 --- a/database/migration/src/main/resources/changesets/enlarge_call_caching_hash_entry_id.xml +++ b/database/migration/src/main/resources/changesets/enlarge_call_caching_hash_entry_id.xml @@ -8,7 +8,7 @@ could both alter the datatype of and add autoincrement to the PK in one shot. The changeset cannot be renamed as the name is part of the Liquibase key that will prevent it from running again in environments that already suffered through the old two-changeset migration. --> - + diff --git a/database/migration/src/main/resources/changesets/events_table.xml b/database/migration/src/main/resources/changesets/events_table.xml index 47014b6f8fb..681ba71c452 100644 --- a/database/migration/src/main/resources/changesets/events_table.xml +++ b/database/migration/src/main/resources/changesets/events_table.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/execution_backend_info.xml b/database/migration/src/main/resources/changesets/execution_backend_info.xml index 1368b9d1b4e..551112a6321 100644 --- a/database/migration/src/main/resources/changesets/execution_backend_info.xml +++ b/database/migration/src/main/resources/changesets/execution_backend_info.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + @@ -20,14 +20,14 @@ - + - + @@ -35,7 +35,7 @@ - + insert into EXECUTION_INFO(EXECUTION_ID, INFO_KEY, INFO_VALUE) select EXECUTION_ID, "JES_RUN_ID", JES_RUN_ID from JES_JOB; @@ -51,7 +51,7 @@ - + update EXECUTION e set BACKEND_TYPE = 'JES' where exists (select 1 from JES_JOB jj where jj.EXECUTION_ID = e.EXECUTION_ID); @@ -64,19 +64,19 @@ - + - + - + diff --git a/database/migration/src/main/resources/changesets/failure_table.xml b/database/migration/src/main/resources/changesets/failure_table.xml index 295e8d366ba..c00a73edc5b 100644 --- a/database/migration/src/main/resources/changesets/failure_table.xml +++ b/database/migration/src/main/resources/changesets/failure_table.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/jes_id_update.xml b/database/migration/src/main/resources/changesets/jes_id_update.xml index cbf10fd7f50..b28ec05ac0f 100644 --- a/database/migration/src/main/resources/changesets/jes_id_update.xml +++ b/database/migration/src/main/resources/changesets/jes_id_update.xml @@ -2,7 +2,7 @@ - + - + Temporary storage area for completed jobs which belong to workflows that are still in progress. @@ -49,14 +49,14 @@ - + - + diff --git a/database/migration/src/main/resources/changesets/job_store_simpletons.xml b/database/migration/src/main/resources/changesets/job_store_simpletons.xml index a251021bfb0..d6e60308018 100644 --- a/database/migration/src/main/resources/changesets/job_store_simpletons.xml +++ b/database/migration/src/main/resources/changesets/job_store_simpletons.xml @@ -6,7 +6,7 @@ - + One row per result simpleton in the job result. Simpleton: a single non-complex WDL value. @@ -35,13 +35,13 @@ - + - + - + There is no attempt at migrating the contents of JOB_STORE.JOB_OUTPUTS to simpletons, this just removes the column. diff --git a/database/migration/src/main/resources/changesets/job_store_tinyints.xml b/database/migration/src/main/resources/changesets/job_store_tinyints.xml index 1e91a696f09..cfe263d18bc 100644 --- a/database/migration/src/main/resources/changesets/job_store_tinyints.xml +++ b/database/migration/src/main/resources/changesets/job_store_tinyints.xml @@ -3,23 +3,23 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + - + - + - + diff --git a/database/migration/src/main/resources/changesets/lengthen_wdl_value.xml b/database/migration/src/main/resources/changesets/lengthen_wdl_value.xml index a138d58e30c..6595fe7f194 100644 --- a/database/migration/src/main/resources/changesets/lengthen_wdl_value.xml +++ b/database/migration/src/main/resources/changesets/lengthen_wdl_value.xml @@ -4,7 +4,7 @@ xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + WDL_VALUE should accept large strings diff --git a/database/migration/src/main/resources/changesets/local_job_allow_null.xml b/database/migration/src/main/resources/changesets/local_job_allow_null.xml index a4b564f1eeb..81598f88da6 100644 --- a/database/migration/src/main/resources/changesets/local_job_allow_null.xml +++ b/database/migration/src/main/resources/changesets/local_job_allow_null.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + The local jobs don't have process ID and result codes at the start. diff --git a/database/migration/src/main/resources/changesets/metadata_journal.xml b/database/migration/src/main/resources/changesets/metadata_journal.xml index 0eeedc27a5a..99ba1d9382f 100644 --- a/database/migration/src/main/resources/changesets/metadata_journal.xml +++ b/database/migration/src/main/resources/changesets/metadata_journal.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + @@ -36,14 +36,14 @@ - + - + @@ -53,7 +53,7 @@ - + diff --git a/database/migration/src/main/resources/changesets/metadata_journal_subsecond_timestamp.xml b/database/migration/src/main/resources/changesets/metadata_journal_subsecond_timestamp.xml index ae463a19225..d1751af3d9a 100644 --- a/database/migration/src/main/resources/changesets/metadata_journal_subsecond_timestamp.xml +++ b/database/migration/src/main/resources/changesets/metadata_journal_subsecond_timestamp.xml @@ -3,14 +3,14 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + - + - + diff --git a/database/migration/src/main/resources/changesets/move_sql_metadata_changelog.xml b/database/migration/src/main/resources/changesets/move_sql_metadata_changelog.xml index dc6c81b59cc..2edd9137aad 100644 --- a/database/migration/src/main/resources/changesets/move_sql_metadata_changelog.xml +++ b/database/migration/src/main/resources/changesets/move_sql_metadata_changelog.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + SELECT COUNT(1) FROM METADATA_ENTRY diff --git a/database/migration/src/main/resources/changesets/nullable_lobs.xml b/database/migration/src/main/resources/changesets/nullable_lobs.xml index 13783585c79..9f977a0aed6 100644 --- a/database/migration/src/main/resources/changesets/nullable_lobs.xml +++ b/database/migration/src/main/resources/changesets/nullable_lobs.xml @@ -2,7 +2,7 @@ - + - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/database/migration/src/main/resources/changesets/rc.xml b/database/migration/src/main/resources/changesets/rc.xml index 5f03d69ea35..b0425f5e42d 100644 --- a/database/migration/src/main/resources/changesets/rc.xml +++ b/database/migration/src/main/resources/changesets/rc.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + Refactor the RC column off LOCAL_JOB up into EXECUTION since it should be usable by all backends. @@ -11,4 +11,4 @@ - \ No newline at end of file + diff --git a/database/migration/src/main/resources/changesets/remove_pre_pbe_tables.xml b/database/migration/src/main/resources/changesets/remove_pre_pbe_tables.xml index e2892b91b93..613eb7bb2ab 100644 --- a/database/migration/src/main/resources/changesets/remove_pre_pbe_tables.xml +++ b/database/migration/src/main/resources/changesets/remove_pre_pbe_tables.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + Remove the old pre-pluggable backend tables. diff --git a/database/migration/src/main/resources/changesets/rename_iteration_to_index.xml b/database/migration/src/main/resources/changesets/rename_iteration_to_index.xml index a85483f7908..afd71b2ae83 100644 --- a/database/migration/src/main/resources/changesets/rename_iteration_to_index.xml +++ b/database/migration/src/main/resources/changesets/rename_iteration_to_index.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + - + - \ No newline at end of file + diff --git a/database/migration/src/main/resources/changesets/rename_workflow_options_in_metadata.xml b/database/migration/src/main/resources/changesets/rename_workflow_options_in_metadata.xml index b189e535a8c..7f1cff48ca6 100644 --- a/database/migration/src/main/resources/changesets/rename_workflow_options_in_metadata.xml +++ b/database/migration/src/main/resources/changesets/rename_workflow_options_in_metadata.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/replace_empty_custom_labels.xml b/database/migration/src/main/resources/changesets/replace_empty_custom_labels.xml index 508497a8912..da7b346d3e0 100644 --- a/database/migration/src/main/resources/changesets/replace_empty_custom_labels.xml +++ b/database/migration/src/main/resources/changesets/replace_empty_custom_labels.xml @@ -7,7 +7,7 @@ - + - + Restart/recover migration from 0.19 to 0.21. @@ -31,7 +31,7 @@ - + Restart/recover migration from 0.19 to 0.21. @@ -76,7 +76,7 @@ - + @@ -93,14 +93,14 @@ columnDataType="LONGTEXT"/> - + Restart/recover migration from 0.19 to 0.21. - + Restart/recover migration from 0.19 to 0.21. @@ -130,7 +130,7 @@ - + Restart/recover migration from 0.19 to 0.21. diff --git a/database/migration/src/main/resources/changesets/runtime_attributes_table.xml b/database/migration/src/main/resources/changesets/runtime_attributes_table.xml index 4b4cfd368cc..a5ea9bb4611 100644 --- a/database/migration/src/main/resources/changesets/runtime_attributes_table.xml +++ b/database/migration/src/main/resources/changesets/runtime_attributes_table.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/sge.xml b/database/migration/src/main/resources/changesets/sge.xml index a3bca146662..aab8faa4b92 100644 --- a/database/migration/src/main/resources/changesets/sge.xml +++ b/database/migration/src/main/resources/changesets/sge.xml @@ -2,7 +2,7 @@ - + @@ -21,7 +21,7 @@ - + diff --git a/database/migration/src/main/resources/changesets/sge_job_execution_unique_key.xml b/database/migration/src/main/resources/changesets/sge_job_execution_unique_key.xml index 71aa97435db..7a362016bc2 100644 --- a/database/migration/src/main/resources/changesets/sge_job_execution_unique_key.xml +++ b/database/migration/src/main/resources/changesets/sge_job_execution_unique_key.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + Adds unique constraints UK_SGE_JOB_EXECUTION_UUID. diff --git a/database/migration/src/main/resources/changesets/standardize_column_names.xml b/database/migration/src/main/resources/changesets/standardize_column_names.xml index a309c8ea28b..50d26c429b4 100644 --- a/database/migration/src/main/resources/changesets/standardize_column_names.xml +++ b/database/migration/src/main/resources/changesets/standardize_column_names.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + Change all Workflow UUID column names to Workflow Execution UUID. @@ -21,7 +21,7 @@ tableName="JOB_STORE"/> - + Choose and implement common call/job identifiers. @@ -39,4 +39,4 @@ tableName="METADATA_JOURNAL"/> - \ No newline at end of file + diff --git a/database/migration/src/main/resources/changesets/standardize_column_names_again.xml b/database/migration/src/main/resources/changesets/standardize_column_names_again.xml index 111bcd74fed..a98e35f3448 100644 --- a/database/migration/src/main/resources/changesets/standardize_column_names_again.xml +++ b/database/migration/src/main/resources/changesets/standardize_column_names_again.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/workflow_store_state_widening.xml b/database/migration/src/main/resources/changesets/workflow_store_state_widening.xml index 157fd43bd3b..5d3b37bce87 100644 --- a/database/migration/src/main/resources/changesets/workflow_store_state_widening.xml +++ b/database/migration/src/main/resources/changesets/workflow_store_state_widening.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/workflow_store_type_and_version.xml b/database/migration/src/main/resources/changesets/workflow_store_type_and_version.xml index bdb38ad09ed..59876e1dffa 100644 --- a/database/migration/src/main/resources/changesets/workflow_store_type_and_version.xml +++ b/database/migration/src/main/resources/changesets/workflow_store_type_and_version.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/changesets/workflow_store_workflow_root_column.xml b/database/migration/src/main/resources/changesets/workflow_store_workflow_root_column.xml index dab1df1b71e..7358ffd4347 100644 --- a/database/migration/src/main/resources/changesets/workflow_store_workflow_root_column.xml +++ b/database/migration/src/main/resources/changesets/workflow_store_workflow_root_column.xml @@ -3,7 +3,7 @@ xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://www.liquibase.org/xml/ns/dbchangelog http://www.liquibase.org/xml/ns/dbchangelog/dbchangelog-3.3.xsd"> - + diff --git a/database/migration/src/main/resources/metadata_changesets/postgresql_metadata_schema.xml b/database/migration/src/main/resources/metadata_changesets/postgresql_metadata_schema.xml new file mode 100644 index 00000000000..d1c5835954a --- /dev/null +++ b/database/migration/src/main/resources/metadata_changesets/postgresql_metadata_schema.xml @@ -0,0 +1,130 @@ + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + diff --git a/database/migration/src/main/resources/sql_metadata_changelog.xml b/database/migration/src/main/resources/sql_metadata_changelog.xml index 05500b36bc5..809942372cc 100644 --- a/database/migration/src/main/resources/sql_metadata_changelog.xml +++ b/database/migration/src/main/resources/sql_metadata_changelog.xml @@ -11,5 +11,6 @@ + diff --git a/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala b/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala index 622e62aaf33..d28801e7fb5 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/MetadataSlickDatabase.scala @@ -34,7 +34,7 @@ class MetadataSlickDatabase(originalDatabaseConfig: Config) override def addMetadataEntries(metadataEntries: Iterable[MetadataEntry]) (implicit ec: ExecutionContext): Future[Unit] = { val action = DBIO.seq(metadataEntries.grouped(insertBatchSize).map(dataAccess.metadataEntries ++= _).toSeq:_*) - runAction(action) + runLobAction(action) } override def metadataEntryExists(workflowExecutionUuid: String)(implicit ec: ExecutionContext): Future[Boolean] = { diff --git a/database/sql/src/main/scala/cromwell/database/slick/SlickDatabase.scala b/database/sql/src/main/scala/cromwell/database/slick/SlickDatabase.scala index 725ee999451..b1d68f5c590 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/SlickDatabase.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/SlickDatabase.scala @@ -8,6 +8,7 @@ import com.typesafe.config.{Config, ConfigFactory} import cromwell.database.slick.tables.DataAccessComponent import cromwell.database.sql.SqlDatabase import net.ceedubs.ficus.Ficus._ +import org.postgresql.util.{PSQLException, ServerErrorMessage} import org.slf4j.LoggerFactory import slick.basic.DatabaseConfig import slick.jdbc.{JdbcCapabilities, JdbcProfile, TransactionIsolation} @@ -58,6 +59,7 @@ abstract class SlickDatabase(override val originalDatabaseConfig: Config) extend override val urlKey = SlickDatabase.urlKey(originalDatabaseConfig) protected val slickConfig = DatabaseConfig.forConfig[JdbcProfile]("", databaseConfig) + lazy val isPostgresql = databaseConfig.getOrElse("db.driver", "unknown") == "org.postgresql.Driver" /* Not a def because we need to have a "stable identifier" for the imports below. @@ -167,10 +169,22 @@ abstract class SlickDatabase(override val originalDatabaseConfig: Config) extend runActionInternal(action.transactionally.withTransactionIsolation(isolationLevel)) } + /* Note that this is only appropriate for actions that do not involve Blob + * or Clob fields in Postgres, since large object support requires running + * transactionally. Use runLobAction instead, which will still run in + * auto-commit mode when using other database engines. + */ protected[this] def runAction[R](action: DBIO[R]): Future[R] = { runActionInternal(action.withPinnedSession) } + /* Wrapper for queries where Clob/Blob types are used + * https://stackoverflow.com/questions/3164072/large-objects-may-not-be-used-in-auto-commit-mode#answer-3164352 + */ + protected[this] def runLobAction[R](action: DBIO[R]): Future[R] = { + if (isPostgresql) runTransaction(action) else runAction(action) + } + private def runActionInternal[R](action: DBIO[R]): Future[R] = { //database.run(action) <-- See comment above private val actionThreadPool Future { @@ -186,6 +200,33 @@ abstract class SlickDatabase(override val originalDatabaseConfig: Config) extend case _ => /* keep going */ } throw rollbackException + case pSQLException: PSQLException => + val detailOption = for { + message <- Option(pSQLException.getServerErrorMessage) + detail <- Option(message.getDetail) + } yield detail + + detailOption match { + case None => throw pSQLException + case Some(_) => + /* + The exception may contain possibly sensitive row contents within the DETAIL section. Remove it. + + Tried adjusting this using configuration: + - log_error_verbosity=TERSE + - log_min_messages=PANIC + - client_min_messages=ERROR + + Instead resorting to reflection. + */ + val message = pSQLException.getServerErrorMessage + val field = classOf[ServerErrorMessage].getDeclaredField("m_mesgParts") + field.setAccessible(true) + val parts = field.get(message).asInstanceOf[java.util.Map[Character, String]] + parts.remove('D') + // The original exception has already stored the DETAIL into a string. So we must create a new Exception. + throw new PSQLException(message) + } } }(actionExecutionContext) } diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingDetritusEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingDetritusEntryComponent.scala index c36b8047d11..db7c1f3d826 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingDetritusEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingDetritusEntryComponent.scala @@ -1,6 +1,6 @@ package cromwell.database.slick.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob import cromwell.database.sql.tables.CallCachingDetritusEntry @@ -16,7 +16,7 @@ trait CallCachingDetritusEntryComponent { def detritusKey = column[String]("DETRITUS_KEY", O.Length(255)) - def detritusValue = column[Option[Clob]]("DETRITUS_VALUE") + def detritusValue = column[Option[SerialClob]]("DETRITUS_VALUE") def callCachingEntryId = column[Int]("CALL_CACHING_ENTRY_ID") diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingSimpletonEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingSimpletonEntryComponent.scala index 7170ae9025c..38a095a9682 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingSimpletonEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/CallCachingSimpletonEntryComponent.scala @@ -1,6 +1,6 @@ package cromwell.database.slick.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob import cromwell.database.sql.tables.CallCachingSimpletonEntry @@ -16,7 +16,7 @@ trait CallCachingSimpletonEntryComponent { def simpletonKey = column[String]("SIMPLETON_KEY", O.Length(255)) - def simpletonValue = column[Option[Clob]]("SIMPLETON_VALUE") + def simpletonValue = column[Option[SerialClob]]("SIMPLETON_VALUE") def wdlType = column[String]("WDL_TYPE", O.Length(255)) diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/CustomLabelEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/CustomLabelEntryComponent.scala index 8121b3705ae..153b6815810 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/CustomLabelEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/CustomLabelEntryComponent.scala @@ -1,12 +1,14 @@ package cromwell.database.slick.tables import cromwell.database.sql.tables.CustomLabelEntry +import shapeless.syntax.std.tuple._ import slick.model.ForeignKeyAction.Cascade trait CustomLabelEntryComponent { this: DriverComponent with WorkflowMetadataSummaryEntryComponent => + import driver.api.TupleMethods._ import driver.api._ class CustomLabelEntries(tag: Tag) @@ -19,8 +21,14 @@ trait CustomLabelEntryComponent { def workflowExecutionUuid = column[String]("WORKFLOW_EXECUTION_UUID", O.Length(100)) - override def * = (customLabelKey, customLabelValue, workflowExecutionUuid, - customLabelEntryId.?) <> (CustomLabelEntry.tupled, CustomLabelEntry.unapply) + def baseProjection = (customLabelKey, customLabelValue, workflowExecutionUuid) + + override def * = baseProjection ~ customLabelEntryId.? <> (CustomLabelEntry.tupled, CustomLabelEntry.unapply) + + def forUpdate = baseProjection.shaped <> ( + tuple => CustomLabelEntry.tupled(tuple :+ None), + CustomLabelEntry.unapply(_: CustomLabelEntry).map(_.reverse.tail.reverse) + ) def fkCustomLabelEntryWorkflowExecutionUuid = foreignKey("FK_CUSTOM_LABEL_ENTRY_WORKFLOW_EXECUTION_UUID", workflowExecutionUuid, workflowMetadataSummaryEntries)(_.workflowExecutionUuid, onDelete = Cascade) @@ -41,7 +49,7 @@ trait CustomLabelEntryComponent { customLabelEntry <- customLabelEntries if customLabelEntry.workflowExecutionUuid === workflowExecutionUuid && customLabelEntry.customLabelKey === labelKey - } yield customLabelEntry) + } yield customLabelEntry.forUpdate) def existsWorkflowIdLabelKeyAndValue(workflowId: Rep[String], labelKey: Rep[String], diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/DriverComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/DriverComponent.scala index d5f78601862..f9343883f4d 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/DriverComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/DriverComponent.scala @@ -1,7 +1,55 @@ package cromwell.database.slick.tables -import slick.jdbc.JdbcProfile +import java.sql.{Blob, Clob} + +import javax.sql.rowset.serial.{SerialBlob, SerialClob} +import org.apache.commons.io.IOUtils +import slick.jdbc.{JdbcProfile, PostgresProfile} trait DriverComponent { val driver: JdbcProfile + + import driver.api._ + + /** Ensure clobs are retrieved inside the transaction, not after */ + implicit val serialClobColumnType = MappedColumnType.base[SerialClob, Clob]( + identity, + { + case serialClob: SerialClob => serialClob + case clob => + /* + PostgreSQL's JDBC driver has issues with non-ascii characters. + https://stackoverflow.com/questions/5043992/postgres-utf-8-clobs-with-jdbc + + It returns bad values for length() and getAsciiStream(), and causes an extra null bytes to be added at the end + of the resultant SerialClob. + + Example via copy_workflow_outputs/unscattered.wdl: + + "... Enfin un peu de francais pour contrer ce raz-de-marée anglais ! ..." + + The 'é' in results in an extra null byte at the end of getAsciiStream(). + */ + val string = IOUtils.toString(clob.getCharacterStream) + new SerialClob(string.toCharArray) + } + ) + + /** Ensure clobs are retrieved inside the transaction, not after */ + implicit val serialBlobColumnType = MappedColumnType.base[SerialBlob, Blob]( + identity, + { + case serialBlob: SerialBlob => serialBlob + case blob => new SerialBlob(blob) + } + ) + + private val shouldQuote = this.driver match { + // https://stackoverflow.com/questions/43111996/why-postgresql-does-not-like-uppercase-table-names#answer-43112096 + case PostgresProfile => true + case _ => false + } + + /** Adds quotes around the string if required by the DBMS. */ + def quoted(string: String) = if (shouldQuote) s""""$string"""" else string } diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreEntryComponent.scala index 6422e535ef7..de5bfe57698 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreEntryComponent.scala @@ -1,6 +1,6 @@ package cromwell.database.slick.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob import cromwell.database.sql.tables.JobStoreEntry @@ -26,7 +26,7 @@ trait JobStoreEntryComponent { def returnCode = column[Option[Int]]("RETURN_CODE") // Only set for failure: - def exceptionMessage = column[Option[Clob]]("EXCEPTION_MESSAGE") + def exceptionMessage = column[Option[SerialClob]]("EXCEPTION_MESSAGE") def retryableFailure = column[Option[Boolean]]("RETRYABLE_FAILURE") diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreSimpletonEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreSimpletonEntryComponent.scala index 40d3f094ea3..e2e9c83dac9 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreSimpletonEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/JobStoreSimpletonEntryComponent.scala @@ -1,6 +1,6 @@ package cromwell.database.slick.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob import cromwell.database.sql.tables.JobStoreSimpletonEntry import slick.model.ForeignKeyAction.Cascade @@ -16,7 +16,7 @@ trait JobStoreSimpletonEntryComponent { def simpletonKey = column[String]("SIMPLETON_KEY", O.Length(255)) - def simpletonValue = column[Option[Clob]]("SIMPLETON_VALUE") + def simpletonValue = column[Option[SerialClob]]("SIMPLETON_VALUE") def wdlType = column[String]("WDL_TYPE", O.Length(255)) diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala index 9e8106aebd4..6e440fdeebf 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/MetadataEntryComponent.scala @@ -1,6 +1,7 @@ package cromwell.database.slick.tables -import java.sql.{Clob, Timestamp} +import java.sql.Timestamp +import javax.sql.rowset.serial.SerialClob import cromwell.database.sql.tables.MetadataEntry @@ -35,7 +36,7 @@ trait MetadataEntryComponent { def metadataKey = column[String]("METADATA_KEY", O.Length(255)) - def metadataValue = column[Option[Clob]]("METADATA_VALUE") + def metadataValue = column[Option[SerialClob]]("METADATA_VALUE") def metadataValueType = column[Option[String]]("METADATA_VALUE_TYPE", O.Length(10)) diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowMetadataSummaryEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowMetadataSummaryEntryComponent.scala index 14721104971..d5c5273d245 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowMetadataSummaryEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowMetadataSummaryEntryComponent.scala @@ -1,11 +1,11 @@ package cromwell.database.slick.tables import java.sql.Timestamp -import java.util.concurrent.atomic.AtomicInteger import cats.data import cats.data.NonEmptyList import cromwell.database.sql.tables.WorkflowMetadataSummaryEntry +import shapeless.syntax.std.tuple._ import slick.jdbc.{GetResult, PositionedParameters, SQLActionBuilder} //noinspection SqlDialectInspection @@ -13,6 +13,7 @@ trait WorkflowMetadataSummaryEntryComponent { this: DriverComponent with CustomLabelEntryComponent with MetadataEntryComponent => + import driver.api.TupleMethods._ import driver.api._ class WorkflowMetadataSummaryEntries(tag: Tag) @@ -35,9 +36,15 @@ trait WorkflowMetadataSummaryEntryComponent { def rootWorkflowExecutionUuid = column[Option[String]]("ROOT_WORKFLOW_EXECUTION_UUID", O.Length(100)) - override def * = (workflowExecutionUuid, workflowName, workflowStatus, startTimestamp, endTimestamp, - submissionTimestamp, parentWorkflowExecutionUuid, rootWorkflowExecutionUuid, - workflowMetadataSummaryEntryId.?) <> (WorkflowMetadataSummaryEntry.tupled, WorkflowMetadataSummaryEntry.unapply) + def baseProjection = (workflowExecutionUuid, workflowName, workflowStatus, startTimestamp, endTimestamp, + submissionTimestamp, parentWorkflowExecutionUuid, rootWorkflowExecutionUuid) + + override def * = baseProjection ~ workflowMetadataSummaryEntryId.? <> (WorkflowMetadataSummaryEntry.tupled, WorkflowMetadataSummaryEntry.unapply) + + def forUpdate = baseProjection.shaped <> ( + tuple => WorkflowMetadataSummaryEntry.tupled(tuple :+ None), + WorkflowMetadataSummaryEntry.unapply(_: WorkflowMetadataSummaryEntry).map(_.reverse.tail.reverse) + ) def ucWorkflowMetadataSummaryEntryWeu = index("UC_WORKFLOW_METADATA_SUMMARY_ENTRY_WEU", workflowExecutionUuid, unique = true) @@ -63,7 +70,7 @@ trait WorkflowMetadataSummaryEntryComponent { (workflowExecutionUuid: Rep[String]) => for { workflowMetadataSummaryEntry <- workflowMetadataSummaryEntries if workflowMetadataSummaryEntry.workflowExecutionUuid === workflowExecutionUuid - } yield workflowMetadataSummaryEntry) + } yield workflowMetadataSummaryEntry.forUpdate) val workflowMetadataSummaryEntryExistsForWorkflowExecutionUuid = Compiled( (workflowExecutionUuid: Rep[String]) => (for { @@ -81,8 +88,8 @@ trait WorkflowMetadataSummaryEntryComponent { def concat(a: SQLActionBuilder, b: SQLActionBuilder): SQLActionBuilder = { SQLActionBuilder(a.queryParts ++ b.queryParts, (p: Unit, pp: PositionedParameters) => { - a.unitPConv.apply(p, pp) - b.unitPConv.apply(p, pp) + a.unitPConv.apply(p, pp) + b.unitPConv.apply(p, pp) }) } @@ -118,79 +125,130 @@ trait WorkflowMetadataSummaryEntryComponent { endTimestampOption: Option[Timestamp], includeSubworkflows: Boolean): SQLActionBuilder = { - val summaryTableAlias = "summaryTable" - val labelsOrTableAlias = "labelsOrMixin" - val labelsAndTableAliases = labelAndKeyLabelValues.zipWithIndex.map { case (labelPair, i) => s"labelAndTable$i" -> labelPair }.toMap + val customLabelEntryTable = quoted("CUSTOM_LABEL_ENTRY") + val workflowMetadataSummaryEntryTable = quoted("WORKFLOW_METADATA_SUMMARY_ENTRY") + + val workflowExecutionUuidColumn = quoted("WORKFLOW_EXECUTION_UUID") + val customLabelKeyColumn = quoted("CUSTOM_LABEL_KEY") + val customLabelValueColumn = quoted("CUSTOM_LABEL_VALUE") + val parentWorkflowExecutionUuidColumn = quoted("PARENT_WORKFLOW_EXECUTION_UUID") + + val summaryTableAlias = quoted("summaryTable") + val labelsOrTableAlias = quoted("labelsOrMixin") + val labelsAndTableAliases = labelAndKeyLabelValues.zipWithIndex.map { + case (labelPair, i) => quoted(s"labelAndTable$i") -> labelPair + }.toMap + + val selectColumns = List( + "WORKFLOW_EXECUTION_UUID", + "WORKFLOW_NAME", + "WORKFLOW_STATUS", + "START_TIMESTAMP", + "END_TIMESTAMP", + "SUBMISSION_TIMESTAMP", + "PARENT_WORKFLOW_EXECUTION_UUID", + "ROOT_WORKFLOW_EXECUTION_UUID", + "WORKFLOW_METADATA_SUMMARY_ENTRY_ID", + ) + .map(quoted) + .mkString(s"$summaryTableAlias.", ", ", "") val select = selectOrCount match { case Select => - sql"""|SELECT #$summaryTableAlias.WORKFLOW_EXECUTION_UUID, - | #$summaryTableAlias.WORKFLOW_NAME, - | #$summaryTableAlias.WORKFLOW_STATUS, - | #$summaryTableAlias.START_TIMESTAMP, - | #$summaryTableAlias.END_TIMESTAMP, - | #$summaryTableAlias.SUBMISSION_TIMESTAMP, - | #$summaryTableAlias.PARENT_WORKFLOW_EXECUTION_UUID, - | #$summaryTableAlias.ROOT_WORKFLOW_EXECUTION_UUID, - | #$summaryTableAlias.WORKFLOW_METADATA_SUMMARY_ENTRY_ID - | """.stripMargin + sql"""|SELECT #$selectColumns + |""".stripMargin case Count => - sql"""SELECT COUNT(1) - | """.stripMargin + sql"""|SELECT COUNT(1) + |""".stripMargin } val labelOrJoin = if (labelOrKeyLabelValues.nonEmpty) { Option( - sql""" JOIN CUSTOM_LABEL_ENTRY #$labelsOrTableAlias on #$summaryTableAlias.WORKFLOW_EXECUTION_UUID = #$labelsOrTableAlias.WORKFLOW_EXECUTION_UUID - | """.stripMargin) + sql"""| JOIN #$customLabelEntryTable #$labelsOrTableAlias + | ON #$summaryTableAlias.#$workflowExecutionUuidColumn + | = #$labelsOrTableAlias.#$workflowExecutionUuidColumn + |""".stripMargin) } else None val labelAndJoins = labelsAndTableAliases.toList.map { case (labelAndTableAlias, _) => - sql""" JOIN CUSTOM_LABEL_ENTRY #$labelAndTableAlias on #$summaryTableAlias.WORKFLOW_EXECUTION_UUID = #$labelAndTableAlias.WORKFLOW_EXECUTION_UUID - | """.stripMargin + sql"""| JOIN #$customLabelEntryTable #$labelAndTableAlias + | ON #$summaryTableAlias.#$workflowExecutionUuidColumn + | = #$labelAndTableAlias.#$workflowExecutionUuidColumn + |""".stripMargin } val from = concatNel(NonEmptyList.of( - sql"""FROM WORKFLOW_METADATA_SUMMARY_ENTRY #$summaryTableAlias - | """.stripMargin) ++ labelOrJoin.toList ++ labelAndJoins ) + sql"""|FROM #$workflowMetadataSummaryEntryTable #$summaryTableAlias + |""".stripMargin) ++ labelOrJoin.toList ++ labelAndJoins) + + def makeSetConstraint(column: String, elements: Set[String]) = { + val list = elements.toList.map(element => sql"""#$summaryTableAlias.#${quoted(column)} = $element""") + NonEmptyList.fromList(list).map(or).toList + } - val statusConstraint = NonEmptyList.fromList(workflowStatuses.toList.map(status => sql"""#$summaryTableAlias.WORKFLOW_STATUS=$status""")).map(or).toList - val nameConstraint = NonEmptyList.fromList(workflowNames.toList.map(name => sql"""#$summaryTableAlias.WORKFLOW_NAME=$name""")).map(or).toList - val idConstraint = NonEmptyList.fromList(workflowExecutionUuids.toList.map(uuid => sql"""#$summaryTableAlias.WORKFLOW_EXECUTION_UUID=$uuid""")).map(or).toList - val submissionTimeConstraint = submissionTimestampOption.map(ts => sql"""#$summaryTableAlias.SUBMISSION_TIMESTAMP>=$ts""").toList - val startTimeConstraint = startTimestampOption.map(ts => sql"""#$summaryTableAlias.START_TIMESTAMP>=$ts""").toList - val endTimeConstraint = endTimestampOption.map(ts => sql"""#$summaryTableAlias.END_TIMESTAMP<=$ts""").toList + def makeTimeConstraint(column: String, comparison: String, elementOption: Option[Timestamp]) = { + elementOption.map(element => sql"""#$summaryTableAlias.#${quoted(column)} #$comparison $element""").toList + } + + val statusConstraint = makeSetConstraint("WORKFLOW_STATUS", workflowStatuses) + val nameConstraint = makeSetConstraint("WORKFLOW_NAME", workflowNames) + val idConstraint = makeSetConstraint("WORKFLOW_EXECUTION_UUID", workflowExecutionUuids) + + val submissionTimeConstraint = makeTimeConstraint("SUBMISSION_TIMESTAMP", ">=", submissionTimestampOption) + val startTimeConstraint = makeTimeConstraint("START_TIMESTAMP", ">=", startTimestampOption) + val endTimeConstraint = makeTimeConstraint("END_TIMESTAMP", "<=", endTimestampOption) // *ALL* of the labelAnd list of KV pairs must exist: - val labelsAndConstraint = NonEmptyList.fromList(labelsAndTableAliases.toList.map { case (labelsAndTableAlias, (labelKey, labelValue)) => - and(NonEmptyList.of(sql"#$labelsAndTableAlias.custom_label_key=$labelKey") :+ sql"#$labelsAndTableAlias.custom_label_value=$labelValue") + val labelsAndConstraint = NonEmptyList.fromList(labelsAndTableAliases.toList.map { + case (labelsAndTableAlias, (labelKey, labelValue)) => + and(NonEmptyList.of( + sql"""#$labelsAndTableAlias.#$customLabelKeyColumn = $labelKey""", + sql"""#$labelsAndTableAlias.#$customLabelValueColumn = $labelValue""", + )) }).map(and).toList // At least one of the labelOr list of KV pairs must exist: - val labelOrConstraint = NonEmptyList.fromList(labelOrKeyLabelValues.toList.map { case (k, v) => - and(NonEmptyList.of(sql"#$labelsOrTableAlias.custom_label_key=$k") :+ sql"#$labelsOrTableAlias.custom_label_value=$v") + val labelOrConstraint = NonEmptyList.fromList(labelOrKeyLabelValues.toList.map { + case (labelKey, labelValue) => + and(NonEmptyList.of( + sql"""#$labelsOrTableAlias.#$customLabelKeyColumn = $labelKey""", + sql"""#$labelsOrTableAlias.#$customLabelValueColumn = $labelValue""", + )) }).map(or).toList - val mixinTableCounter = new AtomicInteger(0) + var mixinTableCounter = 0 def labelExists(labelKey: String, labelValue: String) = { - val tableName = s"labelsMixin" + mixinTableCounter.getAndIncrement() - sql"""EXISTS(SELECT 1 from CUSTOM_LABEL_ENTRY #$tableName WHERE ((#$tableName.WORKFLOW_EXECUTION_UUID = #$summaryTableAlias.WORKFLOW_EXECUTION_UUID) AND (#$tableName.CUSTOM_LABEL_KEY = $labelKey) AND (#$tableName.CUSTOM_LABEL_VALUE = $labelValue)))""" + val tableName = quoted(s"labelsMixin" + mixinTableCounter) + mixinTableCounter += 1 + sql"""|EXISTS ( + | SELECT 1 FROM #$customLabelEntryTable #$tableName + | WHERE ( + | (#$tableName.#$workflowExecutionUuidColumn = #$summaryTableAlias.#$workflowExecutionUuidColumn) + | AND (#$tableName.#$customLabelKeyColumn = $labelKey) + | AND (#$tableName.#$customLabelValueColumn = $labelValue) + | ) + |) + |""".stripMargin } // *ALL* of the excludeLabelOr list of KV pairs must *NOT* exist: - val excludeLabelsOrConstraint = NonEmptyList.fromList(excludeLabelOrValues.toList.map { case (labelKey, labelValue) => not(labelExists(labelKey, labelValue)) } ).map(and).toList + val excludeLabelsOrConstraint = NonEmptyList.fromList(excludeLabelOrValues.toList map { + case (labelKey, labelValue) => not(labelExists(labelKey, labelValue)) + }).map(and).toList // At least one of the excludeLabelAnd list of KV pairs must *NOT* exist: - val excludeLabelsAndConstraint = NonEmptyList.fromList(excludeLabelAndValues.toList.map { case (labelKey, labelValue) => not(labelExists(labelKey, labelValue)) } ).map(or).toList + val excludeLabelsAndConstraint = NonEmptyList.fromList(excludeLabelAndValues.toList.map { + case (labelKey, labelValue) => not(labelExists(labelKey, labelValue)) + }).map(or).toList val includeSubworkflowsConstraint = if (includeSubworkflows) List.empty else { - List(sql"""#$summaryTableAlias.PARENT_WORKFLOW_EXECUTION_UUID IS NULL""".stripMargin) + List(sql"""#$summaryTableAlias.#$parentWorkflowExecutionUuidColumn IS NULL""".stripMargin) } val constraintList = - statusConstraint ++ + statusConstraint ++ nameConstraint ++ idConstraint ++ submissionTimeConstraint ++ @@ -274,18 +332,20 @@ trait WorkflowMetadataSummaryEntryComponent { ) val paginationAddendum: List[SQLActionBuilder] = (page, pageSize) match { - case (Some(p), Some(ps)) => List(sql""" LIMIT #${Integer.max(p-1, 0) * ps},#$ps """) - case (None, Some(ps)) => List(sql""" LIMIT 0,#$ps """) + case (Some(p), Some(ps)) => List(sql""" LIMIT #$ps OFFSET #${ps * ((p - 1) max 0)}""") + case (None, Some(ps)) => List(sql""" LIMIT #$ps OFFSET 0""") case _ => List.empty } - val orderByAddendum = sql""" ORDER BY WORKFLOW_METADATA_SUMMARY_ENTRY_ID DESC - | """.stripMargin + val orderByAddendum = + sql"""| ORDER BY #${quoted("WORKFLOW_METADATA_SUMMARY_ENTRY_ID")} DESC + |""".stripMargin // NB you can preview the prepared statement created here by using, for example: println(result.statements.head) - concatNel((NonEmptyList.of(mainQuery) :+ orderByAddendum) ++ paginationAddendum) - .as[WorkflowMetadataSummaryEntry](rconv = GetResult { r => + val fullQuery = concatNel(NonEmptyList(mainQuery, orderByAddendum :: paginationAddendum)) + + fullQuery.as[WorkflowMetadataSummaryEntry](rconv = GetResult { r => WorkflowMetadataSummaryEntry(r.<<, r.<<, r.<<, r.<<, r.<<, r.<<, r.<<, r.<<, r.<<) }) } diff --git a/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowStoreEntryComponent.scala b/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowStoreEntryComponent.scala index 3d99dafcd0c..d3888c8aa49 100644 --- a/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowStoreEntryComponent.scala +++ b/database/sql/src/main/scala/cromwell/database/slick/tables/WorkflowStoreEntryComponent.scala @@ -1,7 +1,7 @@ package cromwell.database.slick.tables -import java.sql.{Blob, Clob, Timestamp} - +import java.sql.Timestamp +import javax.sql.rowset.serial.{SerialBlob, SerialClob} import cromwell.database.sql.tables.WorkflowStoreEntry trait WorkflowStoreEntryComponent { @@ -21,21 +21,21 @@ trait WorkflowStoreEntryComponent { def workflowTypeVersion = column[Option[String]]("WORKFLOW_TYPE_VERSION", O.Length(255)) - def workflowDefinition = column[Option[Clob]]("WORKFLOW_DEFINITION") + def workflowDefinition = column[Option[SerialClob]]("WORKFLOW_DEFINITION") def workflowUrl = column[Option[String]]("WORKFLOW_URL", O.Length(2000)) - def workflowInputs = column[Option[Clob]]("WORKFLOW_INPUTS") + def workflowInputs = column[Option[SerialClob]]("WORKFLOW_INPUTS") - def workflowOptions = column[Option[Clob]]("WORKFLOW_OPTIONS") + def workflowOptions = column[Option[SerialClob]]("WORKFLOW_OPTIONS") - def customLabels = column[Clob]("CUSTOM_LABELS") + def customLabels = column[SerialClob]("CUSTOM_LABELS") def workflowState = column[String]("WORKFLOW_STATE", O.Length(20)) def submissionTime = column[Timestamp]("SUBMISSION_TIME") - def importsZip = column[Option[Blob]]("IMPORTS_ZIP") + def importsZip = column[Option[SerialBlob]]("IMPORTS_ZIP") def cromwellId = column[Option[String]]("CROMWELL_ID", O.Length(100)) diff --git a/database/sql/src/main/scala/cromwell/database/sql/SqlConverters.scala b/database/sql/src/main/scala/cromwell/database/sql/SqlConverters.scala index 44a7a5e8853..fe1bf5f7dba 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/SqlConverters.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/SqlConverters.scala @@ -32,12 +32,16 @@ object SqlConverters { } implicit class ClobToRawString(val clob: Clob) extends AnyVal { - // yes, it starts at 1 - def toRawString: String = clob.getSubString(1, clob.length.toInt) + def toRawString: String = { + // See notes on empty clob issues in StringToClobOption + val length = clob.length.toInt + // yes, it starts at 1 + if (length == 0) "" else clob.getSubString(1, length) + } } implicit class StringOptionToClobOption(val strOption: Option[String]) extends AnyVal { - def toClobOption: Option[Clob] = strOption.flatMap(_.toClobOption) + def toClobOption: Option[SerialClob] = strOption.flatMap(_.toClobOption) } implicit class StringToClobOption(val str: String) extends AnyVal { @@ -52,17 +56,21 @@ object SqlConverters { import eu.timepit.refined.api.Refined import eu.timepit.refined.collection.NonEmpty - def toClobOption: Option[Clob] = if (str.isEmpty) None else Option(new SerialClob(str.toCharArray)) + def toClobOption: Option[SerialClob] = if (str.isEmpty) None else Option(new SerialClob(str.toCharArray)) - def toClob(default: String Refined NonEmpty): Clob = { + def toClob(default: String Refined NonEmpty): SerialClob = { val nonEmpty = if (str.isEmpty) default.value else str new SerialClob(nonEmpty.toCharArray) } } implicit class BlobToBytes(val blob: Blob) extends AnyVal { - // yes, it starts at 1 - def toBytes: Array[Byte] = blob.getBytes(1, blob.length.toInt) + def toBytes: Array[Byte] = { + // See notes on empty blob issues in BytesOptionToBlob + val length = blob.length.toInt + // yes, it starts at 1 + if (length == 0) Array.empty else blob.getBytes(1, length) + } } implicit class BlobOptionToBytes(val blobOption: Option[Blob]) extends AnyVal { @@ -79,11 +87,11 @@ object SqlConverters { https://github.com/apache/derby/blob/10.13/java/engine/org/apache/derby/iapi/types/HarmonySerialBlob.java#L111 OK! -> https://github.com/arteam/hsqldb/blob/2.3.4/src/org/hsqldb/jdbc/JDBCBlob.java#L184 */ - def toBlobOption: Option[Blob] = bytesOption.flatMap(_.toBlobOption) + def toBlobOption: Option[SerialBlob] = bytesOption.flatMap(_.toBlobOption) } implicit class BytesToBlobOption(val bytes: Array[Byte]) extends AnyVal { - def toBlobOption: Option[Blob] = if (bytes.isEmpty) None else Option(new SerialBlob(bytes)) + def toBlobOption: Option[SerialBlob] = if (bytes.isEmpty) None else Option(new SerialBlob(bytes)) } implicit class EnhancedFiniteDuration(val duration: FiniteDuration) extends AnyVal { diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingDetritusEntry.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingDetritusEntry.scala index 0fd6d87cb15..31e6af183ae 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingDetritusEntry.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingDetritusEntry.scala @@ -1,11 +1,11 @@ package cromwell.database.sql.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob case class CallCachingDetritusEntry ( detritusKey: String, - detritusValue: Option[Clob], + detritusValue: Option[SerialClob], callCachingEntryId: Option[Int] = None, callCachingDetritusEntryId: Option[Int] = None ) diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingSimpletonEntry.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingSimpletonEntry.scala index fb2627ba3be..626246c7bf2 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingSimpletonEntry.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/CallCachingSimpletonEntry.scala @@ -1,11 +1,11 @@ package cromwell.database.sql.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob case class CallCachingSimpletonEntry ( simpletonKey: String, - simpletonValue: Option[Clob], + simpletonValue: Option[SerialClob], wdlType: String, callCachingEntryId: Option[Int] = None, callCachingSimpletonEntryId: Option[Int] = None diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreEntry.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreEntry.scala index bf819e63283..c1a904af3be 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreEntry.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreEntry.scala @@ -1,6 +1,6 @@ package cromwell.database.sql.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob case class JobStoreEntry ( @@ -10,7 +10,7 @@ case class JobStoreEntry jobAttempt: Int, jobSuccessful: Boolean, returnCode: Option[Int], - exceptionMessage: Option[Clob], + exceptionMessage: Option[SerialClob], retryableFailure: Option[Boolean], jobStoreEntryId: Option[Int] = None ) diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreSimpletonEntry.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreSimpletonEntry.scala index 909dd17fea2..e0c66921973 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreSimpletonEntry.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/JobStoreSimpletonEntry.scala @@ -1,11 +1,11 @@ package cromwell.database.sql.tables -import java.sql.Clob +import javax.sql.rowset.serial.SerialClob case class JobStoreSimpletonEntry ( simpletonKey: String, - simpletonValue: Option[Clob], + simpletonValue: Option[SerialClob], wdlType: String, jobStoreEntryId: Option[Int] = None, jobStoreSimpletonEntryId: Option[Int] = None diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/MetadataEntry.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/MetadataEntry.scala index fcc4a40006e..c273c3e47e3 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/MetadataEntry.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/MetadataEntry.scala @@ -1,6 +1,8 @@ package cromwell.database.sql.tables -import java.sql.{Clob, Timestamp} +import java.sql.Timestamp + +import javax.sql.rowset.serial.SerialClob case class MetadataEntry ( @@ -9,7 +11,7 @@ case class MetadataEntry jobIndex: Option[Int], jobAttempt: Option[Int], metadataKey: String, - metadataValue: Option[Clob], + metadataValue: Option[SerialClob], metadataValueType: Option[String], metadataTimestamp: Timestamp, metadataEntryId: Option[Long] = None diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/WorkflowStoreEntry.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/WorkflowStoreEntry.scala index 66f22ae12b7..b969939d16c 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/WorkflowStoreEntry.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/WorkflowStoreEntry.scala @@ -1,21 +1,23 @@ package cromwell.database.sql.tables -import java.sql.{Blob, Clob, Timestamp} +import java.sql.Timestamp + +import javax.sql.rowset.serial.{SerialBlob, SerialClob} case class WorkflowStoreEntry ( workflowExecutionUuid: String, - workflowDefinition: Option[Clob], + workflowDefinition: Option[SerialClob], workflowUrl: Option[String], workflowRoot: Option[String], workflowType: Option[String], workflowTypeVersion: Option[String], - workflowInputs: Option[Clob], - workflowOptions: Option[Clob], + workflowInputs: Option[SerialClob], + workflowOptions: Option[SerialClob], workflowState: String, submissionTime: Timestamp, - importsZip: Option[Blob], - customLabels: Clob, + importsZip: Option[SerialBlob], + customLabels: SerialClob, cromwellId: Option[String], heartbeatTimestamp: Option[Timestamp], workflowStoreEntryId: Option[Int] = None diff --git a/database/sql/src/main/scala/cromwell/database/sql/tables/package.scala b/database/sql/src/main/scala/cromwell/database/sql/tables/package.scala index b9421d7b615..9ed1e801abf 100644 --- a/database/sql/src/main/scala/cromwell/database/sql/tables/package.scala +++ b/database/sql/src/main/scala/cromwell/database/sql/tables/package.scala @@ -20,10 +20,11 @@ package cromwell.database.sql * - `Double` * - `Int` * - `Long` - * - `java.sql.Clob` + * - `javax.sql.rowset.serial.SerialClob` * - `java.sql.Timestamp` * - * Nullable columns should be wrapped in an `Option`. + * Nullable columns should be wrapped in an `Option`. Note that SerialClob is + * required instead of java.sql.Clob, for Postgres support. * * Primary and foreign key columns are the only columns that should be defaulted, as they are to be filled in by the * database, and cannot and should not be set within the business logic. On the other hand, columns to be filled in by diff --git a/docs/Configuring.md b/docs/Configuring.md index 08764bf3382..60248d0bbfd 100644 --- a/docs/Configuring.md +++ b/docs/Configuring.md @@ -297,6 +297,33 @@ url = "jdbc:mysql://host/cromwell?rewriteBatchedStatements=true&serverTimezone=U Using this option does not alter your database's underlying timezone; rather, it causes Cromwell to "speak UTC" when communicating with the DB, and the DB server performs the conversion for you. +**Using Cromwell with Postgresql** + +To use Postgresql as the database, you will need to install and enable the +Large Object extension. If the extension is present, setting up the database +requires just these commands: + +``` +$ createdb cromwell +$ psql -d cromwell -c "create extension lo;" +``` + +Postgresql configuration in Cromwell is very similar to MySQL. An example: + +```hocon +database { + profile = "slick.jdbc.PostgresProfile$" + db { + driver = "org.postgresql.Driver" + url = "jdbc:postgresql//localhost:5432/cromwell" + user = "user" + password = "pass" + port = 5432 + connectionTimeout = 5000 + } +} +``` + ### Abort **Control-C (SIGINT) abort handler** diff --git a/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala b/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala index b28c250490b..b2d73e21fcb 100644 --- a/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala +++ b/engine/src/test/scala/cromwell/engine/workflow/lifecycle/execution/callcaching/CallCachingSlickDatabaseSpec.scala @@ -27,6 +27,8 @@ class CallCachingSlickDatabaseSpec extends FlatSpec with Matchers with ScalaFutu "SlickDatabase (mariadb)" should behave like testWith("database-test-mariadb") + "SlickDatabase (postgresql)" should behave like testWith("database-test-postgresql") + def testWith(configPath: String): Unit = { lazy val databaseConfig = ConfigFactory.load.getConfig(configPath) lazy val dataAccess = new EngineSlickDatabase(databaseConfig) diff --git a/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala b/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala index aab401c9934..62c8b3879b2 100644 --- a/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala +++ b/engine/src/test/scala/cromwell/engine/workflow/workflowstore/SqlWorkflowStoreSpec.scala @@ -31,6 +31,8 @@ class SqlWorkflowStoreSpec extends FlatSpec with Matchers with ScalaFutures with "SqlWorkflowStore (mariadb)" should behave like testWith("database-test-mariadb") + "SqlWorkflowStore (postgresql)" should behave like testWith("database-test-postgresql") + def testWith(configPath: String): Unit = { lazy val databaseConfig = ConfigFactory.load.getConfig(configPath) diff --git a/project/Dependencies.scala b/project/Dependencies.scala index e7c23b5a7de..fcebf3a8f76 100644 --- a/project/Dependencies.scala +++ b/project/Dependencies.scala @@ -61,6 +61,7 @@ object Dependencies { private val owlApiV = "5.1.9" private val paradiseV = "2.1.1" private val pegdownV = "1.6.0" + private val postgresV = "42.2.5" private val rdf4jV = "2.4.2" private val refinedV = "0.9.8" private val rhinoV = "1.7.10" @@ -321,7 +322,8 @@ object Dependencies { private val dbmsDependencies = List( "org.hsqldb" % "hsqldb" % hsqldbV, - "mysql" % "mysql-connector-java" % mysqlV + "mysql" % "mysql-connector-java" % mysqlV, + "org.postgresql" % "postgresql" % postgresV ) private val refinedTypeDependenciesList = List( @@ -338,8 +340,9 @@ object Dependencies { val cloudSupportDependencies = googleApiClientDependencies ++ googleCloudDependencies ++ betterFilesDependencies ++ awsCloudDependencies - val databaseSqlDependencies = configDependencies ++ catsDependencies ++ slickDependencies ++ dbmsDependencies ++ - refinedTypeDependenciesList + val databaseSqlDependencies = List( + "commons-io" % "commons-io" % commonsIoV, + ) ++ configDependencies ++ catsDependencies ++ slickDependencies ++ dbmsDependencies ++ refinedTypeDependenciesList val statsDDependencies = List( "nl.grons" %% "metrics-scala" % metrics3ScalaV, diff --git a/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala b/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala index 275398a6609..4106a5e8e62 100644 --- a/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala +++ b/services/src/test/scala/cromwell/services/ServicesStoreSpec.scala @@ -253,6 +253,8 @@ class ServicesStoreSpec extends FlatSpec with Matchers with ScalaFutures with St "SlickDatabase (mariadb)" should behave like testWith("database-test-mariadb") + "SlickDatabase (postgresql)" should behave like testWith("database-test-postgresql") + def testWith(configPath: String): Unit = { import ServicesStore.EnhancedSqlDatabase @@ -300,14 +302,14 @@ class ServicesStoreSpec extends FlatSpec with Matchers with ScalaFutures with St val future = for { product <- dataAccess.database.run(getProduct) _ <- product match { - case "HSQL Database Engine" => - // HSQLDB doesn't crash because it calls getCharacterStream instead of getSubString. - dataAccess.addJobStores(jobStoreJoins, 1) case "MySQL" => + // MySQL crashes because it calls SerialClob's getSubString instead of getCharacterStream dataAccess.addJobStores(jobStoreJoins, 1).failed map { exception => exception should be(a[SerialException]) exception.getMessage should be("Invalid position in SerialClob object set") } + case "HSQL Database Engine" | "PostgreSQL" => + dataAccess.addJobStores(jobStoreJoins, 1) } } yield () @@ -343,15 +345,15 @@ class ServicesStoreSpec extends FlatSpec with Matchers with ScalaFutures with St val future = for { product <- dataAccess.database.run(getProduct) _ <- product match { - case "HSQL Database Engine" => - // HSQLDB doesn't crash because it calls getBinaryStream instead of getBytes. - dataAccess.addWorkflowStoreEntries(workflowStoreEntries) case "MySQL" => + // MySQL crashes because it calls SerialBlob's getBytes instead of getBinaryStream dataAccess.addWorkflowStoreEntries(workflowStoreEntries).failed map { exception => exception should be(a[SerialException]) exception.getMessage should be("Invalid arguments: position cannot be less than 1 or greater than the length of the SerialBlob") } + case _ => + dataAccess.addWorkflowStoreEntries(workflowStoreEntries) } } yield () diff --git a/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala b/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala index 95749caddd7..8641d9e39b5 100644 --- a/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala +++ b/services/src/test/scala/cromwell/services/keyvalue/impl/KeyValueDatabaseSpec.scala @@ -9,6 +9,7 @@ import cromwell.database.slick.EngineSlickDatabase import cromwell.database.sql.tables.JobKeyValueEntry import cromwell.services.EngineServicesStore import cromwell.services.ServicesStore.EnhancedSqlDatabase +import org.postgresql.util.PSQLException import org.scalatest.concurrent.ScalaFutures import org.scalatest.time.{Millis, Seconds, Span} import org.scalatest.{BeforeAndAfterAll, FlatSpec, Matchers, RecoverMethods} @@ -37,6 +38,11 @@ class KeyValueDatabaseSpec extends FlatSpec with Matchers with ScalaFutures with "Column 'STORE_VALUE' cannot be null" ) + "SlickDatabase (postgresql)" should behave like testWith[PSQLException]( + "database-test-postgresql", + """ERROR: null value in column "STORE_VALUE" violates not-null constraint""" + ) + def testWith[E <: Throwable](configPath: String, failureMessage: String)(implicit classTag: ClassTag[E]): Unit = { lazy val databaseConfig = ConfigFactory.load.getConfig(configPath) lazy val dataAccess = new EngineSlickDatabase(databaseConfig) diff --git a/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala b/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala index 27f112d3e1e..2af41be01fb 100644 --- a/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala +++ b/services/src/test/scala/cromwell/services/metadata/impl/MetadataDatabaseAccessSpec.scala @@ -42,6 +42,8 @@ class MetadataDatabaseAccessSpec extends FlatSpec with Matchers with ScalaFuture "MetadataDatabaseAccess (mariadb)" should behave like testWith("database-test-mariadb") + "MetadataDatabaseAccess (postgresql)" should behave like testWith("database-test-postgresql") + implicit val ec = ExecutionContext.global implicit val defaultPatience = PatienceConfig(scaled(Span(30, Seconds)), scaled(Span(100, Millis))) diff --git a/src/ci/bin/test.inc.sh b/src/ci/bin/test.inc.sh index 9749082c628..bbbad2d60bc 100644 --- a/src/ci/bin/test.inc.sh +++ b/src/ci/bin/test.inc.sh @@ -326,7 +326,7 @@ cromwell::private::create_database_variables() { CROMWELL_BUILD_MARIADB_JDBC_URL="jdbc:mysql://${CROMWELL_BUILD_MARIADB_HOSTNAME}:${CROMWELL_BUILD_MARIADB_PORT}/${CROMWELL_BUILD_MARIADB_SCHEMA}?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" CROMWELL_BUILD_MYSQL_JDBC_URL="jdbc:mysql://${CROMWELL_BUILD_MYSQL_HOSTNAME}:${CROMWELL_BUILD_MYSQL_PORT}/${CROMWELL_BUILD_MYSQL_SCHEMA}?useSSL=false&rewriteBatchedStatements=true&serverTimezone=UTC" - CROMWELL_BUILD_POSTGRESQL_JDBC_URL="jdbc:postgresql://${CROMWELL_BUILD_POSTGRESQL_HOSTNAME}:${CROMWELL_BUILD_POSTGRESQL_PORT}/${CROMWELL_BUILD_POSTGRESQL_SCHEMA}" + CROMWELL_BUILD_POSTGRESQL_JDBC_URL="jdbc:postgresql://${CROMWELL_BUILD_POSTGRESQL_HOSTNAME}:${CROMWELL_BUILD_POSTGRESQL_PORT}/${CROMWELL_BUILD_POSTGRESQL_SCHEMA}?reWriteBatchedInserts=true" export CROMWELL_BUILD_MARIADB_DOCKER_TAG export CROMWELL_BUILD_MARIADB_HOSTNAME From a439100220d3c13fc16cc02282f4fb0c9400a3c3 Mon Sep 17 00:00:00 2001 From: Saloni Shah Date: Wed, 26 Jun 2019 12:08:41 -0400 Subject: [PATCH 28/28] Fixes size() returning 0 for dos inputs [BA-5731] (#5039) --- .../drs_tests/wf_level_file_size.wdl | 11 +++++++++++ .../drs_wf_level_file_size.test | 17 +++++++++++++++++ .../nio/impl/drs/DrsCloudNioFileProvider.scala | 16 ++++++++++++---- .../filesystems/drs/DrsPathBuilder.scala | 18 +++++++++++++++++- .../filesystems/drs/DrsPathBuilderSpec.scala | 8 ++++---- .../resources/papi_application.inc.conf.ctmpl | 12 ++++++++++++ 6 files changed, 73 insertions(+), 9 deletions(-) create mode 100644 centaur/src/main/resources/standardTestCases/drs_tests/wf_level_file_size.wdl create mode 100644 centaur/src/main/resources/standardTestCases/drs_wf_level_file_size.test diff --git a/centaur/src/main/resources/standardTestCases/drs_tests/wf_level_file_size.wdl b/centaur/src/main/resources/standardTestCases/drs_tests/wf_level_file_size.wdl new file mode 100644 index 00000000000..56d89094921 --- /dev/null +++ b/centaur/src/main/resources/standardTestCases/drs_tests/wf_level_file_size.wdl @@ -0,0 +1,11 @@ +version 1.0 + +workflow wf_level_file_size { + File input1 = "dos://wb-mock-drs-dev.storage.googleapis.com/4a3908ad-1f0b-4e2a-8a92-611f2123e8b0" + File input2 = "dos://wb-mock-drs-dev.storage.googleapis.com/0c8e7bc6-fd76-459d-947b-808b0605beb3" + + output { + Float fileSize1 = size(input1) + Float fileSize2 = size(input2) + } +} diff --git a/centaur/src/main/resources/standardTestCases/drs_wf_level_file_size.test b/centaur/src/main/resources/standardTestCases/drs_wf_level_file_size.test new file mode 100644 index 00000000000..67210745838 --- /dev/null +++ b/centaur/src/main/resources/standardTestCases/drs_wf_level_file_size.test @@ -0,0 +1,17 @@ +name: drs_wf_level_read_size +testFormat: workflowsuccess +backends: [Papiv2NoDockerHubConfig] + +files { + workflow: drs_tests/wf_level_file_size.wdl +} + +metadata { + workflowName: wf_level_file_size + status: Succeeded + + "outputs.wf_level_file_size.fileSize1": 43.0 + "outputs.wf_level_file_size.fileSize2": 45.0 +} + + diff --git a/cloud-nio/cloud-nio-impl-drs/src/main/scala/cloud/nio/impl/drs/DrsCloudNioFileProvider.scala b/cloud-nio/cloud-nio-impl-drs/src/main/scala/cloud/nio/impl/drs/DrsCloudNioFileProvider.scala index dbe4e7c0b94..38d443e2f87 100644 --- a/cloud-nio/cloud-nio-impl-drs/src/main/scala/cloud/nio/impl/drs/DrsCloudNioFileProvider.scala +++ b/cloud-nio/cloud-nio-impl-drs/src/main/scala/cloud/nio/impl/drs/DrsCloudNioFileProvider.scala @@ -39,10 +39,18 @@ class DrsCloudNioFileProvider(scheme: String, private def checkIfPathExistsThroughMartha(drsPath: String): IO[Boolean] = { - drsPathResolver.rawMarthaResponse(drsPath).use { marthaResponse => - val errorMsg = s"Status line was null for martha response $marthaResponse." - toIO(Option(marthaResponse.getStatusLine), errorMsg) - }.map(_.getStatusCode == HttpStatus.SC_OK) + /* + * Unlike other cloud providers where directories are identified with a trailing slash at the end like `gs://bucket/dir/`, + * DRS has a concept of bundles for directories (not supported yet). Hence for method `checkDirectoryExists` which appends a trailing '/' + * to see if the current path is a directory, return false + */ + if (drsPath.endsWith("/")) IO(false) + else { + drsPathResolver.rawMarthaResponse(drsPath).use { marthaResponse => + val errorMsg = s"Status line was null for martha response $marthaResponse." + toIO(Option(marthaResponse.getStatusLine), errorMsg) + }.map(_.getStatusCode == HttpStatus.SC_OK) + } } diff --git a/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilder.scala b/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilder.scala index d9cd2566be6..778bf01b95c 100644 --- a/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilder.scala +++ b/filesystems/drs/src/main/scala/cromwell/filesystems/drs/DrsPathBuilder.scala @@ -15,9 +15,25 @@ case class DrsPathBuilder(fileSystemProvider: DrsCloudNioFileSystemProvider) ext override def name: String = "DRS" + /** + * Unlike other cloud providers where directories are identified with a trailing slash at the end like `gs://bucket/dir/`, + * DRS has a concept of bundles for directories (not supported yet). This sanitizes the path by removing the trailing '/'(s) + * so that the CloudNioFileSystemProvider & CloudNioPath does not treat a DRS path ending with '/' as a directory, otherwise + * its size is returned as 0 + */ + private def removeTrailingSlashes(path: String): String = { + if (path.length > (drsScheme.length + 3)) { //3: length of '://' + val pathArray = path.split(s"://") + val transformedPath = pathArray(1).replaceAll("[/]+$", "") + s"$drsScheme://$transformedPath" + } + else path + } + override def build(pathAsString: String): Try[Path] = { if (pathAsString.startsWith(s"$drsScheme://")) { - Try(URI.create(UrlEscapers.urlFragmentEscaper().escape(pathAsString))) flatMap { uri => + val pathWithoutTrailingSlashes = removeTrailingSlashes(pathAsString) + Try(URI.create(UrlEscapers.urlFragmentEscaper().escape(pathWithoutTrailingSlashes))) flatMap { uri => if (!Option(uri.getScheme).exists(_.equalsIgnoreCase(fileSystemProvider.getScheme))) { Failure(new IllegalArgumentException(s"$pathAsString does not have a $drsScheme scheme.")) } else { diff --git a/filesystems/drs/src/test/scala/cromwell/filesystems/drs/DrsPathBuilderSpec.scala b/filesystems/drs/src/test/scala/cromwell/filesystems/drs/DrsPathBuilderSpec.scala index 713f7e19ac6..9d5f227125c 100644 --- a/filesystems/drs/src/test/scala/cromwell/filesystems/drs/DrsPathBuilderSpec.scala +++ b/filesystems/drs/src/test/scala/cromwell/filesystems/drs/DrsPathBuilderSpec.scala @@ -99,8 +99,8 @@ class DrsPathBuilderSpec extends TestKitSuite with FlatSpecLike with Matchers wi description = "a path ending in /", path = s"dos://$bucket/hello/world/", normalize = false, - pathAsString = s"dos://$bucket/hello/world/", - pathWithoutScheme = s"$bucket/hello/world/", + pathAsString = s"dos://$bucket/hello/world", + pathWithoutScheme = s"$bucket/hello/world", parent = s"dos://$bucket/hello/", getParent = s"dos://$bucket/hello/", root = s"dos://$bucket/", @@ -278,8 +278,8 @@ class DrsPathBuilderSpec extends TestKitSuite with FlatSpecLike with Matchers wi root = s"dos://blah/", name = "", getFileName = null, - getNameCount = 0, - isAbsolute = true), + getNameCount = 1, + isAbsolute = false), GoodPath( description = "an absolute path without a host", diff --git a/src/ci/resources/papi_application.inc.conf.ctmpl b/src/ci/resources/papi_application.inc.conf.ctmpl index 45389907230..c5051409bed 100644 --- a/src/ci/resources/papi_application.inc.conf.ctmpl +++ b/src/ci/resources/papi_application.inc.conf.ctmpl @@ -10,6 +10,9 @@ engine { auth = "service_account" project = "broad-dsde-cromwell-dev" } + drs { + auth = "service_account" + } } } @@ -63,3 +66,12 @@ services { } } } + +filesystems.drs.global.config.martha.url = "https://us-central1-broad-dsde-dev.cloudfunctions.net/martha_v2" + +drs { + localization { + docker-image = "broadinstitute/cromwell-dos:34-d8acfe3" + command-template = "mkdir -p $(dirname ${containerPath}) && /scripts/dosUrlLocalizer.sc ${drsPath} ${containerPath}" + } +}