From bd60ed2f87737023809b43629f7ed443e9c739a0 Mon Sep 17 00:00:00 2001 From: Simon Date: Thu, 12 Oct 2023 18:12:38 +0200 Subject: [PATCH] Migrate remote context resolution and distage modules to Cats Effect (#4361) * Migrate remote context resolution and distage modules to Cats Effect --------- Co-authored-by: Simon Dumas --- .../ch/epfl/bluebrain/nexus/delta/Main.scala | 3 +- .../delta/plugin/WiringInitializer.scala | 33 +++++---- .../nexus/delta/wiring/DeltaModule.scala | 21 +++--- .../nexus/delta/wiring/EventsModule.scala | 17 +++-- .../nexus/delta/wiring/IdentitiesModule.scala | 3 +- .../nexus/delta/wiring/ProjectsModule.scala | 31 ++++---- .../nexus/delta/wiring/StreamModule.scala | 12 ++-- .../bluebrain/nexus/delta/MainSuite.scala | 4 +- .../nexus/delta/plugin/PluginLoaderSpec.scala | 18 +++-- .../migration/MigrateEffectSyntax.scala | 1 + .../plugins/archive/ArchivePluginModule.scala | 3 +- .../RemoteContextResolutionFixture.scala | 18 ++--- .../blazegraph/BlazegraphPluginModule.scala | 70 +++++++++++-------- .../delta/plugins/blazegraph/Fixtures.scala | 20 +++--- .../CompositeViewsPluginModule.scala | 54 ++++++++------ .../config/CompositeViewsConfig.scala | 6 +- .../plugins/compositeviews/Fixtures.scala | 22 +++--- .../ElasticSearchPluginModule.scala | 65 +++++++++-------- .../GraphAnalyticsPluginModule.scala | 5 +- .../delta/plugins/jira/JiraPluginModule.scala | 3 +- .../ProjectDeletionModule.scala | 3 +- .../ProjectDeletionPluginDef.scala | 5 +- .../ProjectDeletionRoutesSpec.scala | 4 +- .../model/ProjectDeletionConfigSpec.scala | 4 +- .../plugins/search/SearchPluginModule.scala | 3 +- .../plugins/storage/StoragePluginConfig.scala | 21 +++--- .../plugins/storage/StoragePluginModule.scala | 37 +++++----- .../RemoteContextResolutionFixture.scala | 16 ++--- .../nexus/delta/rdf/graph/Graph.scala | 2 + .../delta/rdf/jsonld/CompactedJsonLd.scala | 19 +++-- .../delta/rdf/jsonld/ExpandedJsonLd.scala | 37 ++++++---- .../delta/rdf/jsonld/api/JsonLdApi.scala | 12 ++-- .../delta/rdf/jsonld/api/JsonLdJavaApi.scala | 48 +++++++------ .../rdf/jsonld/context/ContextValue.scala | 8 +-- .../rdf/jsonld/context/JsonLdContext.scala | 3 +- .../context/RemoteContextResolution.scala | 38 +++++----- .../bluebrain/nexus/delta/rdf/Fixtures.scala | 2 + .../delta/sdk/auth/AuthTokenProvider.scala | 8 +-- .../sdk/model/MetadataContextValue.scala | 5 +- .../resolvers/ResolverContextResolution.scala | 11 ++- .../nexus/delta/sdk/SerializationSuite.scala | 4 +- .../sdk/directives/AuthDirectivesSpec.scala | 14 ++-- .../ResolverContextResolutionSuite.scala | 4 +- .../nexus/delta/sdk/utils/Fixtures.scala | 16 ++--- 44 files changed, 397 insertions(+), 336 deletions(-) diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/Main.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/Main.scala index 337a353b00..137d3c2ffe 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/Main.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/Main.scala @@ -5,6 +5,7 @@ import akka.actor.typed.scaladsl.LoggerOps import akka.actor.typed.scaladsl.adapter._ import akka.actor.{ActorSystem => ActorSystemClassic} import akka.http.scaladsl.Http +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import akka.http.scaladsl.server.{ExceptionHandler, RejectionHandler, Route, RouteResult} import cats.effect.{ExitCode, Resource} import ch.epfl.bluebrain.nexus.delta.config.{AppConfig, BuildInfo} @@ -56,7 +57,7 @@ object Main extends BIOApp { (cfg, config, cl, pluginDefs) <- Resource.eval(loadPluginsAndConfig(loaderConfig)) _ <- Resource.eval(KamonMonitoring.initialize(config)) modules = DeltaModule(cfg, config, cl) - (plugins, locator) <- WiringInitializer(modules, pluginDefs) + (plugins, locator) <- WiringInitializer(modules, pluginDefs).mapK(ioToTaskK) _ <- bootstrap(locator, plugins) } yield locator diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugin/WiringInitializer.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugin/WiringInitializer.scala index 1d1b87f745..35fef91f40 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugin/WiringInitializer.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugin/WiringInitializer.scala @@ -1,18 +1,21 @@ package ch.epfl.bluebrain.nexus.delta.plugin -import cats.effect.Resource +import cats.effect.{IO, Resource} +import cats.syntax.traverse._ +import cats.syntax.flatMap._ +import cats.syntax.monadError._ +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ +import ch.epfl.bluebrain.nexus.delta.kernel.Logger import ch.epfl.bluebrain.nexus.delta.sdk.error.PluginError.PluginInitializationError import ch.epfl.bluebrain.nexus.delta.sdk.plugin.{Plugin, PluginDef} -import com.typesafe.scalalogging.Logger import distage.{Injector, Roots} import izumi.distage.model.Locator import izumi.distage.model.definition.ModuleDef import izumi.distage.modules.DefaultModule -import monix.bio.{IO, Task} object WiringInitializer { - private val logger = Logger[WiringInitializer.type] + private val logger = Logger.cats[WiringInitializer.type] /** * Combines the [[ModuleDef]] of the passed ''serviceModule'' with the ones provided by the plugins. Afterwards @@ -21,23 +24,25 @@ object WiringInitializer { def apply( serviceModule: ModuleDef, pluginsDef: List[PluginDef] - ): Resource[Task, (List[Plugin], Locator)] = { + ): Resource[IO, (List[Plugin], Locator)] = { val pluginsInfoModule = new ModuleDef { make[List[PluginDef]].from(pluginsDef) } val appModules = (serviceModule :: pluginsInfoModule :: pluginsDef.map(_.module)).merge // workaround for: java.lang.NoClassDefFoundError: zio/blocking/package$Blocking$Service - implicit val defaultModule: DefaultModule[Task] = DefaultModule.empty - Injector[Task]() + implicit val defaultModule: DefaultModule[IO] = DefaultModule.empty + Injector[IO]() .produce(appModules, Roots.Everything) .toCats .evalMap { locator => - IO.traverse(pluginsDef) { plugin => - Task.delay(logger.info(s"Initializing plugin ${plugin.info.name}...")) >> - plugin.initialize(locator).tapEval { _ => - Task.delay(logger.info(s"Plugin ${plugin.info.name} initialized.")) - } - }.map(_ -> locator) - .mapError(e => PluginInitializationError(e.getMessage)) + pluginsDef + .traverse { plugin => + logger.info(s"Initializing plugin ${plugin.info.name}...") >> + toCatsIO(plugin.initialize(locator)).flatTap { _ => + logger.info(s"Plugin ${plugin.info.name} initialized.") + } + } + .map(_ -> locator) + .adaptError(e => PluginInitializationError(e.getMessage)) } } } diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/DeltaModule.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/DeltaModule.scala index 8bfba3c365..a3f9ba978b 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/DeltaModule.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/DeltaModule.scala @@ -11,6 +11,7 @@ import cats.data.NonEmptyList import cats.effect.{Clock, ContextShift, IO, Resource, Sync, Timer} import ch.epfl.bluebrain.nexus.delta.Main.pluginsMaxPriority import ch.epfl.bluebrain.nexus.delta.config.AppConfig +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.kernel.utils.UUIDF import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.contexts import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api.{JsonLdApi, JsonLdJavaApi} @@ -34,7 +35,7 @@ import ch.epfl.bluebrain.nexus.delta.sourcing.execution.EvaluationExecution import ch.megard.akka.http.cors.scaladsl.settings.CorsSettings import com.typesafe.config.Config import izumi.distage.model.definition.{Id, ModuleDef} -import monix.bio.{Task, UIO} +import monix.bio.UIO import monix.execution.Scheduler import org.slf4j.{Logger, LoggerFactory} @@ -50,7 +51,7 @@ import scala.concurrent.duration.DurationInt * the raw merged and resolved configuration */ class DeltaModule(appCfg: AppConfig, config: Config)(implicit classLoader: ClassLoader) extends ModuleDef { - addImplicit[Sync[Task]] + addImplicit[Sync[IO]] make[AppConfig].from(appCfg) make[Config].from(config) @@ -66,7 +67,7 @@ class DeltaModule(appCfg: AppConfig, config: Config)(implicit classLoader: Class implicit val scheduler: Scheduler = Scheduler.global make[Transactors].fromResource { - Transactors.init(appCfg.database) + Transactors.init(appCfg.database).mapK(taskToIoK) } make[List[PluginDescription]].from { (pluginsDef: List[PluginDef]) => pluginsDef.map(_.info) } @@ -102,7 +103,9 @@ class DeltaModule(appCfg: AppConfig, config: Config)(implicit classLoader: Class .merge(otherCtxResolutions.toSeq: _*) } - make[JsonLdApi].fromValue(new JsonLdJavaApi(appCfg.jsonLdApi)) + make[JsonLdApi].from { contextShift: ContextShift[IO] => + new JsonLdJavaApi(appCfg.jsonLdApi)(contextShift) + } make[Clock[UIO]].from(Clock[UIO]) make[Clock[IO]].from(Clock.create[IO]) @@ -116,17 +119,19 @@ class DeltaModule(appCfg: AppConfig, config: Config)(implicit classLoader: Class List("@context", "@id", "@type", "reason", "details", "sourceId", "projectionId", "_total", "_results") ) ) - make[ActorSystem[Nothing]].fromResource { - val make = Task.delay( + make[ActorSystem[Nothing]].fromResource { (timer: Timer[IO], contextShift: ContextShift[IO]) => + implicit val t: Timer[IO] = timer + implicit val cs: ContextShift[IO] = contextShift + val make = IO.delay( ActorSystem[Nothing]( Behaviors.empty, appCfg.description.fullName, BootstrapSetup().withConfig(config).withClassloader(classLoader) ) ) - val release = (as: ActorSystem[Nothing]) => { + val release = (as: ActorSystem[Nothing]) => { import akka.actor.typed.scaladsl.adapter._ - Task.deferFuture(as.toClassic.terminate()).timeout(15.seconds).void + IO.fromFuture(IO(as.toClassic.terminate()).timeout(15.seconds)).void } Resource.make(make)(release) } diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/EventsModule.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/EventsModule.scala index efa82606cd..1d49b0ba49 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/EventsModule.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/EventsModule.scala @@ -5,6 +5,7 @@ import ch.epfl.bluebrain.nexus.delta.config.AppConfig import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolution import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering import ch.epfl.bluebrain.nexus.delta.routes.{ElemRoutes, EventsRoutes} +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk._ import ch.epfl.bluebrain.nexus.delta.sdk.acls.AclCheck import ch.epfl.bluebrain.nexus.delta.sdk.directives.DeltaSchemeDirectives @@ -32,13 +33,15 @@ object EventsModule extends ModuleDef { xas: Transactors, jo: JsonKeyOrdering ) => - SseEventLog( - sseEncoders, - organizations.fetch(_).void, - projects.fetch(_).map { p => (p.value.organizationUuid, p.value.uuid) }, - config.sse, - xas - )(jo) + toCatsIO( + SseEventLog( + sseEncoders, + organizations.fetch(_).void, + projects.fetch(_).map { p => (p.value.organizationUuid, p.value.uuid) }, + config.sse, + xas + )(jo) + ) } make[SseElemStream].from { (qc: QueryConfig, xas: Transactors) => diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/IdentitiesModule.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/IdentitiesModule.scala index 489e0387e8..b8717e697a 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/IdentitiesModule.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/IdentitiesModule.scala @@ -16,7 +16,6 @@ import ch.epfl.bluebrain.nexus.delta.sdk.identities.{Identities, IdentitiesImpl} import ch.epfl.bluebrain.nexus.delta.sdk.model.BaseUri import ch.epfl.bluebrain.nexus.delta.sdk.realms.Realms import izumi.distage.model.definition.{Id, ModuleDef} -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ /** * Identities module wiring config. @@ -28,7 +27,7 @@ object IdentitiesModule extends ModuleDef { make[CacheConfig].from((cfg: AppConfig) => cfg.identities) make[Identities].fromEffect { (realms: Realms, hc: HttpClient @Id("realm"), config: CacheConfig) => - IdentitiesImpl(realms, hc, config).toUIO + IdentitiesImpl(realms, hc, config) } make[OpenIdAuthService].from { (httpClient: HttpClient @Id("realm"), realms: Realms) => diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/ProjectsModule.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/ProjectsModule.scala index e8c881138d..c12698757b 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/ProjectsModule.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/ProjectsModule.scala @@ -1,6 +1,6 @@ package ch.epfl.bluebrain.nexus.delta.wiring -import cats.effect.Clock +import cats.effect.{Clock, IO} import ch.epfl.bluebrain.nexus.delta.Main.pluginsMaxPriority import ch.epfl.bluebrain.nexus.delta.config.AppConfig import ch.epfl.bluebrain.nexus.delta.kernel.utils.UUIDF @@ -8,6 +8,7 @@ import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.contexts import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContextResolution} import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering import ch.epfl.bluebrain.nexus.delta.routes.ProjectsRoutes +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk._ import ch.epfl.bluebrain.nexus.delta.sdk.acls.{AclCheck, Acls} import ch.epfl.bluebrain.nexus.delta.sdk.deletion.{ProjectDeletionCoordinator, ProjectDeletionTask} @@ -28,7 +29,7 @@ import ch.epfl.bluebrain.nexus.delta.sdk.sse.SseEncoder import ch.epfl.bluebrain.nexus.delta.sourcing.Transactors import ch.epfl.bluebrain.nexus.delta.sourcing.stream.Supervisor import izumi.distage.model.definition.{Id, ModuleDef} -import monix.bio.{Task, UIO} +import monix.bio.UIO import monix.execution.Scheduler /** @@ -58,7 +59,7 @@ object ProjectsModule extends ModuleDef { clock: Clock[UIO], uuidF: UUIDF ) => - Task.pure( + IO.pure( ProjectsImpl( organizations.fetchActiveOrganization(_).mapError(WrappedOrganizationRejection), ValidateProjectDeletion(xas, config.projects.deletion.enabled), @@ -71,7 +72,7 @@ object ProjectsModule extends ModuleDef { } make[ProjectsStatistics].fromEffect { (xas: Transactors) => - ProjectsStatistics(xas) + toCatsIO(ProjectsStatistics(xas)) } make[ProjectProvisioning].from { @@ -81,7 +82,7 @@ object ProjectsModule extends ModuleDef { make[FetchContext[ContextRejection]].fromEffect { (organizations: Organizations, projects: Projects, quotas: Quotas) => - Task.pure(FetchContext(organizations, projects, quotas)) + IO.pure(FetchContext(organizations, projects, quotas)) } make[ProjectDeletionCoordinator].fromEffect { @@ -94,18 +95,20 @@ object ProjectsModule extends ModuleDef { xas: Transactors, clock: Clock[UIO] ) => - ProjectDeletionCoordinator( - projects, - deletionTasks, - config.projects.deletion, - serviceAccount, - supervisor, - xas - )(clock) + toCatsIO( + ProjectDeletionCoordinator( + projects, + deletionTasks, + config.projects.deletion, + serviceAccount, + supervisor, + xas + )(clock) + ) } make[UUIDCache].fromEffect { (config: AppConfig, xas: Transactors) => - UUIDCache(config.projects.cache, config.organizations.cache, xas) + toCatsIO(UUIDCache(config.projects.cache, config.organizations.cache, xas)) } make[DeltaSchemeDirectives].from { diff --git a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/StreamModule.scala b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/StreamModule.scala index 41d1c22b09..cad8640a15 100644 --- a/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/StreamModule.scala +++ b/delta/app/src/main/scala/ch/epfl/bluebrain/nexus/delta/wiring/StreamModule.scala @@ -1,6 +1,7 @@ package ch.epfl.bluebrain.nexus.delta.wiring import cats.effect.{Clock, IO, Sync, Timer} +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk.ResourceShifts import ch.epfl.bluebrain.nexus.delta.sdk.stream.GraphResourceStream import ch.epfl.bluebrain.nexus.delta.sourcing.config.{ProjectionConfig, QueryConfig} @@ -9,14 +10,13 @@ import ch.epfl.bluebrain.nexus.delta.sourcing.stream._ import ch.epfl.bluebrain.nexus.delta.sourcing.stream.pipes._ import ch.epfl.bluebrain.nexus.delta.sourcing.{DeleteExpired, PurgeElemFailures, Transactors} import izumi.distage.model.definition.ModuleDef -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ -import monix.bio.{Task, UIO} +import monix.bio.UIO /** * Indexing specific wiring. */ object StreamModule extends ModuleDef { - addImplicit[Sync[Task]] + addImplicit[Sync[IO]] make[GraphResourceStream].from { ( @@ -52,16 +52,16 @@ object StreamModule extends ModuleDef { make[Supervisor].fromResource { (projections: Projections, projectionErrors: ProjectionErrors, cfg: ProjectionConfig) => - Supervisor(projections, projectionErrors, cfg) + Supervisor(projections, projectionErrors, cfg).mapK(taskToIoK) } make[DeleteExpired].fromEffect { (supervisor: Supervisor, config: ProjectionConfig, xas: Transactors, clock: Clock[IO], timer: Timer[IO]) => - DeleteExpired(supervisor, config, xas)(clock, timer).toUIO + DeleteExpired(supervisor, config, xas)(clock, timer) } make[PurgeElemFailures].fromEffect { (supervisor: Supervisor, config: ProjectionConfig, xas: Transactors, clock: Clock[UIO]) => - PurgeElemFailures(supervisor, config, xas)(clock) + toCatsIO(PurgeElemFailures(supervisor, config, xas)(clock)) } } diff --git a/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/MainSuite.scala b/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/MainSuite.scala index 4eaef30cd8..8602f11ce1 100644 --- a/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/MainSuite.scala +++ b/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/MainSuite.scala @@ -1,7 +1,7 @@ package ch.epfl.bluebrain.nexus.delta import akka.http.scaladsl.server.Route -import cats.effect.Resource +import cats.effect.{IO, Resource} import ch.epfl.bluebrain.nexus.delta.plugin.PluginsLoader.PluginLoaderConfig import ch.epfl.bluebrain.nexus.delta.sdk.plugin.PluginDef import ch.epfl.bluebrain.nexus.delta.wiring.DeltaModule @@ -42,7 +42,7 @@ class MainSuite extends BioSuite with MainSuite.Fixture { val modules: Module = (DeltaModule(cfg, config, cl) :: pluginsInfoModule :: pDefs.map(_.module)).merge PlanVerifier() - .verify[Task]( + .verify[IO]( bindings = modules, roots = Roots.Everything, providedKeys = Set.empty, diff --git a/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugin/PluginLoaderSpec.scala b/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugin/PluginLoaderSpec.scala index d1560e957f..98b647717b 100644 --- a/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugin/PluginLoaderSpec.scala +++ b/delta/app/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugin/PluginLoaderSpec.scala @@ -8,6 +8,7 @@ import ch.epfl.bluebrain.nexus.testkit.IOValues import com.typesafe.config.impl.ConfigImpl import izumi.distage.model.definition.ModuleDef import monix.bio.Task +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import monix.execution.Scheduler import monix.execution.Scheduler.Implicits.global import org.scalatest.matchers.should.Matchers @@ -25,15 +26,18 @@ class PluginLoaderSpec extends AnyWordSpecLike with ScalatestRouteTest with Matc val config = PluginLoaderConfig("../plugins/test-plugin/target") "load plugins from .jar in a directory" in { val (_, pluginsDef) = PluginsLoader(config).load.accepted - WiringInitializer(serviceModule, pluginsDef).use { case (_, locator) => - Task.delay { - val route = locator.get[Set[PriorityRoute]].head - pluginsDef.head.priority shouldEqual 10 - Get("/test-plugin") ~> route.route ~> check { - responseAs[String] shouldEqual "http://localhost" + WiringInitializer(serviceModule, pluginsDef) + .mapK(ioToTaskK) + .use { case (_, locator) => + Task.delay { + val route = locator.get[Set[PriorityRoute]].head + pluginsDef.head.priority shouldEqual 10 + Get("/test-plugin") ~> route.route ~> check { + responseAs[String] shouldEqual "http://localhost" + } } } - }.accepted + .accepted } "load overriding priority" in { diff --git a/delta/kernel/src/main/scala/ch/epfl/bluebrain/nexus/delta/kernel/effect/migration/MigrateEffectSyntax.scala b/delta/kernel/src/main/scala/ch/epfl/bluebrain/nexus/delta/kernel/effect/migration/MigrateEffectSyntax.scala index 39214c20b5..3420488176 100644 --- a/delta/kernel/src/main/scala/ch/epfl/bluebrain/nexus/delta/kernel/effect/migration/MigrateEffectSyntax.scala +++ b/delta/kernel/src/main/scala/ch/epfl/bluebrain/nexus/delta/kernel/effect/migration/MigrateEffectSyntax.scala @@ -16,6 +16,7 @@ trait MigrateEffectSyntax { val taskToIoK: Task ~> IO = λ[Task ~> IO](toCatsIO(_)) val ioToUioK: IO ~> UIO = λ[IO ~> UIO](_.toUIO) + val ioToTaskK: IO ~> Task = λ[IO ~> Task](Task.from(_)) } diff --git a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivePluginModule.scala b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivePluginModule.scala index 70360672ee..1af0b8fc63 100644 --- a/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivePluginModule.scala +++ b/delta/plugins/archive/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/ArchivePluginModule.scala @@ -21,7 +21,6 @@ import ch.epfl.bluebrain.nexus.delta.sourcing.Transactors import ch.epfl.bluebrain.nexus.delta.sourcing.execution.EvaluationExecution import com.typesafe.config.Config import izumi.distage.model.definition.{Id, ModuleDef} -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ /** * Archive plugin wiring. @@ -29,7 +28,7 @@ import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ object ArchivePluginModule extends ModuleDef { implicit private val classLoader: ClassLoader = getClass.getClassLoader - make[ArchivePluginConfig].fromEffect { cfg: Config => ArchivePluginConfig.load(cfg).toUIO } + make[ArchivePluginConfig].fromEffect { cfg: Config => ArchivePluginConfig.load(cfg) } make[ArchiveDownload].from { ( diff --git a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/RemoteContextResolutionFixture.scala b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/RemoteContextResolutionFixture.scala index aebf0ed58c..437413ebb1 100644 --- a/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/RemoteContextResolutionFixture.scala +++ b/delta/plugins/archive/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/archive/RemoteContextResolutionFixture.scala @@ -10,15 +10,15 @@ import ch.epfl.bluebrain.nexus.testkit.IOValues trait RemoteContextResolutionFixture extends IOValues { implicit private val cl: ClassLoader = getClass.getClassLoader - implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixed( - storageContexts.storages -> ContextValue.fromFile("contexts/storages.json").accepted, - storageContexts.storagesMetadata -> ContextValue.fromFile("contexts/storages-metadata.json").accepted, - fileContexts.files -> ContextValue.fromFile("contexts/files.json").accepted, - contexts.archives -> ContextValue.fromFile("contexts/archives.json").accepted, - contexts.archivesMetadata -> ContextValue.fromFile("contexts/archives-metadata.json").accepted, - Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json").accepted, - Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json").accepted, - Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json").accepted + implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixedIO( + storageContexts.storages -> ContextValue.fromFile("contexts/storages.json"), + storageContexts.storagesMetadata -> ContextValue.fromFile("contexts/storages-metadata.json"), + fileContexts.files -> ContextValue.fromFile("contexts/files.json"), + contexts.archives -> ContextValue.fromFile("contexts/archives.json"), + contexts.archivesMetadata -> ContextValue.fromFile("contexts/archives-metadata.json"), + Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json"), + Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json"), + Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json") ) } diff --git a/delta/plugins/blazegraph/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/BlazegraphPluginModule.scala b/delta/plugins/blazegraph/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/BlazegraphPluginModule.scala index 9d0947ae92..8ef5fcea0d 100644 --- a/delta/plugins/blazegraph/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/BlazegraphPluginModule.scala +++ b/delta/plugins/blazegraph/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/BlazegraphPluginModule.scala @@ -15,6 +15,7 @@ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteCon import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering import ch.epfl.bluebrain.nexus.delta.sdk.IndexingAction.AggregateIndexingAction import ch.epfl.bluebrain.nexus.delta.sdk._ +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk.acls.AclCheck import ch.epfl.bluebrain.nexus.delta.sdk.deletion.ProjectDeletionTask import ch.epfl.bluebrain.nexus.delta.sdk.directives.DeltaSchemeDirectives @@ -23,7 +24,6 @@ import ch.epfl.bluebrain.nexus.delta.sdk.http.HttpClient import ch.epfl.bluebrain.nexus.delta.sdk.identities.Identities import ch.epfl.bluebrain.nexus.delta.sdk.identities.model.ServiceAccount import ch.epfl.bluebrain.nexus.delta.sdk.model._ -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk.model.metrics.ScopedEventMetricEncoder import ch.epfl.bluebrain.nexus.delta.sdk.permissions.Permissions import ch.epfl.bluebrain.nexus.delta.sdk.projects.FetchContext @@ -62,11 +62,13 @@ class BlazegraphPluginModule(priority: Int) extends ModuleDef { make[BlazegraphSlowQueryDeleter].fromEffect { (supervisor: Supervisor, store: BlazegraphSlowQueryStore, cfg: BlazegraphViewsConfig) => - BlazegraphSlowQueryDeleter.start( - supervisor, - store, - cfg.slowQueries.logTtl, - cfg.slowQueries.deleteExpiredLogsEvery + toCatsIO( + BlazegraphSlowQueryDeleter.start( + supervisor, + store, + cfg.slowQueries.logTtl, + cfg.slowQueries.deleteExpiredLogsEvery + ) ) } @@ -123,15 +125,17 @@ class BlazegraphPluginModule(priority: Int) extends ModuleDef { clock: Clock[UIO], uuidF: UUIDF ) => - BlazegraphViews( - fetchContext.mapRejection(ProjectContextRejection), - contextResolution, - validate, - client, - config.eventLog, - config.prefix, - xas - )(api, clock, uuidF) + toCatsIO( + BlazegraphViews( + fetchContext.mapRejection(ProjectContextRejection), + contextResolution, + validate, + client, + config.eventLog, + config.prefix, + xas + )(api, clock, uuidF) + ) } make[BlazegraphCoordinator].fromEffect { @@ -144,14 +148,16 @@ class BlazegraphPluginModule(priority: Int) extends ModuleDef { config: BlazegraphViewsConfig, baseUri: BaseUri ) => - BlazegraphCoordinator( - views, - graphStream, - registry, - supervisor, - client, - config - )(baseUri) + toCatsIO( + BlazegraphCoordinator( + views, + graphStream, + registry, + supervisor, + client, + config + )(baseUri) + ) } make[BlazegraphViewsQuery].fromEffect { @@ -164,14 +170,16 @@ class BlazegraphPluginModule(priority: Int) extends ModuleDef { cfg: BlazegraphViewsConfig, xas: Transactors ) => - BlazegraphViewsQuery( - aclCheck, - fetchContext.mapRejection(ProjectContextRejection), - views, - client, - slowQueryLogger, - cfg.prefix, - xas + toCatsIO( + BlazegraphViewsQuery( + aclCheck, + fetchContext.mapRejection(ProjectContextRejection), + views, + client, + slowQueryLogger, + cfg.prefix, + xas + ) ) } diff --git a/delta/plugins/blazegraph/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/Fixtures.scala b/delta/plugins/blazegraph/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/Fixtures.scala index 0ce1e406df..e0ca124b9f 100644 --- a/delta/plugins/blazegraph/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/Fixtures.scala +++ b/delta/plugins/blazegraph/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/blazegraph/Fixtures.scala @@ -13,16 +13,16 @@ trait Fixtures extends IOValues { implicit val api: JsonLdApi = JsonLdJavaApi.strict - implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixed( - blazegraph -> ContextValue.fromFile("contexts/sparql.json").accepted, - blazegraphMetadata -> ContextValue.fromFile("contexts/sparql-metadata.json").accepted, - Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json").accepted, - Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json").accepted, - Vocabulary.contexts.shacl -> ContextValue.fromFile("contexts/shacl.json").accepted, - Vocabulary.contexts.statistics -> ContextValue.fromFile("/contexts/statistics.json").accepted, - Vocabulary.contexts.offset -> ContextValue.fromFile("/contexts/offset.json").accepted, - Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json").accepted, - Vocabulary.contexts.search -> ContextValue.fromFile("contexts/search.json").accepted + implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixedIO( + blazegraph -> ContextValue.fromFile("contexts/sparql.json"), + blazegraphMetadata -> ContextValue.fromFile("contexts/sparql-metadata.json"), + Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json"), + Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json"), + Vocabulary.contexts.shacl -> ContextValue.fromFile("contexts/shacl.json"), + Vocabulary.contexts.statistics -> ContextValue.fromFile("/contexts/statistics.json"), + Vocabulary.contexts.offset -> ContextValue.fromFile("/contexts/offset.json"), + Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json"), + Vocabulary.contexts.search -> ContextValue.fromFile("contexts/search.json") ) def alwaysValidate: ValidateBlazegraphView = (_: BlazegraphViewValue) => IO.unit diff --git a/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/CompositeViewsPluginModule.scala b/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/CompositeViewsPluginModule.scala index 8884072d76..e761582993 100644 --- a/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/CompositeViewsPluginModule.scala +++ b/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/CompositeViewsPluginModule.scala @@ -124,16 +124,18 @@ class CompositeViewsPluginModule(priority: Int) extends ModuleDef { uuidF: UUIDF, clock: Clock[UIO] ) => - CompositeViews( - fetchContext.mapRejection(ProjectContextRejection), - contextResolution, - validate, - config, - xas - )( - api, - clock, - uuidF + toCatsIO( + CompositeViews( + fetchContext.mapRejection(ProjectContextRejection), + contextResolution, + validate, + config, + xas + )( + api, + clock, + uuidF + ) ) } @@ -155,9 +157,11 @@ class CompositeViewsPluginModule(priority: Int) extends ModuleDef { config.restartCheckInterval )(clock) - CompositeRestartStore - .deleteExpired(compositeRestartStore, supervisor, projectionConfig)(clock) - .as(compositeProjections) + toCatsIO( + CompositeRestartStore + .deleteExpired(compositeRestartStore, supervisor, projectionConfig)(clock) + .as(compositeProjections) + ) } make[CompositeSpaces].from { @@ -189,9 +193,11 @@ class CompositeViewsPluginModule(priority: Int) extends ModuleDef { api: JsonLdApi, cr: RemoteContextResolution @Id("aggregate") ) => - JsonLdContext(listingsMetadataCtx.value)(api, cr, JsonLdOptions.defaults) - .map(_.aliasesInv.keySet.map(Triple.predicate)) - .map(MetadataPredicates) + toCatsIO( + JsonLdContext(listingsMetadataCtx.value)(api, cr, JsonLdOptions.defaults) + .map(_.aliasesInv.keySet.map(Triple.predicate)) + .map(MetadataPredicates) + ) } make[RemoteGraphStream].from { @@ -234,13 +240,15 @@ class CompositeViewsPluginModule(priority: Int) extends ModuleDef { config: CompositeViewsConfig, xas: Transactors ) => - Task.when(isCompositeMigrationRunning)(new MigrateCompositeViews(xas).run.void) >> - CompositeViewsCoordinator( - compositeViews, - supervisor, - lifecycle, - config - ) + toCatsIO( + Task.when(isCompositeMigrationRunning)(new MigrateCompositeViews(xas).run.void) >> + CompositeViewsCoordinator( + compositeViews, + supervisor, + lifecycle, + config + ) + ) } many[ProjectDeletionTask].add { (views: CompositeViews) => CompositeViewsDeletionTask(views) } diff --git a/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/config/CompositeViewsConfig.scala b/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/config/CompositeViewsConfig.scala index 3fc003b23e..96db24e27b 100644 --- a/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/config/CompositeViewsConfig.scala +++ b/delta/plugins/composite-views/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/config/CompositeViewsConfig.scala @@ -1,6 +1,7 @@ package ch.epfl.bluebrain.nexus.delta.plugins.compositeviews.config import akka.http.scaladsl.model.Uri +import cats.effect.IO import ch.epfl.bluebrain.nexus.delta.plugins.blazegraph.config.BlazegraphViewsConfig.Credentials import ch.epfl.bluebrain.nexus.delta.plugins.compositeviews.config.CompositeViewsConfig.SinkConfig.SinkConfig import ch.epfl.bluebrain.nexus.delta.plugins.compositeviews.config.CompositeViewsConfig.{BlazegraphAccess, RemoteSourceClientConfig, SourcesConfig} @@ -10,7 +11,6 @@ import ch.epfl.bluebrain.nexus.delta.sdk.instances._ import ch.epfl.bluebrain.nexus.delta.sdk.model.search.PaginationConfig import ch.epfl.bluebrain.nexus.delta.sourcing.config.{BatchConfig, EventLogConfig} import com.typesafe.config.Config -import monix.bio.UIO import pureconfig.error.CannotConvert import pureconfig.generic.auto._ import pureconfig.generic.semiauto.deriveReader @@ -136,8 +136,8 @@ object CompositeViewsConfig { /** * Converts a [[Config]] into an [[CompositeViewsConfig]] */ - def load(config: Config): UIO[CompositeViewsConfig] = - UIO.delay { + def load(config: Config): IO[CompositeViewsConfig] = + IO.delay { ConfigSource .fromConfig(config) .at("plugins.composite-views") diff --git a/delta/plugins/composite-views/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/Fixtures.scala b/delta/plugins/composite-views/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/Fixtures.scala index 8086f56690..98a52a40d9 100644 --- a/delta/plugins/composite-views/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/Fixtures.scala +++ b/delta/plugins/composite-views/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/compositeviews/Fixtures.scala @@ -12,16 +12,16 @@ trait Fixtures extends IOValues { implicit val api: JsonLdApi = JsonLdJavaApi.strict - implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixed( - iri"http://music.com/context" -> ContextValue.fromFile("indexing/music-context.json").accepted, - contexts.compositeViews -> ContextValue.fromFile("contexts/composite-views.json").accepted, - contexts.compositeViewsMetadata -> ContextValue.fromFile("contexts/composite-views-metadata.json").accepted, - Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json").accepted, - Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json").accepted, - Vocabulary.contexts.shacl -> ContextValue.fromFile("contexts/shacl.json").accepted, - Vocabulary.contexts.statistics -> ContextValue.fromFile("/contexts/statistics.json").accepted, - Vocabulary.contexts.offset -> ContextValue.fromFile("/contexts/offset.json").accepted, - Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json").accepted, - Vocabulary.contexts.search -> ContextValue.fromFile("contexts/search.json").accepted + implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixedIO( + iri"http://music.com/context" -> ContextValue.fromFile("indexing/music-context.json"), + contexts.compositeViews -> ContextValue.fromFile("contexts/composite-views.json"), + contexts.compositeViewsMetadata -> ContextValue.fromFile("contexts/composite-views-metadata.json"), + Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json"), + Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json"), + Vocabulary.contexts.shacl -> ContextValue.fromFile("contexts/shacl.json"), + Vocabulary.contexts.statistics -> ContextValue.fromFile("/contexts/statistics.json"), + Vocabulary.contexts.offset -> ContextValue.fromFile("/contexts/offset.json"), + Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json"), + Vocabulary.contexts.search -> ContextValue.fromFile("contexts/search.json") ) } diff --git a/delta/plugins/elasticsearch/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/elasticsearch/ElasticSearchPluginModule.scala b/delta/plugins/elasticsearch/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/elasticsearch/ElasticSearchPluginModule.scala index f76e602bc4..1387d097e9 100644 --- a/delta/plugins/elasticsearch/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/elasticsearch/ElasticSearchPluginModule.scala +++ b/delta/plugins/elasticsearch/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/elasticsearch/ElasticSearchPluginModule.scala @@ -3,6 +3,7 @@ package ch.epfl.bluebrain.nexus.delta.plugins.elasticsearch import akka.actor.typed.ActorSystem import cats.effect.Clock import ch.epfl.bluebrain.nexus.delta.kernel.utils.UUIDF +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.plugins.elasticsearch.client.ElasticSearchClient import ch.epfl.bluebrain.nexus.delta.plugins.elasticsearch.config.ElasticSearchViewsConfig import ch.epfl.bluebrain.nexus.delta.plugins.elasticsearch.deletion.{ElasticSearchDeletionTask, EventMetricsDeletionTask} @@ -90,14 +91,16 @@ class ElasticSearchPluginModule(priority: Int) extends ModuleDef { clock: Clock[UIO], uuidF: UUIDF ) => - ElasticSearchViews( - fetchContext.mapRejection(ProjectContextRejection), - contextResolution, - validateElasticSearchView, - config.eventLog, - config.prefix, - xas - )(api, clock, uuidF) + toCatsIO( + ElasticSearchViews( + fetchContext.mapRejection(ProjectContextRejection), + contextResolution, + validateElasticSearchView, + config.eventLog, + config.prefix, + xas + )(api, clock, uuidF) + ) } make[ElasticSearchCoordinator].fromEffect { @@ -110,14 +113,16 @@ class ElasticSearchPluginModule(priority: Int) extends ModuleDef { config: ElasticSearchViewsConfig, cr: RemoteContextResolution @Id("aggregate") ) => - ElasticSearchCoordinator( - views, - graphStream, - registry, - supervisor, - client, - config - )(cr) + toCatsIO( + ElasticSearchCoordinator( + views, + graphStream, + registry, + supervisor, + client, + config + )(cr) + ) } make[EventMetricsProjection].fromEffect { @@ -128,14 +133,16 @@ class ElasticSearchPluginModule(priority: Int) extends ModuleDef { client: ElasticSearchClient, config: ElasticSearchViewsConfig ) => - EventMetricsProjection( - metricEncoders, - supervisor, - client, - xas, - config.batch, - config.metricsQuery, - config.prefix + toCatsIO( + EventMetricsProjection( + metricEncoders, + supervisor, + client, + xas, + config.batch, + config.metricsQuery, + config.prefix + ) ) } @@ -379,10 +386,12 @@ class ElasticSearchPluginModule(priority: Int) extends ModuleDef { } make[ElasticSearchView.Shift].fromEffect { (views: ElasticSearchViews, base: BaseUri) => - for { - defaultMapping <- defaultElasticsearchMapping - defaultSettings <- defaultElasticsearchSettings - } yield ElasticSearchView.shift(views, defaultMapping, defaultSettings)(base) + toCatsIO( + for { + defaultMapping <- defaultElasticsearchMapping + defaultSettings <- defaultElasticsearchSettings + } yield ElasticSearchView.shift(views, defaultMapping, defaultSettings)(base) + ) } many[ResourceShift[_, _, _]].ref[ElasticSearchView.Shift] diff --git a/delta/plugins/graph-analytics/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/graph/analytics/GraphAnalyticsPluginModule.scala b/delta/plugins/graph-analytics/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/graph/analytics/GraphAnalyticsPluginModule.scala index 1ab92b7104..a67ff9ae24 100644 --- a/delta/plugins/graph-analytics/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/graph/analytics/GraphAnalyticsPluginModule.scala +++ b/delta/plugins/graph-analytics/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/graph/analytics/GraphAnalyticsPluginModule.scala @@ -7,6 +7,7 @@ import ch.epfl.bluebrain.nexus.delta.plugins.graph.analytics.model.GraphAnalytic import ch.epfl.bluebrain.nexus.delta.plugins.graph.analytics.routes.GraphAnalyticsRoutes import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContextResolution} import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk._ import ch.epfl.bluebrain.nexus.delta.sdk.acls.AclCheck import ch.epfl.bluebrain.nexus.delta.sdk.directives.DeltaSchemeDirectives @@ -48,7 +49,9 @@ class GraphAnalyticsPluginModule(priority: Int) extends ModuleDef { client: ElasticSearchClient, config: GraphAnalyticsConfig ) => - GraphAnalyticsCoordinator(projects, analyticsStream, supervisor, client, config) + toCatsIO( + GraphAnalyticsCoordinator(projects, analyticsStream, supervisor, client, config) + ) } make[GraphAnalyticsViewsQuery].from { (client: ElasticSearchClient, config: GraphAnalyticsConfig) => diff --git a/delta/plugins/jira/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/jira/JiraPluginModule.scala b/delta/plugins/jira/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/jira/JiraPluginModule.scala index 236b589b30..4d7b4522a1 100644 --- a/delta/plugins/jira/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/jira/JiraPluginModule.scala +++ b/delta/plugins/jira/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/jira/JiraPluginModule.scala @@ -3,7 +3,6 @@ package ch.epfl.bluebrain.nexus.delta.plugins.jira import cats.effect.{Clock, IO} import ch.epfl.bluebrain.nexus.delta.plugins.jira.config.JiraConfig import ch.epfl.bluebrain.nexus.delta.plugins.jira.routes.JiraRoutes -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolution import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering import ch.epfl.bluebrain.nexus.delta.sdk._ @@ -21,7 +20,7 @@ class JiraPluginModule(priority: Int) extends ModuleDef { make[JiraConfig].from { JiraConfig.load(_) } make[JiraClient].fromEffect { (xas: Transactors, jiraConfig: JiraConfig, clock: Clock[IO]) => - JiraClient(TokenStore(xas)(clock), jiraConfig).toUIO + JiraClient(TokenStore(xas)(clock), jiraConfig) } make[JiraRoutes].from { diff --git a/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionModule.scala b/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionModule.scala index dffbd083da..7274455bcc 100644 --- a/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionModule.scala +++ b/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionModule.scala @@ -1,7 +1,6 @@ package ch.epfl.bluebrain.nexus.delta.plugins.projectdeletion import cats.effect.{Clock, IO, Timer} -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.plugins.projectdeletion.model.{contexts, ProjectDeletionConfig} import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContextResolution} import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering @@ -42,6 +41,6 @@ class ProjectDeletionModule(priority: Int) extends ModuleDef { supervisor: Supervisor, clock: Clock[IO], timer: Timer[IO] - ) => ProjectDeletionRunner.start(projects, config, projectStatistics, supervisor)(clock, timer).toUIO + ) => ProjectDeletionRunner.start(projects, config, projectStatistics, supervisor)(clock, timer) } } diff --git a/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionPluginDef.scala b/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionPluginDef.scala index c144594891..f706cef799 100644 --- a/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionPluginDef.scala +++ b/delta/plugins/project-deletion/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionPluginDef.scala @@ -1,12 +1,13 @@ package ch.epfl.bluebrain.nexus.delta.plugins.projectdeletion +import cats.effect.IO import ch.epfl.bluebrain.nexus.delta.plugins.projectdeletion.model.ProjectDeletionConfig import ch.epfl.bluebrain.nexus.delta.sdk.model.ComponentDescription.PluginDescription import ch.epfl.bluebrain.nexus.delta.sdk.model.Name import ch.epfl.bluebrain.nexus.delta.sdk.plugin.{Plugin, PluginDef} import izumi.distage.model.Locator import izumi.distage.model.definition.ModuleDef -import monix.bio.{Task, UIO} +import monix.bio.Task import pureconfig.ConfigSource class ProjectDeletionPluginDef extends PluginDef { @@ -16,7 +17,7 @@ class ProjectDeletionPluginDef extends PluginDef { */ override def module: ModuleDef = new ModuleDef { make[ProjectDeletionConfig].fromEffect { - UIO.delay { + IO.delay { ConfigSource .fromConfig(pluginConfigObject) .loadOrThrow[ProjectDeletionConfig] diff --git a/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionRoutesSpec.scala b/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionRoutesSpec.scala index 1ee30edec1..49d1b94b76 100644 --- a/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionRoutesSpec.scala +++ b/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/ProjectDeletionRoutesSpec.scala @@ -24,8 +24,8 @@ class ProjectDeletionRoutesSpec implicit private val cl: ClassLoader = getClass.getClassLoader implicit private val ordering: JsonKeyOrdering = JsonKeyOrdering.default() implicit private val baseUri: BaseUri = BaseUri("http://localhost", Label.unsafe("v1")) - implicit private val rcr: RemoteContextResolution = RemoteContextResolution.fixed( - contexts.projectDeletion -> ContextValue.fromFile("contexts/project-deletion.json").accepted + implicit private val rcr: RemoteContextResolution = RemoteContextResolution.fixedIO( + contexts.projectDeletion -> ContextValue.fromFile("contexts/project-deletion.json") ) "A ProjectDeletionRoutes" should { diff --git a/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/model/ProjectDeletionConfigSpec.scala b/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/model/ProjectDeletionConfigSpec.scala index 75516f03ba..ddf6b1ab25 100644 --- a/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/model/ProjectDeletionConfigSpec.scala +++ b/delta/plugins/project-deletion/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/projectdeletion/model/ProjectDeletionConfigSpec.scala @@ -17,8 +17,8 @@ class ProjectDeletionConfigSpec extends AnyWordSpecLike with Matchers with IOVal implicit private val cl: ClassLoader = getClass.getClassLoader implicit private val api: JsonLdApi = JsonLdJavaApi.strict - implicit private val rcr: RemoteContextResolution = RemoteContextResolution.fixed( - contexts.projectDeletion -> ContextValue.fromFile("contexts/project-deletion.json").accepted + implicit private val rcr: RemoteContextResolution = RemoteContextResolution.fixedIO( + contexts.projectDeletion -> ContextValue.fromFile("contexts/project-deletion.json") ) "A ProjectDeletionConfig" should { diff --git a/delta/plugins/search/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/search/SearchPluginModule.scala b/delta/plugins/search/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/search/SearchPluginModule.scala index 9a2449c478..69768723c4 100644 --- a/delta/plugins/search/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/search/SearchPluginModule.scala +++ b/delta/plugins/search/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/search/SearchPluginModule.scala @@ -1,6 +1,5 @@ package ch.epfl.bluebrain.nexus.delta.plugins.search -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.plugins.compositeviews.CompositeViews import ch.epfl.bluebrain.nexus.delta.plugins.compositeviews.config.CompositeViewsConfig import ch.epfl.bluebrain.nexus.delta.plugins.compositeviews.indexing.CompositeProjectionLifeCycle @@ -20,7 +19,7 @@ import izumi.distage.model.definition.Id class SearchPluginModule(priority: Int) extends ModuleDef { - make[SearchConfig].fromEffect { (cfg: Config) => SearchConfig.load(cfg).toUIO } + make[SearchConfig].fromEffect { (cfg: Config) => SearchConfig.load(cfg) } make[Search].from { ( diff --git a/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginConfig.scala b/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginConfig.scala index 5f1b8a912a..946cdf9848 100644 --- a/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginConfig.scala +++ b/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginConfig.scala @@ -1,11 +1,12 @@ package ch.epfl.bluebrain.nexus.delta.plugins.storage +import cats.effect.IO +import cats.syntax.all._ +import ch.epfl.bluebrain.nexus.delta.kernel.Logger import ch.epfl.bluebrain.nexus.delta.plugins.storage.files.FilesConfig import ch.epfl.bluebrain.nexus.delta.plugins.storage.storages.StoragesConfig import ch.epfl.bluebrain.nexus.delta.sdk.Defaults import com.typesafe.config.Config -import com.typesafe.scalalogging.Logger -import monix.bio.UIO import pureconfig.generic.semiauto.deriveReader import pureconfig.{ConfigReader, ConfigSource} @@ -17,25 +18,25 @@ final case class StoragePluginConfig( object StoragePluginConfig { - private val logger: Logger = Logger[StoragePluginConfig] + private val logger = Logger.cats[StoragePluginConfig] /** * Converts a [[Config]] into an [[StoragePluginConfig]] */ - def load(config: Config): UIO[StoragePluginConfig] = - UIO + def load(config: Config): IO[StoragePluginConfig] = + IO .delay { ConfigSource .fromConfig(config) .at("plugins.storage") .loadOrThrow[StoragePluginConfig] } - .tapEval { config => - UIO.when(config.storages.storageTypeConfig.amazon.isDefined) { - UIO.delay(logger.info("Amazon S3 storage is enabled")) + .flatTap { config => + IO.whenA(config.storages.storageTypeConfig.amazon.isDefined) { + logger.info("Amazon S3 storage is enabled") } >> - UIO.when(config.storages.storageTypeConfig.remoteDisk.isDefined) { - UIO.delay(logger.info("Remote-disk storage is enabled")) + IO.whenA(config.storages.storageTypeConfig.remoteDisk.isDefined) { + logger.info("Remote-disk storage is enabled") } } diff --git a/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginModule.scala b/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginModule.scala index 2e582af7bb..1f590164e2 100644 --- a/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginModule.scala +++ b/delta/plugins/storage/src/main/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/StoragePluginModule.scala @@ -2,7 +2,8 @@ package ch.epfl.bluebrain.nexus.delta.plugins.storage import akka.actor import akka.actor.typed.ActorSystem -import cats.effect.Clock +import cats.effect.{Clock, IO} +import cats.syntax.all._ import ch.epfl.bluebrain.nexus.delta.kernel.utils.UUIDF import ch.epfl.bluebrain.nexus.delta.plugins.elasticsearch.client.ElasticSearchClient import ch.epfl.bluebrain.nexus.delta.plugins.elasticsearch.config.ElasticSearchViewsConfig @@ -46,7 +47,7 @@ import ch.epfl.bluebrain.nexus.delta.sourcing.model.Label import ch.epfl.bluebrain.nexus.delta.sourcing.stream.Supervisor import com.typesafe.config.Config import izumi.distage.model.definition.{Id, ModuleDef} -import monix.bio.{Task, UIO} +import monix.bio.UIO import monix.execution.Scheduler /** @@ -81,18 +82,20 @@ class StoragePluginModule(priority: Int) extends ModuleDef { ) => implicit val classicAs: actor.ActorSystem = as.classicSystem implicit val storageTypeConfig: StorageTypeConfig = cfg.storages.storageTypeConfig - Storages( - fetchContext.mapRejection(StorageRejection.ProjectContextRejection), - contextResolution, - permissions.fetchPermissionSet.toUIO, - StorageAccess.apply(_, _, remoteDiskStorageClient, storageTypeConfig), - xas, - cfg.storages, - serviceAccount - )( - api, - clock, - uuidF + toCatsIO( + Storages( + fetchContext.mapRejection(StorageRejection.ProjectContextRejection), + contextResolution, + permissions.fetchPermissionSet.toUIO, + StorageAccess.apply(_, _, remoteDiskStorageClient, storageTypeConfig), + xas, + cfg.storages, + serviceAccount + )( + api, + clock, + uuidF + ) ) } @@ -169,7 +172,7 @@ class StoragePluginModule(priority: Int) extends ModuleDef { remoteDiskStorageClient: RemoteDiskStorageClient, scheduler: Scheduler ) => - Task + IO .delay( Files( fetchContext.mapRejection(FileRejection.ProjectContextRejection), @@ -187,8 +190,8 @@ class StoragePluginModule(priority: Int) extends ModuleDef { as ) ) - .tapEval { files => - Files.startDigestStream(files, supervisor, storageTypeConfig) + .flatTap { files => + toCatsIO(Files.startDigestStream(files, supervisor, storageTypeConfig)) } } diff --git a/delta/plugins/storage/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/RemoteContextResolutionFixture.scala b/delta/plugins/storage/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/RemoteContextResolutionFixture.scala index b220ccfc0e..8d6e6a5ed6 100644 --- a/delta/plugins/storage/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/RemoteContextResolutionFixture.scala +++ b/delta/plugins/storage/src/test/scala/ch/epfl/bluebrain/nexus/delta/plugins/storage/RemoteContextResolutionFixture.scala @@ -12,13 +12,13 @@ trait RemoteContextResolutionFixture extends IOValues { implicit val api: JsonLdApi = JsonLdJavaApi.strict - implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixed( - storageContexts.storages -> ContextValue.fromFile("/contexts/storages.json").accepted, - storageContexts.storagesMetadata -> ContextValue.fromFile("/contexts/storages-metadata.json").accepted, - fileContexts.files -> ContextValue.fromFile("/contexts/files.json").accepted, - Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json").accepted, - Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json").accepted, - Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json").accepted, - Vocabulary.contexts.search -> ContextValue.fromFile("contexts/search.json").accepted + implicit val rcr: RemoteContextResolution = RemoteContextResolution.fixedIO( + storageContexts.storages -> ContextValue.fromFile("/contexts/storages.json"), + storageContexts.storagesMetadata -> ContextValue.fromFile("/contexts/storages-metadata.json"), + fileContexts.files -> ContextValue.fromFile("/contexts/files.json"), + Vocabulary.contexts.metadata -> ContextValue.fromFile("contexts/metadata.json"), + Vocabulary.contexts.error -> ContextValue.fromFile("contexts/error.json"), + Vocabulary.contexts.tags -> ContextValue.fromFile("contexts/tags.json"), + Vocabulary.contexts.search -> ContextValue.fromFile("contexts/search.json") ) } diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/graph/Graph.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/graph/Graph.scala index aa1de46c08..1c5832744f 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/graph/Graph.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/graph/Graph.scala @@ -7,6 +7,7 @@ import ch.epfl.bluebrain.nexus.delta.rdf.RdfError.{ConversionError, SparqlConstr import ch.epfl.bluebrain.nexus.delta.rdf.Triple.{obj, predicate, subject, Triple} import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.rdf import ch.epfl.bluebrain.nexus.delta.rdf._ +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.rdf.graph.Graph.{fakeId, rdfType} import ch.epfl.bluebrain.nexus.delta.rdf.implicits._ import ch.epfl.bluebrain.nexus.delta.rdf.jena.writer.DotWriter._ @@ -214,6 +215,7 @@ final case class Graph private (rootNode: IriOrBNode, value: DatasetGraph) { sel ctx = dotContext(rootResource, resolvedCtx) string <- ioTryOrRdfError(RDFWriter.create().lang(DOT).source(collapseGraphs).context(ctx).asString(), DOT.getName) + .toBIO[RdfError] } yield Dot(string, rootNode) /** diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/CompactedJsonLd.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/CompactedJsonLd.scala index 14156acb48..980a948c32 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/CompactedJsonLd.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/CompactedJsonLd.scala @@ -3,6 +3,7 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.{BNode, Iri} import ch.epfl.bluebrain.nexus.delta.rdf.graph.Graph import ch.epfl.bluebrain.nexus.delta.rdf.implicits._ +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api.{JsonLdApi, JsonLdOptions} import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.JsonLdContext.keywords import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context._ @@ -95,9 +96,12 @@ object CompactedJsonLd { contextValue: ContextValue, input: Json )(implicit api: JsonLdApi, rcr: RemoteContextResolution, opts: JsonLdOptions): IO[RdfError, CompactedJsonLd] = - api.compact(input, contextValue).map { compacted => - CompactedJsonLd(rootId, contextValue, compacted.remove(keywords.context)) - } + api + .compact(input, contextValue) + .map { compacted => + CompactedJsonLd(rootId, contextValue, compacted.remove(keywords.context)) + } + .toBIO[RdfError] /** * Creates a [[CompactedJsonLd]] document framed on the passed ''rootId''. @@ -116,9 +120,12 @@ object CompactedJsonLd { )(implicit api: JsonLdApi, rcr: RemoteContextResolution, opts: JsonLdOptions): IO[RdfError, CompactedJsonLd] = rootId.asIri.map(iri => contextValue.contextObj deepMerge JsonObject(keywords.id -> iri.asJson)) match { case Some(frame) => - api.frame(input, frame.asJson).map { compacted => - CompactedJsonLd(rootId, contextValue, compacted.remove(keywords.context)) - } + api + .frame(input, frame.asJson) + .map { compacted => + CompactedJsonLd(rootId, contextValue, compacted.remove(keywords.context)) + } + .toBIO[RdfError] case _ => apply(rootId, contextValue, input) } diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/ExpandedJsonLd.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/ExpandedJsonLd.scala index 6d1d93cf9d..89e90181aa 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/ExpandedJsonLd.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/ExpandedJsonLd.scala @@ -3,6 +3,7 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld import cats.implicits._ import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.{BNode, Iri} import ch.epfl.bluebrain.nexus.delta.rdf.RdfError.{InvalidIri, UnexpectedJsonLd} +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.nxv import ch.epfl.bluebrain.nexus.delta.rdf.graph.Graph import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api.{JsonLdApi, JsonLdOptions} @@ -192,21 +193,24 @@ object ExpandedJsonLd { resolution: RemoteContextResolution, opts: JsonLdOptions ): IO[RdfError, ExplainResult[ExpandedJsonLd]] = - api.explainExpand(input).flatMap { - case explain if explain.value.isEmpty => - // try to add a predicate and value in order for the expanded jsonld to at least detect the @id - for { - fallback <- api.explainExpand(input deepMerge Json.obj(fakeKey -> "fake".asJson)) - result <- fallback.evalMap { value => IO.fromEither(expanded(value).map(_.copy(obj = JsonObject.empty))) } - } yield result - case explain => - explain.evalMap { value => - expandedWithGraphSupport(value).map { - case (result, isGraph) if isGraph => ExpandedJsonLd(bNode, result.obj.remove(keywords.id)) - case (result, _) => result + api + .explainExpand(input) + .flatMap { + case explain if explain.value.isEmpty => + // try to add a predicate and value in order for the expanded jsonld to at least detect the @id + for { + fallback <- api.explainExpand(input deepMerge Json.obj(fakeKey -> "fake".asJson)) + result <- fallback.evalMap { value => IO.fromEither(expanded(value).map(_.copy(obj = JsonObject.empty))) } + } yield result + case explain => + explain.evalMap { value => + expandedWithGraphSupport(value).map { + case (result, isGraph) if isGraph => ExpandedJsonLd(bNode, result.obj.remove(keywords.id)) + case (result, _) => result + } } - } - } + } + .toBIO[RdfError] /** * Construct an [[ExpandedJsonLd]] from an existing sequence of [[ExpandedJsonLd]] merging the overriding fields. @@ -237,7 +241,10 @@ object ExpandedJsonLd { for { (expandedSeqFinal, isGraph) <- if (expandedSeq.size > 1) - api.expand(Json.obj(keywords.id -> graphId.asJson, keywords.graph -> expandedSeq.asJson)).map(_ -> true) + api + .expand(Json.obj(keywords.id -> graphId.asJson, keywords.graph -> expandedSeq.asJson)) + .map(_ -> true) + .toBIO[RdfError] else UIO.pure((expandedSeq, false)) result <- IO.fromEither(expanded(expandedSeqFinal)) diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdApi.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdApi.scala index 0cf9800e0d..0a059415c6 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdApi.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdApi.scala @@ -1,9 +1,9 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api +import cats.effect.IO import ch.epfl.bluebrain.nexus.delta.rdf.{ExplainResult, RdfError} import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, JsonLdContext, RemoteContextResolution} import io.circe.{Json, JsonObject} -import monix.bio.IO import org.apache.jena.sparql.core.DatasetGraph /** @@ -17,23 +17,23 @@ trait JsonLdApi { private[rdf] def compact( input: Json, ctx: ContextValue - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, JsonObject] + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[JsonObject] private[rdf] def expand( input: Json - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, Seq[JsonObject]] + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[Seq[JsonObject]] /** * Performs the expand operation and provides details on its execution */ private[rdf] def explainExpand( input: Json - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, ExplainResult[Seq[JsonObject]]] + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[ExplainResult[Seq[JsonObject]]] private[rdf] def frame( input: Json, frame: Json - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, JsonObject] + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[JsonObject] private[rdf] def toRdf(input: Json)(implicit opts: JsonLdOptions): Either[RdfError, DatasetGraph] @@ -41,6 +41,6 @@ trait JsonLdApi { private[rdf] def context( value: ContextValue - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, JsonLdContext] + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[JsonLdContext] } diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdJavaApi.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdJavaApi.scala index 83dbce09fc..64a2de8ed8 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdJavaApi.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/api/JsonLdJavaApi.scala @@ -1,5 +1,6 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api -import cats.implicits._ +import cats.effect.{ContextShift, IO} +import cats.syntax.all._ import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.Iri import ch.epfl.bluebrain.nexus.delta.rdf.{ExplainResult, RdfError} import ch.epfl.bluebrain.nexus.delta.rdf.RdfError.{ConversionError, RemoteContextCircularDependency, RemoteContextError, UnexpectedJsonLd, UnexpectedJsonLdContext} @@ -13,27 +14,27 @@ import com.github.jsonldjava.core.{Context, DocumentLoader, JsonLdError, JsonLdO import com.github.jsonldjava.utils.JsonUtils import io.circe.syntax._ import io.circe.{parser, Json, JsonObject} -import monix.bio.IO import org.apache.jena.query.DatasetFactory import org.apache.jena.riot.RDFFormat.{JSONLD_EXPAND_FLAT => EXPAND} import org.apache.jena.riot.system.ErrorHandlerFactory import org.apache.jena.riot._ import org.apache.jena.sparql.core.DatasetGraph +import scala.concurrent.ExecutionContext import scala.jdk.CollectionConverters._ import scala.util.Try /** * Json-LD high level API implementation by Json-LD Java library */ -final class JsonLdJavaApi(config: JsonLdApiConfig) extends JsonLdApi { +final class JsonLdJavaApi(config: JsonLdApiConfig)(implicit contextShift: ContextShift[IO]) extends JsonLdApi { System.setProperty(DocumentLoader.DISALLOW_REMOTE_CONTEXT_LOADING, "true") override private[rdf] def compact( input: Json, ctx: ContextValue - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, JsonObject] = + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[JsonObject] = for { obj <- ioTryOrRdfError(JsonUtils.fromString(input.noSpaces), "building input") ctxObj <- ioTryOrRdfError(JsonUtils.fromString(ctx.toString), "building context") @@ -44,12 +45,12 @@ final class JsonLdJavaApi(config: JsonLdApiConfig) extends JsonLdApi { override private[rdf] def expand( input: Json - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, Seq[JsonObject]] = + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[Seq[JsonObject]] = explainExpand(input).map(_.value) override private[rdf] def explainExpand( input: Json - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, ExplainResult[Seq[JsonObject]]] = + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[ExplainResult[Seq[JsonObject]]] = for { obj <- ioTryOrRdfError(JsonUtils.fromString(input.noSpaces), "building input") remoteContexts <- remoteContexts(input) @@ -61,7 +62,7 @@ final class JsonLdJavaApi(config: JsonLdApiConfig) extends JsonLdApi { override private[rdf] def frame( input: Json, frame: Json - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, JsonObject] = + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[JsonObject] = for { obj <- ioTryOrRdfError(JsonUtils.fromString(input.noSpaces), "building input") ff <- ioTryOrRdfError(JsonUtils.fromString(frame.noSpaces), "building frame") @@ -104,31 +105,30 @@ final class JsonLdJavaApi(config: JsonLdApiConfig) extends JsonLdApi { override private[rdf] def context( value: ContextValue - )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[RdfError, JsonLdContext] = + )(implicit opts: JsonLdOptions, rcr: RemoteContextResolution): IO[JsonLdContext] = for { dl <- documentLoader(value.contextObj.asJson) jOpts = toOpts(dl) ctx <- IO.fromTry(Try(new Context(jOpts).parse(JsonUtils.fromString(value.toString)))) - .mapError(err => UnexpectedJsonLdContext(err.getMessage)) + .adaptError { err => UnexpectedJsonLdContext(err.getMessage) } pm = ctx.getPrefixes(true).asScala.toMap.map { case (k, v) => k -> iri"$v" } aliases = (ctx.getPrefixes(false).asScala.toMap -- pm.keySet).map { case (k, v) => k -> iri"$v" } } yield JsonLdContext(value, getIri(ctx, keywords.base), getIri(ctx, keywords.vocab), aliases, pm) private def remoteContexts( jsons: Json* - )(implicit rcr: RemoteContextResolution): IO[RemoteContextError, Map[Iri, RemoteContext]] = - IO.parTraverseUnordered(jsons)(rcr(_)) - .bimap( - RemoteContextError, - _.foldLeft(Map.empty[Iri, RemoteContext])(_ ++ _) - ) + )(implicit rcr: RemoteContextResolution): IO[Map[Iri, RemoteContext]] = + jsons + .parTraverse(rcr(_)) + .adaptError { case r: RemoteContextResolutionError => RemoteContextError(r) } + .map(_.foldLeft(Map.empty[Iri, RemoteContext])(_ ++ _)) private def documentLoader(remoteContexts: Map[Iri, RemoteContext]): DocumentLoader = remoteContexts.foldLeft(new DocumentLoader()) { case (dl, (iri, ctx)) => dl.addInjectedDoc(iri.toString, ctx.value.contextObj.asJson.noSpaces) } - private def documentLoader(jsons: Json*)(implicit rcr: RemoteContextResolution): IO[RdfError, DocumentLoader] = + private def documentLoader(jsons: Json*)(implicit rcr: RemoteContextResolution): IO[DocumentLoader] = remoteContexts(jsons: _*).map(documentLoader) private def toOpts(dl: DocumentLoader = new DocumentLoader)(implicit options: JsonLdOptions): JsonLdJavaOptions = { @@ -176,18 +176,20 @@ object JsonLdJavaApi { /** * Creates an API with a config with strict values */ - val strict: JsonLdApi = new JsonLdJavaApi( - JsonLdApiConfig(strict = true, extraChecks = true, errorHandling = ErrorHandling.Strict) - ) + def strict(implicit contextShift: ContextShift[IO] = IO.contextShift(ExecutionContext.global)): JsonLdApi = + new JsonLdJavaApi( + JsonLdApiConfig(strict = true, extraChecks = true, errorHandling = ErrorHandling.Strict) + ) /** * Creates an API with a config with lenient values */ - val lenient: JsonLdApi = new JsonLdJavaApi( - JsonLdApiConfig(strict = false, extraChecks = false, errorHandling = ErrorHandling.NoWarning) - ) + def lenient(implicit contextShift: ContextShift[IO] = IO.contextShift(ExecutionContext.global)): JsonLdApi = + new JsonLdJavaApi( + JsonLdApiConfig(strict = false, extraChecks = false, errorHandling = ErrorHandling.NoWarning) + ) - private[rdf] def ioTryOrRdfError[A](value: => A, stage: String): IO[RdfError, A] = + private[rdf] def ioTryOrRdfError[A](value: => A, stage: String): IO[A] = IO.fromEither(tryOrRdfError(value, stage)) private[rdf] def tryOrRdfError[A](value: => A, stage: String): Either[RdfError, A] = diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/ContextValue.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/ContextValue.scala index cba3b1157c..8698c603f0 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/ContextValue.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/ContextValue.scala @@ -1,6 +1,7 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context -import ch.epfl.bluebrain.nexus.delta.kernel.utils.ClasspathResourceUtils +import cats.effect.IO +import ch.epfl.bluebrain.nexus.delta.kernel.utils.CatsEffectsClasspathResourceUtils import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.Iri import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.contexts import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.ContextValue.{ContextObject, ContextRemoteIri} @@ -9,7 +10,6 @@ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.decoder.JsonLdDecoder import ch.epfl.bluebrain.nexus.delta.rdf.syntax._ import io.circe.syntax._ import io.circe.{Decoder, Encoder, Json, JsonObject} -import monix.bio.UIO /** * The Json value of the @context key @@ -184,8 +184,8 @@ object ContextValue { /** * Loads a [[ContextValue]] form the passed ''resourcePath'' */ - final def fromFile(resourcePath: String)(implicit cl: ClassLoader): UIO[ContextValue] = - ClasspathResourceUtils.ioJsonContentOf(resourcePath).map(_.topContextValueOrEmpty).hideErrors + final def fromFile(resourcePath: String)(implicit cl: ClassLoader): IO[ContextValue] = + CatsEffectsClasspathResourceUtils.ioJsonContentOf(resourcePath).map(_.topContextValueOrEmpty) /** * Constructs a [[ContextValue]] from a json. The value of the json must be the value of the @context key diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/JsonLdContext.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/JsonLdContext.scala index 307f476cb6..ec7558cd93 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/JsonLdContext.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/JsonLdContext.scala @@ -1,6 +1,7 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.Iri +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.rdf.RdfError import ch.epfl.bluebrain.nexus.delta.rdf.implicits._ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api.{JsonLdApi, JsonLdOptions} @@ -222,7 +223,7 @@ object JsonLdContext { def apply( contextValue: ContextValue )(implicit api: JsonLdApi, resolution: RemoteContextResolution, opts: JsonLdOptions): IO[RdfError, JsonLdContext] = - api.context(contextValue) + api.context(contextValue).toBIO[RdfError] /** * @return diff --git a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/RemoteContextResolution.scala b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/RemoteContextResolution.scala index 9a1a668795..8a1e11ac8a 100644 --- a/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/RemoteContextResolution.scala +++ b/delta/rdf/src/main/scala/ch/epfl/bluebrain/nexus/delta/rdf/jsonld/context/RemoteContextResolution.scala @@ -1,15 +1,14 @@ package ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context -import ch.epfl.bluebrain.nexus.delta.kernel.utils.ClasspathResourceError +import cats.effect.{ContextShift, IO} +import cats.syntax.all._ import ch.epfl.bluebrain.nexus.delta.kernel.utils.ClasspathResourceError._ import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.Iri import ch.epfl.bluebrain.nexus.delta.rdf.implicits._ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.ContextValue._ -import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolution.Result -import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolutionError.{RemoteContextNotFound, RemoteContextWrongPayload} import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContext.StaticContext +import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolutionError.{RemoteContextNotFound, RemoteContextWrongPayload} import io.circe.Json -import monix.bio.IO trait RemoteContextResolution { self => @@ -19,7 +18,7 @@ trait RemoteContextResolution { self => * @return * the expected Json payload response from the passed ''iri'' */ - def resolve(iri: Iri): Result[RemoteContext] + def resolve(iri: Iri): IO[RemoteContext] /** * From a given ''json'', resolve all its remote context IRIs. @@ -28,19 +27,20 @@ trait RemoteContextResolution { self => * a Map where the keys are the IRIs resolved and the values the @context value from the payload of the resolved * wrapped in an IO */ - final def apply(json: Json): Result[Map[Iri, RemoteContext]] = { + final def apply(json: Json)(implicit contextShift: ContextShift[IO]): IO[Map[Iri, RemoteContext]] = { def inner( ctx: Set[ContextValue], resolved: Map[Iri, RemoteContext] = Map.empty - ): Result[Map[Iri, RemoteContext]] = { + ): IO[Map[Iri, RemoteContext]] = { val uris: Set[Iri] = ctx.flatMap(remoteIRIs).diff(resolved.keySet) for { - curResolved <- IO.parTraverseUnordered(uris)(uri => resolve(uri).map(uri -> _)) + curResolved <- uris.parUnorderedTraverse { uri => resolve(uri).map(uri -> _) } curResolvedMap = curResolved.toMap accResolved = curResolvedMap ++ resolved - recurseResolved <- - IO.parTraverseUnordered(curResolvedMap.values)(context => inner(Set(context.value), accResolved)) + recurseResolved <- curResolvedMap.values.toSet.parUnorderedTraverse { context => + inner(Set(context.value), accResolved) + } } yield recurseResolved.foldLeft(accResolved)(_ ++ _) } @@ -59,17 +59,15 @@ trait RemoteContextResolution { self => */ def merge(others: RemoteContextResolution*): RemoteContextResolution = (iri: Iri) => { - val tasks = self.resolve(iri) :: others.map(_.resolve(iri)).toList - IO.tailRecM(tasks) { + val ios = self.resolve(iri) :: others.map(_.resolve(iri)).toList + ios.tailRecM { case Nil => IO.raiseError(RemoteContextNotFound(iri)) // that never happens - case head :: Nil => head.map(Right.apply) - case head :: tail => head.map(Right.apply).onErrorFallbackTo(IO.pure(Left(tail))) + case head :: tail => head.attempt.map(_.leftMap(_ => tail)) } } } object RemoteContextResolution { - type Result[A] = IO[RemoteContextResolutionError, A] /** * Helper method to construct a [[RemoteContextResolution]] . @@ -77,10 +75,10 @@ object RemoteContextResolution { * @param f * a pair of [[Iri]] and the resolved Result of [[ContextValue]] */ - final def fixedIO(f: (Iri, Result[ContextValue])*): RemoteContextResolution = new RemoteContextResolution { + final def fixedIO(f: (Iri, IO[ContextValue])*): RemoteContextResolution = new RemoteContextResolution { private val map = f.toMap - override def resolve(iri: Iri): Result[RemoteContext] = + override def resolve(iri: Iri): IO[RemoteContext] = map.get(iri) match { case Some(result) => result.map { value => StaticContext(iri, value) } case None => IO.raiseError(RemoteContextNotFound(iri)) @@ -91,11 +89,11 @@ object RemoteContextResolution { * Helper method to construct a [[RemoteContextResolution]] . * * @param f - * a pair of [[Iri]] and the resolved [[ContextValue]] or a [[ClasspathResourceError]] + * a pair of [[Iri]] and the resolved [[ContextValue]] */ - final def fixedIOResource(f: (Iri, IO[ClasspathResourceError, ContextValue])*): RemoteContextResolution = + final def fixedIOResource(f: (Iri, IO[ContextValue])*): RemoteContextResolution = fixedIO(f.map { case (iri, io) => - iri -> io.memoizeOnSuccess.mapError { + iri -> io.adaptError { case _: InvalidJson | _: InvalidJsonObject => RemoteContextWrongPayload(iri) case _: ResourcePathNotFound => RemoteContextNotFound(iri) } diff --git a/delta/rdf/src/test/scala/ch/epfl/bluebrain/nexus/delta/rdf/Fixtures.scala b/delta/rdf/src/test/scala/ch/epfl/bluebrain/nexus/delta/rdf/Fixtures.scala index a337ecf306..f8ab73be3a 100644 --- a/delta/rdf/src/test/scala/ch/epfl/bluebrain/nexus/delta/rdf/Fixtures.scala +++ b/delta/rdf/src/test/scala/ch/epfl/bluebrain/nexus/delta/rdf/Fixtures.scala @@ -7,6 +7,7 @@ import ch.epfl.bluebrain.nexus.delta.rdf.implicits._ import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.api.{JsonLdApi, JsonLdJavaApi} import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContextResolution} import ch.epfl.bluebrain.nexus.testkit._ +import ch.epfl.bluebrain.nexus.testkit.ce.CatsIOValues import monix.execution.schedulers.CanBlock import org.scalatest.OptionValues @@ -15,6 +16,7 @@ trait Fixtures with CirceLiteral with OptionValues with IOValues + with CatsIOValues with EitherValuable with TestMatchers { diff --git a/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/auth/AuthTokenProvider.scala b/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/auth/AuthTokenProvider.scala index 2e84f0b0f6..1522261577 100644 --- a/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/auth/AuthTokenProvider.scala +++ b/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/auth/AuthTokenProvider.scala @@ -3,11 +3,9 @@ package ch.epfl.bluebrain.nexus.delta.sdk.auth import cats.effect.{Clock, IO} import ch.epfl.bluebrain.nexus.delta.kernel.Logger import ch.epfl.bluebrain.nexus.delta.kernel.cache.LocalCache -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration.MigrateEffectSyntax import ch.epfl.bluebrain.nexus.delta.kernel.jwt.{AuthToken, ParsedToken} import ch.epfl.bluebrain.nexus.delta.kernel.utils.IOInstant import ch.epfl.bluebrain.nexus.delta.sdk.auth.Credentials.ClientCredentials -import monix.bio import java.time.{Duration, Instant} @@ -19,10 +17,9 @@ trait AuthTokenProvider { } object AuthTokenProvider { - def apply(authService: OpenIdAuthService)(implicit clock: Clock[IO]): bio.UIO[AuthTokenProvider] = { + def apply(authService: OpenIdAuthService)(implicit clock: Clock[IO]): IO[AuthTokenProvider] = { LocalCache[ClientCredentials, ParsedToken]() .map(cache => new CachingOpenIdAuthTokenProvider(authService, cache)) - .toBIO } def anonymousForTest: AuthTokenProvider = new AnonymousAuthTokenProvider def fixedForTest(token: String): AuthTokenProvider = new AuthTokenProvider { @@ -43,8 +40,7 @@ private class CachingOpenIdAuthTokenProvider( cache: LocalCache[ClientCredentials, ParsedToken] )(implicit clock: Clock[IO] -) extends AuthTokenProvider - with MigrateEffectSyntax { +) extends AuthTokenProvider { private val logger = Logger.cats[CachingOpenIdAuthTokenProvider] diff --git a/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/model/MetadataContextValue.scala b/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/model/MetadataContextValue.scala index 89f075e13e..0d28a201bf 100644 --- a/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/model/MetadataContextValue.scala +++ b/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/model/MetadataContextValue.scala @@ -1,8 +1,7 @@ package ch.epfl.bluebrain.nexus.delta.sdk.model -import ch.epfl.bluebrain.nexus.delta.kernel.utils.ClasspathResourceError +import cats.effect.IO import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.ContextValue -import monix.bio.IO /** * A [[ContextValue]] that is specialized for metadata @@ -29,6 +28,6 @@ object MetadataContextValue { /** * Loads a [[MetadataContextValue]] form the passed ''resourcePath'' */ - final def fromFile(resourcePath: String)(implicit cl: ClassLoader): IO[ClasspathResourceError, MetadataContextValue] = + final def fromFile(resourcePath: String)(implicit cl: ClassLoader): IO[MetadataContextValue] = ContextValue.fromFile(resourcePath).map(MetadataContextValue.apply) } diff --git a/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolution.scala b/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolution.scala index 068ae7d24d..d695f1cf75 100644 --- a/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolution.scala +++ b/delta/sdk/src/main/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolution.scala @@ -3,11 +3,9 @@ package ch.epfl.bluebrain.nexus.delta.sdk.resolvers import cats.effect.IO import cats.syntax.all._ import ch.epfl.bluebrain.nexus.delta.kernel.Logger -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.Iri -import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolution.Result import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.RemoteContextResolutionError.RemoteContextNotAccessible -import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContext, RemoteContextResolution, RemoteContextResolutionError} +import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContext, RemoteContextResolution} import ch.epfl.bluebrain.nexus.delta.sdk._ import ch.epfl.bluebrain.nexus.delta.sdk.acls.AclCheck import ch.epfl.bluebrain.nexus.delta.sdk.identities.model.Caller @@ -38,11 +36,12 @@ final class ResolverContextResolution(val rcr: RemoteContextResolution, resolveR // The instance is living inside the scope of a request so we can cache the resolutions private val cache: concurrent.Map[Iri, RemoteContext] = new concurrent.TrieMap - override def resolve(iri: Iri): Result[RemoteContext] = { + override def resolve(iri: Iri): IO[RemoteContext] = { IO.pure(cache.get(iri)).flatMap { case Some(s) => IO.pure(s) case None => - toCatsIO(rcr.resolve(iri)) + rcr + .resolve(iri) .handleErrorWith(_ => resolveResource(ResourceRef(iri), projectRef, caller).flatMap { case Left(report) => @@ -61,7 +60,7 @@ final class ResolverContextResolution(val rcr: RemoteContextResolution, resolveR logger.debug(s"Iri $iri has been resolved for project $projectRef and caller $caller.subject") } } - }.toBIO[RemoteContextResolutionError] + } } } diff --git a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/SerializationSuite.scala b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/SerializationSuite.scala index 33ed1b0000..1b228dc42f 100644 --- a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/SerializationSuite.scala +++ b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/SerializationSuite.scala @@ -30,8 +30,8 @@ abstract class SerializationSuite implicit def res: RemoteContextResolution = RemoteContextResolution.fixed( - contexts.shacl -> ContextValue.fromFile("contexts/shacl.json").runSyncUnsafe(), - contexts.schemasMetadata -> ContextValue.fromFile("contexts/schemas-metadata.json").runSyncUnsafe() + contexts.shacl -> ContextValue.fromFile("contexts/shacl.json").unsafeRunSync(), + contexts.schemasMetadata -> ContextValue.fromFile("contexts/schemas-metadata.json").unsafeRunSync() ) def loadEvents[E](module: String, eventsToFile: (E, String)*): Map[E, (Json, JsonObject)] = diff --git a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/directives/AuthDirectivesSpec.scala b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/directives/AuthDirectivesSpec.scala index f3169950b8..0375474b50 100644 --- a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/directives/AuthDirectivesSpec.scala +++ b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/directives/AuthDirectivesSpec.scala @@ -6,6 +6,7 @@ import akka.http.scaladsl.server.Directives._ import akka.http.scaladsl.server.{ExceptionHandler, Route} import cats.effect.IO import ch.epfl.bluebrain.nexus.delta.kernel.jwt.AuthToken +import ch.epfl.bluebrain.nexus.delta.kernel.jwt.TokenRejection.InvalidAccessToken import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.contexts import ch.epfl.bluebrain.nexus.delta.rdf.jsonld.context.{ContextValue, RemoteContextResolution} import ch.epfl.bluebrain.nexus.delta.rdf.utils.JsonKeyOrdering @@ -13,20 +14,21 @@ import ch.epfl.bluebrain.nexus.delta.sdk.acls.AclSimpleCheck import ch.epfl.bluebrain.nexus.delta.sdk.acls.model.AclAddress import ch.epfl.bluebrain.nexus.delta.sdk.error.ServiceError.AuthorizationFailed import ch.epfl.bluebrain.nexus.delta.sdk.identities.Identities -import ch.epfl.bluebrain.nexus.delta.sdk.identities.model.Caller.Anonymous -import ch.epfl.bluebrain.nexus.delta.kernel.jwt.TokenRejection.InvalidAccessToken +import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import ch.epfl.bluebrain.nexus.delta.sdk.identities.model.Caller +import ch.epfl.bluebrain.nexus.delta.sdk.identities.model.Caller.Anonymous import ch.epfl.bluebrain.nexus.delta.sdk.marshalling.RdfExceptionHandler import ch.epfl.bluebrain.nexus.delta.sdk.model.BaseUri import ch.epfl.bluebrain.nexus.delta.sdk.permissions.model.Permission import ch.epfl.bluebrain.nexus.delta.sdk.utils.RouteHelpers import ch.epfl.bluebrain.nexus.delta.sourcing.model.Identity.{Subject, User} import ch.epfl.bluebrain.nexus.delta.sourcing.model.Label -import ch.epfl.bluebrain.nexus.testkit.{IOValues, TestHelpers} +import ch.epfl.bluebrain.nexus.testkit.TestHelpers +import ch.epfl.bluebrain.nexus.testkit.ce.CatsIOValues import monix.execution.Scheduler.Implicits.global import org.scalatest.matchers.should.Matchers -class AuthDirectivesSpec extends RouteHelpers with TestHelpers with Matchers with IOValues { +class AuthDirectivesSpec extends RouteHelpers with TestHelpers with Matchers with CatsIOValues { implicit private val cl: ClassLoader = getClass.getClassLoader @@ -57,9 +59,7 @@ class AuthDirectivesSpec extends RouteHelpers with TestHelpers with Matchers wit } } - val aclCheck = AclSimpleCheck( - (user, AclAddress.Root, Set(permission)) - ).accepted + val aclCheck = toCatsIO(AclSimpleCheck((user, AclAddress.Root, Set(permission)))).accepted val directives = new AuthDirectives(identities, aclCheck) {} diff --git a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolutionSuite.scala b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolutionSuite.scala index ee99041de8..200f36bb48 100644 --- a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolutionSuite.scala +++ b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/resolvers/ResolverContextResolutionSuite.scala @@ -1,7 +1,6 @@ package ch.epfl.bluebrain.nexus.delta.sdk.resolvers import akka.http.scaladsl.model.Uri -import ch.epfl.bluebrain.nexus.delta.kernel.effect.migration._ import cats.effect.IO import ch.epfl.bluebrain.nexus.delta.rdf.IriOrBNode.Iri import ch.epfl.bluebrain.nexus.delta.rdf.Vocabulary.{contexts, nxv, schemas} @@ -75,8 +74,7 @@ class ResolverContextResolutionSuite extends CatsEffectSuite with TestHelpers { private val resolverContextResolution = ResolverContextResolution(rcr, resourceResolution) - private def resolve(iri: Iri) = - toCatsIO(resolverContextResolution(project).resolve(iri)) + private def resolve(iri: Iri) = resolverContextResolution(project).resolve(iri) test("Resolve correctly static contexts") { val expected = StaticContext(contexts.metadata, metadataContext) diff --git a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/utils/Fixtures.scala b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/utils/Fixtures.scala index b7caf819a8..619b784e2a 100644 --- a/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/utils/Fixtures.scala +++ b/delta/sdk/src/test/scala/ch/epfl/bluebrain/nexus/delta/sdk/utils/Fixtures.scala @@ -11,14 +11,14 @@ trait Fixtures extends IOValues { implicit val api: JsonLdApi = JsonLdJavaApi.strict implicit val rcr: RemoteContextResolution = - RemoteContextResolution.fixed( - contexts.shacl -> ContextValue.fromFile("contexts/shacl.json").accepted, - contexts.schemasMetadata -> ContextValue.fromFile("contexts/schemas-metadata.json").accepted, - contexts.error -> ContextValue.fromFile(("contexts/error.json")).accepted, - contexts.metadata -> ContextValue.fromFile(("contexts/metadata.json")).accepted, - contexts.permissions -> ContextValue.fromFile(("contexts/permissions.json")).accepted, - contexts.organizations -> ContextValue.fromFile(("contexts/organizations.json")).accepted, - contexts.resolvers -> ContextValue.fromFile(("contexts/resolvers.json")).accepted + RemoteContextResolution.fixedIO( + contexts.shacl -> ContextValue.fromFile("contexts/shacl.json"), + contexts.schemasMetadata -> ContextValue.fromFile("contexts/schemas-metadata.json"), + contexts.error -> ContextValue.fromFile("contexts/error.json"), + contexts.metadata -> ContextValue.fromFile("contexts/metadata.json"), + contexts.permissions -> ContextValue.fromFile("contexts/permissions.json"), + contexts.organizations -> ContextValue.fromFile("contexts/organizations.json"), + contexts.resolvers -> ContextValue.fromFile("contexts/resolvers.json") ) }